getValByRegex(String regex) {
item.getValue() instanceof String) {
m = p.matcher((String)item.getKey());
if(m.find()) { // match
- result.put((String) item.getKey(),
- substituteVars(getProps().getProperty((String) item.getKey())));
+ resultKeys.add((String) item.getKey());
}
}
}
+ resultKeys.forEach(item ->
+ result.put(item, substituteVars(getProps().getProperty(item))));
return result;
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/ReconfigurableBase.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/ReconfigurableBase.java
index 8cacbdcdac039..0352025ada154 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/ReconfigurableBase.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/ReconfigurableBase.java
@@ -18,9 +18,9 @@
package org.apache.hadoop.conf;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Maps;
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Maps;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.conf.ReconfigurationUtil.PropertyChange;
import org.slf4j.Logger;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/AesCtrCryptoCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/AesCtrCryptoCodec.java
index 3e52560259638..a1ddca6e20967 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/AesCtrCryptoCodec.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/AesCtrCryptoCodec.java
@@ -20,7 +20,7 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
-import com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import java.io.IOException;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoCodec.java
index bcf4a65ec24d4..3061020f1e6fd 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoCodec.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoCodec.java
@@ -31,8 +31,8 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import com.google.common.base.Splitter;
-import com.google.common.collect.Lists;
+import org.apache.hadoop.thirdparty.com.google.common.base.Splitter;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX;
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_CIPHER_SUITE_KEY;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoInputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoInputStream.java
index 9e601e26cf944..0c156e3548d21 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoInputStream.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoInputStream.java
@@ -30,7 +30,7 @@
import java.util.Queue;
import java.util.concurrent.ConcurrentLinkedQueue;
-import com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.ByteBufferPositionedReadable;
@@ -46,9 +46,13 @@
import org.apache.hadoop.fs.Seekable;
import org.apache.hadoop.fs.StreamCapabilities;
import org.apache.hadoop.fs.StreamCapabilitiesPolicy;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
import org.apache.hadoop.io.ByteBufferPool;
import org.apache.hadoop.util.StringUtils;
+import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics;
+
/**
* CryptoInputStream decrypts data. It is not thread-safe. AES CTR mode is
* required in order to ensure that the plain text and cipher text have a 1:1
@@ -66,7 +70,7 @@ public class CryptoInputStream extends FilterInputStream implements
Seekable, PositionedReadable, ByteBufferReadable, HasFileDescriptor,
CanSetDropBehind, CanSetReadahead, HasEnhancedByteBufferAccess,
ReadableByteChannel, CanUnbuffer, StreamCapabilities,
- ByteBufferPositionedReadable {
+ ByteBufferPositionedReadable, IOStatisticsSource {
private final byte[] oneByteBuf = new byte[1];
private final CryptoCodec codec;
private final Decryptor decryptor;
@@ -867,8 +871,16 @@ public boolean hasCapability(String capability) {
+ " does not expose its stream capabilities.");
}
return ((StreamCapabilities) in).hasCapability(capability);
+ case StreamCapabilities.IOSTATISTICS:
+ return (in instanceof StreamCapabilities)
+ && ((StreamCapabilities) in).hasCapability(capability);
default:
return false;
}
}
+
+ @Override
+ public IOStatistics getIOStatistics() {
+ return retrieveIOStatistics(in);
+ }
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java
index 8d11043937612..38c430fcd99c0 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java
@@ -28,8 +28,13 @@
import org.apache.hadoop.fs.CanSetDropBehind;
import org.apache.hadoop.fs.StreamCapabilities;
import org.apache.hadoop.fs.Syncable;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+import org.apache.hadoop.fs.impl.StoreImplementationUtils;
-import com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+
+import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics;
/**
* CryptoOutputStream encrypts data. It is not thread-safe. AES CTR mode is
@@ -48,7 +53,7 @@
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class CryptoOutputStream extends FilterOutputStream implements
- Syncable, CanSetDropBehind, StreamCapabilities {
+ Syncable, CanSetDropBehind, StreamCapabilities, IOStatisticsSource {
private final byte[] oneByteBuf = new byte[1];
private final CryptoCodec codec;
private final Encryptor encryptor;
@@ -308,9 +313,11 @@ private void freeBuffers() {
@Override
public boolean hasCapability(String capability) {
- if (out instanceof StreamCapabilities) {
- return ((StreamCapabilities) out).hasCapability(capability);
- }
- return false;
+ return StoreImplementationUtils.hasCapability(out, capability);
+ }
+
+ @Override
+ public IOStatistics getIOStatistics() {
+ return retrieveIOStatistics(out);
}
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoStreamUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoStreamUtils.java
index b55f84226d3cd..9d2b4b9e23389 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoStreamUtils.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoStreamUtils.java
@@ -29,7 +29,7 @@
import org.apache.hadoop.fs.Seekable;
import org.apache.hadoop.util.CleanerUtil;
-import com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/JceAesCtrCryptoCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/JceAesCtrCryptoCodec.java
index de0e5dd6268f2..1c670f76f4859 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/JceAesCtrCryptoCodec.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/JceAesCtrCryptoCodec.java
@@ -29,7 +29,7 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
-import com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/OpensslAesCtrCryptoCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/OpensslAesCtrCryptoCodec.java
index a127925a7a538..0963cb6005ed8 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/OpensslAesCtrCryptoCodec.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/OpensslAesCtrCryptoCodec.java
@@ -29,7 +29,7 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
-import com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import org.apache.hadoop.crypto.random.OpensslSecureRandom;
import org.apache.hadoop.util.ReflectionUtils;
import org.slf4j.Logger;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/OpensslCipher.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/OpensslCipher.java
index 0a2ba52e555e5..0fe81d7ca1087 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/OpensslCipher.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/OpensslCipher.java
@@ -29,7 +29,7 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.util.NativeCodeLoader;
-import com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import org.apache.hadoop.util.PerformanceAdvisory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/CachingKeyProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/CachingKeyProvider.java
index 7a66e1e4bab48..4f456e54a4d61 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/CachingKeyProvider.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/CachingKeyProvider.java
@@ -22,9 +22,9 @@
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
+import org.apache.hadoop.thirdparty.com.google.common.cache.CacheBuilder;
+import org.apache.hadoop.thirdparty.com.google.common.cache.CacheLoader;
+import org.apache.hadoop.thirdparty.com.google.common.cache.LoadingCache;
/**
* A KeyProviderExtension implementation providing a short lived
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/JavaKeyStoreProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/JavaKeyStoreProvider.java
index 7951af56bc8f9..3c3099e113567 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/JavaKeyStoreProvider.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/JavaKeyStoreProvider.java
@@ -18,7 +18,7 @@
package org.apache.hadoop.crypto.key;
-import com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
@@ -32,7 +32,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
import javax.crypto.spec.SecretKeySpec;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java
index 00d7a7dfce0f7..8aa64e2ceb6ae 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java
@@ -29,7 +29,7 @@
import javax.crypto.spec.IvParameterSpec;
import javax.crypto.spec.SecretKeySpec;
-import com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.crypto.CryptoCodec;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderDelegationTokenExtension.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderDelegationTokenExtension.java
index 05d99ed0810fc..e23d8b8e4a774 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderDelegationTokenExtension.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderDelegationTokenExtension.java
@@ -17,7 +17,7 @@
*/
package org.apache.hadoop.crypto.key;
-import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.security.Credentials;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyShell.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyShell.java
index f2f6f1801c2dd..2cc011c0df34d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyShell.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyShell.java
@@ -25,7 +25,7 @@
import java.util.List;
import java.util.Map;
-import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.commons.lang3.StringUtils;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
index 71ed4557b357b..bc56f0e28676b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
@@ -79,9 +79,9 @@
import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.CryptoExtension;
import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Strings;
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.base.Strings;
import static org.apache.hadoop.util.KMSUtil.checkNotEmpty;
import static org.apache.hadoop.util.KMSUtil.checkNotNull;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java
index ee2295cff77f3..4d19ea32e7fc6 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java
@@ -50,8 +50,8 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
/**
* A simple LoadBalancing KMSClientProvider that round-robins requests
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/ValueQueue.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/ValueQueue.java
index 7d26acbf21a03..5ec78b3bd50ba 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/ValueQueue.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/ValueQueue.java
@@ -33,11 +33,11 @@
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
-import com.google.common.base.Preconditions;
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.cache.CacheBuilder;
+import org.apache.hadoop.thirdparty.com.google.common.cache.CacheLoader;
+import org.apache.hadoop.thirdparty.com.google.common.cache.LoadingCache;
+import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.hadoop.classification.InterfaceAudience;
/**
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/random/OpensslSecureRandom.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/random/OpensslSecureRandom.java
index 1863f5ec2035f..a7a609ce440b6 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/random/OpensslSecureRandom.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/random/OpensslSecureRandom.java
@@ -22,7 +22,7 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.util.NativeCodeLoader;
-import com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import org.apache.hadoop.util.PerformanceAdvisory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Abortable.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Abortable.java
new file mode 100644
index 0000000000000..d2fd174795831
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Abortable.java
@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Abort data being written to a stream, so that close() does
+ * not write the data. It is implemented by output streams in
+ * some object stores, and passed through {@link FSDataOutputStream}.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public interface Abortable {
+
+ /**
+ * Abort the active operation without the output becoming visible.
+ *
+ * This is to provide ability to cancel the write on stream; once
+ * a stream is aborted, the write MUST NOT become visible.
+ *
+ * @throws UnsupportedOperationException if the operation is not supported.
+ * @return the result.
+ */
+ AbortableResult abort();
+
+ /**
+ * Interface for the result of aborts; allows subclasses to extend
+ * (IOStatistics etc) or for future enhancements if ever needed.
+ */
+ interface AbortableResult {
+
+ /**
+ * Was the stream already closed/aborted?
+ * @return true if a close/abort operation had already
+ * taken place.
+ */
+ boolean alreadyClosed();
+
+ /**
+ * Any exception caught during cleanup operations,
+ * exceptions whose raising/catching does not change
+ * the semantics of the abort.
+ * @return an exception or null.
+ */
+ IOException anyCleanupException();
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java
index 1df68b647c99a..b911e0d508422 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java
@@ -56,7 +56,7 @@
import org.apache.hadoop.util.LambdaUtils;
import org.apache.hadoop.util.Progressable;
-import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -865,6 +865,20 @@ public abstract FileStatus getFileStatus(final Path f)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException;
+ /**
+ * Synchronize client metadata state.
+ *
+ * In some FileSystem implementations such as HDFS metadata
+ * synchronization is essential to guarantee consistency of read requests
+ * particularly in HA setting.
+ * @throws IOException
+ * @throws UnsupportedOperationException
+ */
+ public void msync() throws IOException, UnsupportedOperationException {
+ throw new UnsupportedOperationException(getClass().getCanonicalName() +
+ " does not support method msync");
+ }
+
/**
* The specification of this method matches that of
* {@link FileContext#access(Path, FsAction)}
@@ -1383,4 +1397,34 @@ public boolean hasPathCapability(final Path path,
return false;
}
}
+
+ /**
+ * Create a multipart uploader.
+ * @param basePath file path under which all files are uploaded
+ * @return a MultipartUploaderBuilder object to build the uploader
+ * @throws IOException if some early checks cause IO failures.
+ * @throws UnsupportedOperationException if support is checked early.
+ */
+ @InterfaceStability.Unstable
+ public MultipartUploaderBuilder createMultipartUploader(Path basePath)
+ throws IOException {
+ methodNotSupported();
+ return null;
+ }
+
+ /**
+ * Helper method that throws an {@link UnsupportedOperationException} for the
+ * current {@link FileSystem} method being called.
+ */
+ protected final void methodNotSupported() {
+ // The order of the stacktrace elements is (from top to bottom):
+ // - java.lang.Thread.getStackTrace
+ // - org.apache.hadoop.fs.FileSystem.methodNotSupported
+ // -
+ // therefore, to find out the current method name, we use the element at
+ // index 2.
+ String name = Thread.currentThread().getStackTrace()[2].getMethodName();
+ throw new UnsupportedOperationException(getClass().getCanonicalName() +
+ " does not support method " + name);
+ }
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BufferedFSInputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BufferedFSInputStream.java
index 973b136bb3ab2..59345f5d25caf 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BufferedFSInputStream.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BufferedFSInputStream.java
@@ -21,9 +21,14 @@
import java.io.EOFException;
import java.io.FileDescriptor;
import java.io.IOException;
+import java.util.StringJoiner;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+
+import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics;
/**
@@ -33,7 +38,8 @@
@InterfaceAudience.Private
@InterfaceStability.Unstable
public class BufferedFSInputStream extends BufferedInputStream
-implements Seekable, PositionedReadable, HasFileDescriptor {
+ implements Seekable, PositionedReadable, HasFileDescriptor,
+ IOStatisticsSource, StreamCapabilities {
/**
* Creates a BufferedFSInputStream
* with the specified buffer size,
@@ -126,4 +132,34 @@ public FileDescriptor getFileDescriptor() throws IOException {
return null;
}
}
+
+ /**
+ * If the inner stream supports {@link StreamCapabilities},
+ * forward the probe to it.
+ * Otherwise: return false.
+ *
+ * @param capability string to query the stream support for.
+ * @return true if a capability is known to be supported.
+ */
+ @Override
+ public boolean hasCapability(final String capability) {
+ if (in instanceof StreamCapabilities) {
+ return ((StreamCapabilities) in).hasCapability(capability);
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public IOStatistics getIOStatistics() {
+ return retrieveIOStatistics(in);
+ }
+
+ @Override
+ public String toString() {
+ return new StringJoiner(", ",
+ BufferedFSInputStream.class.getSimpleName() + "[", "]")
+ .add("in=" + in)
+ .toString();
+ }
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ByteBufferUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ByteBufferUtil.java
index c31c29b5b6d31..5708c906fe764 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ByteBufferUtil.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ByteBufferUtil.java
@@ -26,7 +26,7 @@
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.io.ByteBufferPool;
-import com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
@InterfaceAudience.Private
@InterfaceStability.Evolving
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CanSetDropBehind.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CanSetDropBehind.java
index 2e2d98b9c5462..0077838920a9e 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CanSetDropBehind.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CanSetDropBehind.java
@@ -36,6 +36,6 @@ public interface CanSetDropBehind {
* UnsupportedOperationException If this stream doesn't support
* setting the drop-behind.
*/
- public void setDropBehind(Boolean dropCache)
+ void setDropBehind(Boolean dropCache)
throws IOException, UnsupportedOperationException;
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java
index cc9c284c9fa55..0256a58f46368 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java
@@ -29,7 +29,7 @@
import java.util.List;
import java.util.concurrent.CompletableFuture;
-import com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
@@ -38,11 +38,15 @@
import org.apache.hadoop.fs.impl.OpenFileParameters;
import org.apache.hadoop.fs.permission.AclEntry;
import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+import org.apache.hadoop.fs.statistics.IOStatisticsSupport;
import org.apache.hadoop.util.DataChecksum;
import org.apache.hadoop.util.LambdaUtils;
import org.apache.hadoop.util.Progressable;
import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs;
+import static org.apache.hadoop.fs.impl.StoreImplementationUtils.isProbeForSyncable;
/****************************************************************
* Abstract Checksumed FileSystem.
@@ -134,7 +138,8 @@ private int getSumBufferSize(int bytesPerSum, int bufferSize) {
* For open()'s FSInputStream
* It verifies that data matches checksums.
*******************************************************/
- private static class ChecksumFSInputChecker extends FSInputChecker {
+ private static class ChecksumFSInputChecker extends FSInputChecker implements
+ IOStatisticsSource {
private ChecksumFileSystem fs;
private FSDataInputStream datas;
private FSDataInputStream sums;
@@ -270,6 +275,17 @@ protected int readChunk(long pos, byte[] buf, int offset, int len,
}
return nread;
}
+
+ /**
+ * Get the IO Statistics of the nested stream, falling back to
+ * null if the stream does not implement the interface
+ * {@link IOStatisticsSource}.
+ * @return an IOStatistics instance or null
+ */
+ @Override
+ public IOStatistics getIOStatistics() {
+ return IOStatisticsSupport.retrieveIOStatistics(datas);
+ }
}
private static class FSDataBoundedInputStream extends FSDataInputStream {
@@ -395,7 +411,8 @@ public static long getChecksumLength(long size, int bytesPerSum) {
/** This class provides an output stream for a checksummed file.
* It generates checksums for data. */
- private static class ChecksumFSOutputSummer extends FSOutputSummer {
+ private static class ChecksumFSOutputSummer extends FSOutputSummer
+ implements IOStatisticsSource, StreamCapabilities {
private FSDataOutputStream datas;
private FSDataOutputStream sums;
private static final float CHKSUM_AS_FRACTION = 0.01f;
@@ -449,6 +466,31 @@ protected void checkClosed() throws IOException {
throw new ClosedChannelException();
}
}
+
+ /**
+ * Get the IO Statistics of the nested stream, falling back to
+ * null if the stream does not implement the interface
+ * {@link IOStatisticsSource}.
+ * @return an IOStatistics instance or null
+ */
+ @Override
+ public IOStatistics getIOStatistics() {
+ return IOStatisticsSupport.retrieveIOStatistics(datas);
+ }
+
+ /**
+ * Probe the inner stream for a capability.
+ * Syncable operations are rejected before being passed down.
+ * @param capability string to query the stream support for.
+ * @return true if a capability is known to be supported.
+ */
+ @Override
+ public boolean hasCapability(final String capability) {
+ if (isProbeForSyncable(capability)) {
+ return false;
+ }
+ return datas.hasCapability(capability);
+ }
}
@Override
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
index a68012b06d2bc..0044b6ac6c317 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
@@ -164,6 +164,27 @@ public class CommonConfigurationKeysPublic {
public static final String FS_AUTOMATIC_CLOSE_KEY = "fs.automatic.close";
/** Default value for FS_AUTOMATIC_CLOSE_KEY */
public static final boolean FS_AUTOMATIC_CLOSE_DEFAULT = true;
+
+ /**
+ * Number of filesystems instances can be created in parallel.
+ *
+ * A higher number here does not necessarily improve performance, especially
+ * for object stores, where multiple threads may be attempting to create an FS
+ * instance for the same URI.
+ *
+ * Default value: {@value}.
+ */
+ public static final String FS_CREATION_PARALLEL_COUNT =
+ "fs.creation.parallel.count";
+
+ /**
+ * Default value for {@link #FS_CREATION_PARALLEL_COUNT}.
+ *
+ * Default value: {@value}.
+ */
+ public static final int FS_CREATION_PARALLEL_COUNT_DEFAULT =
+ 64;
+
/**
* @see
*
@@ -988,5 +1009,14 @@ public class CommonConfigurationKeysPublic {
public static final String HADOOP_PROMETHEUS_ENABLED =
"hadoop.prometheus.endpoint.enabled";
public static final boolean HADOOP_PROMETHEUS_ENABLED_DEFAULT = false;
+
+ /**
+ * @see
+ *
+ * core-default.xml
+ */
+ public static final String HADOOP_HTTP_IDLE_TIMEOUT_MS_KEY =
+ "hadoop.http.idle_timeout.ms";
+ public static final int HADOOP_HTTP_IDLE_TIMEOUT_MS_DEFAULT = 60000;
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonPathCapabilities.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonPathCapabilities.java
index fb46ef81e36fa..df932df43aebd 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonPathCapabilities.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonPathCapabilities.java
@@ -131,4 +131,19 @@ private CommonPathCapabilities() {
@InterfaceStability.Unstable
public static final String FS_EXPERIMENTAL_BATCH_LISTING =
"fs.capability.batch.listing";
+
+ /**
+ * Does the store support multipart uploading?
+ * Value: {@value}.
+ */
+ public static final String FS_MULTIPART_UPLOADER =
+ "fs.capability.multipart.uploader";
+
+
+ /**
+ * Stream abort() capability implemented by {@link Abortable#abort()}.
+ * Value: {@value}.
+ */
+ public static final String ABORTABLE_STREAM =
+ "fs.capability.outputstream.abortable";
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DF.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DF.java
index 7a94088e4062d..30c2faeb24d1f 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DF.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DF.java
@@ -30,7 +30,7 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.util.Shell;
-import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
/** Filesystem disk space usage statistics.
* Uses the unix 'df' program to get mount points, and java.io.File for
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DU.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DU.java
index 6e374c97c3eda..89ac7c3e7cbbe 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DU.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DU.java
@@ -17,7 +17,7 @@
*/
package org.apache.hadoop.fs;
-import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DelegationTokenRenewer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DelegationTokenRenewer.java
index 2feb9375255c3..193c52c60d949 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DelegationTokenRenewer.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DelegationTokenRenewer.java
@@ -18,7 +18,7 @@
package org.apache.hadoop.fs;
-import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
import java.io.IOException;
import java.lang.ref.WeakReference;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataInputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataInputStream.java
index 31f82975899e1..b143a4cb63d19 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataInputStream.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataInputStream.java
@@ -29,6 +29,10 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.impl.StoreImplementationUtils;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+import org.apache.hadoop.fs.statistics.IOStatisticsSupport;
import org.apache.hadoop.io.ByteBufferPool;
import org.apache.hadoop.util.IdentityHashStore;
@@ -40,7 +44,7 @@ public class FSDataInputStream extends DataInputStream
implements Seekable, PositionedReadable,
ByteBufferReadable, HasFileDescriptor, CanSetDropBehind, CanSetReadahead,
HasEnhancedByteBufferAccess, CanUnbuffer, StreamCapabilities,
- ByteBufferPositionedReadable {
+ ByteBufferPositionedReadable, IOStatisticsSource {
/**
* Map ByteBuffers that we have handed out to readers to ByteBufferPool
* objects
@@ -234,10 +238,7 @@ public void unbuffer() {
@Override
public boolean hasCapability(String capability) {
- if (in instanceof StreamCapabilities) {
- return ((StreamCapabilities) in).hasCapability(capability);
- }
- return false;
+ return StoreImplementationUtils.hasCapability(in, capability);
}
/**
@@ -267,4 +268,15 @@ public void readFully(long position, ByteBuffer buf) throws IOException {
"unsupported by " + in.getClass().getCanonicalName());
}
}
+
+ /**
+ * Get the IO Statistics of the nested stream, falling back to
+ * null if the stream does not implement the interface
+ * {@link IOStatisticsSource}.
+ * @return an IOStatistics instance or null
+ */
+ @Override
+ public IOStatistics getIOStatistics() {
+ return IOStatisticsSupport.retrieveIOStatistics(in);
+ }
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStream.java
index 5b604e58e2360..94c56b713c1eb 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStream.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStream.java
@@ -24,13 +24,18 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.impl.StoreImplementationUtils;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+import org.apache.hadoop.fs.statistics.IOStatisticsSupport;
/** Utility that wraps a {@link OutputStream} in a {@link DataOutputStream}.
*/
@InterfaceAudience.Public
@InterfaceStability.Stable
public class FSDataOutputStream extends DataOutputStream
- implements Syncable, CanSetDropBehind, StreamCapabilities {
+ implements Syncable, CanSetDropBehind, StreamCapabilities,
+ IOStatisticsSource, Abortable {
private final OutputStream wrappedStream;
private static class PositionCache extends FilterOutputStream {
@@ -122,10 +127,7 @@ public OutputStream getWrappedStream() {
@Override
public boolean hasCapability(String capability) {
- if (wrappedStream instanceof StreamCapabilities) {
- return ((StreamCapabilities) wrappedStream).hasCapability(capability);
- }
- return false;
+ return StoreImplementationUtils.hasCapability(wrappedStream, capability);
}
@Override // Syncable
@@ -155,4 +157,32 @@ public void setDropBehind(Boolean dropBehind) throws IOException {
"not support setting the drop-behind caching setting.");
}
}
+
+ /**
+ * Get the IO Statistics of the nested stream, falling back to
+ * empty statistics if the stream does not implement the interface
+ * {@link IOStatisticsSource}.
+ * @return an IOStatistics instance.
+ */
+ @Override
+ public IOStatistics getIOStatistics() {
+ return IOStatisticsSupport.retrieveIOStatistics(wrappedStream);
+ }
+
+ /**
+ * Invoke {@code abort()} on the wrapped stream if it
+ * is Abortable, otherwise raise an
+ * {@code UnsupportedOperationException}.
+ * @throws UnsupportedOperationException if not available.
+ * @return the result.
+ */
+ @Override
+ public AbortableResult abort() {
+ if (wrappedStream instanceof Abortable) {
+ return ((Abortable) wrappedStream).abort();
+ } else {
+ throw new UnsupportedOperationException(
+ FSExceptionMessages.ABORTABLE_UNSUPPORTED);
+ }
+ }
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStreamBuilder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStreamBuilder.java
index 62a3182dfba20..7a6792817b750 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStreamBuilder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStreamBuilder.java
@@ -28,7 +28,7 @@
import java.io.IOException;
import java.util.EnumSet;
-import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull;
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT;
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSExceptionMessages.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSExceptionMessages.java
index a8e7b71bb119c..f4616f1d72bc7 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSExceptionMessages.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSExceptionMessages.java
@@ -51,4 +51,10 @@ public class FSExceptionMessages {
public static final String PERMISSION_DENIED_BY_STICKY_BIT =
"Permission denied by sticky bit";
+
+ /**
+ * A call was made to abort(), but it is not supported.
+ */
+ public static final String ABORTABLE_UNSUPPORTED =
+ "Abortable.abort() is not supported";
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSInputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSInputStream.java
index 672ab15f16c3b..ad2642f7db963 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSInputStream.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSInputStream.java
@@ -21,9 +21,12 @@
import java.io.IOException;
import java.io.InputStream;
-import com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.statistics.IOStatisticsLogging;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -134,4 +137,23 @@ public void readFully(long position, byte[] buffer)
throws IOException {
readFully(position, buffer, 0, buffer.length);
}
+
+ /**
+ * toString method returns the superclass toString, but if the subclass
+ * implements {@link IOStatisticsSource} then those statistics are
+ * extracted and included in the output.
+ * That is: statistics of subclasses are automatically reported.
+ * @return a string value.
+ */
+ @Override
+ public String toString() {
+ final StringBuilder sb = new StringBuilder(super.toString());
+ sb.append('{');
+ if (this instanceof IOStatisticsSource) {
+ sb.append(IOStatisticsLogging.ioStatisticsSourceToString(
+ (IOStatisticsSource) this));
+ }
+ sb.append('}');
+ return sb.toString();
+ }
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java
index 2458b2f40d8d7..aaa19adf8c6a4 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java
@@ -33,7 +33,8 @@
*/
@InterfaceAudience.LimitedPrivate({"HDFS"})
@InterfaceStability.Unstable
-abstract public class FSOutputSummer extends OutputStream {
+abstract public class FSOutputSummer extends OutputStream implements
+ StreamCapabilities {
// data checksum
private final DataChecksum sum;
// internal buffer for storing data before it is checksumed
@@ -254,4 +255,9 @@ protected synchronized void setChecksumBufSize(int size) {
protected synchronized void resetChecksumBufSize() {
setChecksumBufSize(sum.getBytesPerChecksum() * BUFFER_NUM_CHUNKS);
}
+
+ @Override
+ public boolean hasCapability(String capability) {
+ return false;
+ }
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
index df93e89750ee0..422eae9337771 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
@@ -65,7 +65,8 @@
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.ShutdownHookManager;
-import com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.htrace.core.Tracer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -507,10 +508,9 @@ public static FileContext getLocalFSFileContext(final Configuration aConf)
return getFileContext(FsConstants.LOCAL_FS_URI, aConf);
}
- /* This method is needed for tests. */
+ @VisibleForTesting
@InterfaceAudience.Private
- @InterfaceStability.Unstable /* return type will change to AFS once
- HADOOP-6223 is completed */
+ @InterfaceStability.Unstable
public AbstractFileSystem getDefaultFileSystem() {
return defaultFS;
}
@@ -1249,6 +1249,16 @@ public FileStatus next(final AbstractFileSystem fs, final Path p)
}.resolve(this, absF);
}
+ /**
+ * Synchronize client metadata state.
+ *
+ * @throws IOException
+ * @throws UnsupportedOperationException
+ */
+ public void msync() throws IOException, UnsupportedOperationException {
+ defaultFS.msync();
+ }
+
/**
* Checks if the user can access a path. The mode specifies which access
* checks to perform. If the requested permissions are granted, then the
@@ -2957,4 +2967,31 @@ public boolean hasPathCapability(Path path, String capability)
(fs, p) -> fs.hasPathCapability(p, capability));
}
+ /**
+ * Return a set of server default configuration values based on path.
+ * @param path path to fetch server defaults
+ * @return server default configuration values for path
+ * @throws IOException an I/O error occurred
+ */
+ public FsServerDefaults getServerDefaults(final Path path)
+ throws IOException {
+ return FsLinkResolution.resolve(this,
+ fixRelativePart(path),
+ (fs, p) -> fs.getServerDefaults(p));
+ }
+
+ /**
+ * Create a multipart uploader.
+ * @param basePath file path under which all files are uploaded
+ * @return a MultipartUploaderBuilder object to build the uploader
+ * @throws IOException if some early checks cause IO failures.
+ * @throws UnsupportedOperationException if support is checked early.
+ */
+ @InterfaceStability.Unstable
+ public MultipartUploaderBuilder createMultipartUploader(Path basePath)
+ throws IOException {
+ return FsLinkResolution.resolve(this,
+ fixRelativePart(basePath),
+ (fs, p) -> fs.createMultipartUploader(p));
+ }
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileEncryptionInfo.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileEncryptionInfo.java
index 4cfce2eed6c2c..5444103855d26 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileEncryptionInfo.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileEncryptionInfo.java
@@ -24,8 +24,8 @@
import org.apache.hadoop.crypto.CipherSuite;
import org.apache.hadoop.crypto.CryptoProtocolVersion;
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkArgument;
+import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull;
/**
* FileEncryptionInfo encapsulates all the encryption-related information for
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
index 358db744e65be..528f6c270f4ee 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
@@ -21,6 +21,7 @@
import java.io.Closeable;
import java.io.FileNotFoundException;
import java.io.IOException;
+import java.io.InterruptedIOException;
import java.lang.ref.WeakReference;
import java.lang.ref.ReferenceQueue;
import java.net.URI;
@@ -44,6 +45,7 @@
import java.util.Stack;
import java.util.TreeSet;
import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Semaphore;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.logging.Log;
@@ -75,6 +77,7 @@
import org.apache.hadoop.security.token.DelegationTokenIssuer;
import org.apache.hadoop.util.ClassUtil;
import org.apache.hadoop.util.DataChecksum;
+import org.apache.hadoop.util.DurationInfo;
import org.apache.hadoop.util.LambdaUtils;
import org.apache.hadoop.util.Progressable;
import org.apache.hadoop.util.ReflectionUtils;
@@ -83,12 +86,12 @@
import org.apache.htrace.core.Tracer;
import org.apache.htrace.core.TraceScope;
-import com.google.common.base.Preconditions;
-import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkArgument;
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.*;
import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs;
@@ -132,22 +135,35 @@
* New methods may be marked as Unstable or Evolving for their initial release,
* as a warning that they are new and may change based on the
* experience of use in applications.
+ *
* Important note for developers
- *
- * If you're making changes here to the public API or protected methods,
+ *
+ * If you are making changes here to the public API or protected methods,
* you must review the following subclasses and make sure that
* they are filtering/passing through new methods as appropriate.
+ *
*
- * {@link FilterFileSystem}: methods are passed through.
+ * {@link FilterFileSystem}: methods are passed through. If not,
+ * then {@code TestFilterFileSystem.MustNotImplement} must be
+ * updated with the unsupported interface.
+ * Furthermore, if the new API's support is probed for via
+ * {@link #hasPathCapability(Path, String)} then
+ * {@link FilterFileSystem#hasPathCapability(Path, String)}
+ * must return false, always.
+ *
* {@link ChecksumFileSystem}: checksums are created and
* verified.
+ *
* {@code TestHarFileSystem} will need its {@code MustNotImplement}
* interface updated.
+ *
*
* There are some external places your changes will break things.
* Do co-ordinate changes here.
+ *
*
* HBase: HBoss
+ *
* Hive: HiveShim23
* {@code shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java}
*
@@ -187,7 +203,7 @@ public abstract class FileSystem extends Configured
public static final String USER_HOME_PREFIX = "/user";
/** FileSystem cache. */
- static final Cache CACHE = new Cache();
+ static final Cache CACHE = new Cache(new Configuration());
/** The key this instance is stored under in the cache. */
private Cache.Key key;
@@ -607,6 +623,7 @@ public static LocalFileSystem newInstanceLocal(Configuration conf)
* @throws IOException a problem arose closing one or more filesystem.
*/
public static void closeAll() throws IOException {
+ debugLogFileSystemClose("closeAll", "");
CACHE.closeAll();
}
@@ -617,10 +634,24 @@ public static void closeAll() throws IOException {
* @throws IOException a problem arose closing one or more filesystem.
*/
public static void closeAllForUGI(UserGroupInformation ugi)
- throws IOException {
+ throws IOException {
+ debugLogFileSystemClose("closeAllForUGI", "UGI: " + ugi);
CACHE.closeAll(ugi);
}
+ private static void debugLogFileSystemClose(String methodName,
+ String additionalInfo) {
+ if (LOGGER.isDebugEnabled()) {
+ Throwable throwable = new Throwable().fillInStackTrace();
+ LOGGER.debug("FileSystem.{}() by method: {}); {}", methodName,
+ throwable.getStackTrace()[2], additionalInfo);
+ if (LOGGER.isTraceEnabled()) {
+ LOGGER.trace("FileSystem.{}() full stack trace:", methodName,
+ throwable);
+ }
+ }
+ }
+
/**
* Qualify a path to one which uses this FileSystem and, if relative,
* made absolute.
@@ -2200,7 +2231,9 @@ private void fetchMore() throws IOException {
@Override
@SuppressWarnings("unchecked")
public T next() throws IOException {
- Preconditions.checkState(hasNext(), "No more items in iterator");
+ if (!hasNext()) {
+ throw new NoSuchElementException("No more items in iterator");
+ }
if (i == entries.getEntries().length) {
fetchMore();
}
@@ -2556,9 +2589,15 @@ public void completeLocalOutput(Path fsOutputFile, Path tmpLocalFile)
*/
@Override
public void close() throws IOException {
+ debugLogFileSystemClose("close", "Key: " + key + "; URI: " + getUri()
+ + "; Object Identity Hash: "
+ + Integer.toHexString(System.identityHashCode(this)));
// delete all files that were marked as delete-on-exit.
- processDeleteOnExit();
- CACHE.remove(this.key, this);
+ try {
+ processDeleteOnExit();
+ } finally {
+ CACHE.remove(this.key, this);
+ }
}
/**
@@ -2641,6 +2680,20 @@ public short getDefaultReplication(Path path) {
*/
public abstract FileStatus getFileStatus(Path f) throws IOException;
+ /**
+ * Synchronize client metadata state.
+ *
+ * In some FileSystem implementations such as HDFS metadata
+ * synchronization is essential to guarantee consistency of read requests
+ * particularly in HA setting.
+ * @throws IOException
+ * @throws UnsupportedOperationException
+ */
+ public void msync() throws IOException, UnsupportedOperationException {
+ throw new UnsupportedOperationException(getClass().getCanonicalName() +
+ " does not support method msync");
+ }
+
/**
* Checks if the user can access a path. The mode specifies which access
* checks to perform. If the requested permissions are granted, then the
@@ -3405,7 +3458,9 @@ public static Class extends FileSystem> getFileSystemClass(String scheme,
private static FileSystem createFileSystem(URI uri, Configuration conf)
throws IOException {
Tracer tracer = FsTracer.get(conf);
- try(TraceScope scope = tracer.newScope("FileSystem#createFileSystem")) {
+ try(TraceScope scope = tracer.newScope("FileSystem#createFileSystem");
+ DurationInfo ignored =
+ new DurationInfo(LOGGER, false, "Creating FS %s", uri)) {
scope.addKVAnnotation("scheme", uri.getScheme());
Class extends FileSystem> clazz =
getFileSystemClass(uri.getScheme(), conf);
@@ -3428,15 +3483,39 @@ private static FileSystem createFileSystem(URI uri, Configuration conf)
}
/** Caching FileSystem objects. */
- static class Cache {
+ static final class Cache {
private final ClientFinalizer clientFinalizer = new ClientFinalizer();
private final Map map = new HashMap<>();
private final Set toAutoClose = new HashSet<>();
+ /** Semaphore used to serialize creation of new FS instances. */
+ private final Semaphore creatorPermits;
+
+ /**
+ * Counter of the number of discarded filesystem instances
+ * in this cache. Primarily for testing, but it could possibly
+ * be made visible as some kind of metric.
+ */
+ private final AtomicLong discardedInstances = new AtomicLong(0);
+
/** A variable that makes all objects in the cache unique. */
private static AtomicLong unique = new AtomicLong(1);
+ /**
+ * Instantiate. The configuration is used to read the
+ * count of permits issued for concurrent creation
+ * of filesystem instances.
+ * @param conf configuration
+ */
+ Cache(final Configuration conf) {
+ int permits = conf.getInt(FS_CREATION_PARALLEL_COUNT,
+ FS_CREATION_PARALLEL_COUNT_DEFAULT);
+ checkArgument(permits > 0, "Invalid value of %s: %s",
+ FS_CREATION_PARALLEL_COUNT, permits);
+ creatorPermits = new Semaphore(permits);
+ }
+
FileSystem get(URI uri, Configuration conf) throws IOException{
Key key = new Key(uri, conf);
return getInternal(uri, conf, key);
@@ -3470,33 +3549,86 @@ private FileSystem getInternal(URI uri, Configuration conf, Key key)
if (fs != null) {
return fs;
}
-
- fs = createFileSystem(uri, conf);
- final long timeout = conf.getTimeDuration(SERVICE_SHUTDOWN_TIMEOUT,
- SERVICE_SHUTDOWN_TIMEOUT_DEFAULT,
- ShutdownHookManager.TIME_UNIT_DEFAULT);
- synchronized (this) { // refetch the lock again
- FileSystem oldfs = map.get(key);
- if (oldfs != null) { // a file system is created while lock is releasing
- fs.close(); // close the new file system
- return oldfs; // return the old file system
- }
-
- // now insert the new file system into the map
- if (map.isEmpty()
- && !ShutdownHookManager.get().isShutdownInProgress()) {
- ShutdownHookManager.get().addShutdownHook(clientFinalizer,
- SHUTDOWN_HOOK_PRIORITY, timeout,
- ShutdownHookManager.TIME_UNIT_DEFAULT);
+ // fs not yet created, acquire lock
+ // to construct an instance.
+ try (DurationInfo d = new DurationInfo(LOGGER, false,
+ "Acquiring creator semaphore for %s", uri)) {
+ creatorPermits.acquire();
+ } catch (InterruptedException e) {
+ // acquisition was interrupted; convert to an IOE.
+ throw (IOException)new InterruptedIOException(e.toString())
+ .initCause(e);
+ }
+ FileSystem fsToClose = null;
+ try {
+ // See if FS was instantiated by another thread while waiting
+ // for the permit.
+ synchronized (this) {
+ fs = map.get(key);
}
- fs.key = key;
- map.put(key, fs);
- if (conf.getBoolean(
- FS_AUTOMATIC_CLOSE_KEY, FS_AUTOMATIC_CLOSE_DEFAULT)) {
- toAutoClose.add(key);
+ if (fs != null) {
+ LOGGER.debug("Filesystem {} created while awaiting semaphore", uri);
+ return fs;
}
- return fs;
+ // create the filesystem
+ fs = createFileSystem(uri, conf);
+ final long timeout = conf.getTimeDuration(SERVICE_SHUTDOWN_TIMEOUT,
+ SERVICE_SHUTDOWN_TIMEOUT_DEFAULT,
+ ShutdownHookManager.TIME_UNIT_DEFAULT);
+ // any FS to close outside of the synchronized section
+ synchronized (this) { // lock on the Cache object
+
+ // see if there is now an entry for the FS, which happens
+ // if another thread's creation overlapped with this one.
+ FileSystem oldfs = map.get(key);
+ if (oldfs != null) {
+ // a file system was created in a separate thread.
+ // save the FS reference to close outside all locks,
+ // and switch to returning the oldFS
+ fsToClose = fs;
+ fs = oldfs;
+ } else {
+ // register the clientFinalizer if needed and shutdown isn't
+ // already active
+ if (map.isEmpty()
+ && !ShutdownHookManager.get().isShutdownInProgress()) {
+ ShutdownHookManager.get().addShutdownHook(clientFinalizer,
+ SHUTDOWN_HOOK_PRIORITY, timeout,
+ ShutdownHookManager.TIME_UNIT_DEFAULT);
+ }
+ // insert the new file system into the map
+ fs.key = key;
+ map.put(key, fs);
+ if (conf.getBoolean(
+ FS_AUTOMATIC_CLOSE_KEY, FS_AUTOMATIC_CLOSE_DEFAULT)) {
+ toAutoClose.add(key);
+ }
+ }
+ } // end of synchronized block
+ } finally {
+ // release the creator permit.
+ creatorPermits.release();
}
+ if (fsToClose != null) {
+ LOGGER.debug("Duplicate FS created for {}; discarding {}",
+ uri, fs);
+ discardedInstances.incrementAndGet();
+ // close the new file system
+ // note this will briefly remove and reinstate "fsToClose" from
+ // the map. It is done in a synchronized block so will not be
+ // visible to others.
+ IOUtils.cleanupWithLogger(LOGGER, fsToClose);
+ }
+ return fs;
+ }
+
+ /**
+ * Get the count of discarded instances.
+ * @return the new instance.
+ */
+ @VisibleForTesting
+ long getDiscardedInstances() {
+ return discardedInstances.get();
}
synchronized void remove(Key key, FileSystem fs) {
@@ -4643,4 +4775,17 @@ public CompletableFuture build() throws IOException {
}
+ /**
+ * Create a multipart uploader.
+ * @param basePath file path under which all files are uploaded
+ * @return a MultipartUploaderBuilder object to build the uploader
+ * @throws IOException if some early checks cause IO failures.
+ * @throws UnsupportedOperationException if support is checked early.
+ */
+ @InterfaceStability.Unstable
+ public MultipartUploaderBuilder createMultipartUploader(Path basePath)
+ throws IOException {
+ methodNotSupported();
+ return null;
+ }
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystemStorageStatistics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystemStorageStatistics.java
index 43c23abadea44..f717e03692378 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystemStorageStatistics.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystemStorageStatistics.java
@@ -20,7 +20,7 @@
import java.util.Iterator;
import java.util.NoSuchElementException;
-import com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.FileSystem.Statistics.StatisticsData;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
index 7bc93f9bf5db8..73ca6e65216e7 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
@@ -398,6 +398,12 @@ public static boolean copy(FileSystem srcFS, FileStatus srcStatus,
Configuration conf) throws IOException {
Path src = srcStatus.getPath();
dst = checkDest(src.getName(), dstFS, dst, overwrite);
+
+ if (srcFS.makeQualified(src).equals(dstFS.makeQualified(dst))) {
+ throw new PathOperationException("Source (" + src + ") and destination " +
+ "(" + dst + ") are equal in the copy command.");
+ }
+
if (srcStatus.isDirectory()) {
checkDependencies(srcFS, src, dstFS, dst);
if (!dstFS.mkdirs(dst)) {
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java
index cf12ea3898a7f..607aa263622f6 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java
@@ -41,6 +41,8 @@
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.util.Progressable;
+import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs;
+
/****************************************************************
* A FilterFileSystem contains
* some other file system, which it uses as
@@ -460,6 +462,11 @@ public FileStatus getFileStatus(Path f) throws IOException {
return fs.getFileStatus(f);
}
+ @Override
+ public void msync() throws IOException, UnsupportedOperationException {
+ fs.msync();
+ }
+
@Override
public void access(Path path, FsAction mode) throws AccessControlException,
FileNotFoundException, IOException {
@@ -728,7 +735,16 @@ protected CompletableFuture openFileWithOptions(
@Override
public boolean hasPathCapability(final Path path, final String capability)
throws IOException {
- return fs.hasPathCapability(path, capability);
+ switch (validatePathCapabilityArgs(makeQualified(path), capability)) {
+ case CommonPathCapabilities.FS_MULTIPART_UPLOADER:
+ case CommonPathCapabilities.FS_EXPERIMENTAL_BATCH_LISTING:
+ // operations known to be unsupported, irrespective of what
+ // the wrapped class implements.
+ return false;
+ default:
+ // the feature is not implemented.
+ return fs.hasPathCapability(path, capability);
+ }
}
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFs.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFs.java
index e197506edc88b..7d979b37b4a50 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFs.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFs.java
@@ -124,6 +124,11 @@ public FileStatus getFileStatus(Path f)
return myFs.getFileStatus(f);
}
+ @Override
+ public void msync() throws IOException, UnsupportedOperationException {
+ myFs.msync();
+ }
+
@Override
public void access(Path path, FsAction mode) throws AccessControlException,
FileNotFoundException, UnresolvedLinkException, IOException {
@@ -448,4 +453,10 @@ public boolean hasPathCapability(final Path path,
throws IOException {
return myFs.hasPathCapability(path, capability);
}
+
+ @Override
+ public MultipartUploaderBuilder createMultipartUploader(final Path basePath)
+ throws IOException {
+ return myFs.createMultipartUploader(basePath);
+ }
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsConstants.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsConstants.java
index cfef1c3827917..603454210644d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsConstants.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsConstants.java
@@ -42,4 +42,7 @@ public interface FsConstants {
*/
public static final URI VIEWFS_URI = URI.create("viewfs:///");
public static final String VIEWFS_SCHEME = "viewfs";
+ String FS_VIEWFS_OVERLOAD_SCHEME_TARGET_FS_IMPL_PATTERN =
+ "fs.viewfs.overload.scheme.target.%s.impl";
+ String VIEWFS_TYPE = "viewfs";
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsTracer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsTracer.java
index e422336739a44..6fab4bdfebc99 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsTracer.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsTracer.java
@@ -17,7 +17,7 @@
*/
package org.apache.hadoop.fs;
-import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsUrlConnection.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsUrlConnection.java
index c5429d2370250..11b3e91e86c3a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsUrlConnection.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsUrlConnection.java
@@ -24,7 +24,7 @@
import java.net.URL;
import java.net.URLConnection;
-import com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/GlobalStorageStatistics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/GlobalStorageStatistics.java
index 2dba525e5d9d1..0cf3b62bccfe9 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/GlobalStorageStatistics.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/GlobalStorageStatistics.java
@@ -23,7 +23,7 @@
import java.util.NoSuchElementException;
import java.util.TreeMap;
-import com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import org.apache.hadoop.classification.InterfaceAudience;
/**
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Globber.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Globber.java
index f301f22057925..9cdcb4ac4acd3 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Globber.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Globber.java
@@ -32,7 +32,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull;
/**
* Implementation of {@link FileSystem#globStatus(Path, PathFilter)}.
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java
index 5f4c4a236e96c..7e12d0a11e953 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java
@@ -35,6 +35,7 @@
import java.net.URISyntaxException;
import java.net.URLDecoder;
import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs;
@@ -513,41 +514,22 @@ private void fileStatusesInIndex(HarStatus parent, List statuses)
if (!parentString.endsWith(Path.SEPARATOR)){
parentString += Path.SEPARATOR;
}
- Path harPath = new Path(parentString);
- int harlen = harPath.depth();
- final Map cache = new TreeMap();
-
- for (HarStatus hstatus : metadata.archive.values()) {
- String child = hstatus.getName();
- if ((child.startsWith(parentString))) {
- Path thisPath = new Path(child);
- if (thisPath.depth() == harlen + 1) {
- statuses.add(toFileStatus(hstatus, cache));
- }
- }
+
+ for (String child: parent.children) {
+ Path p = new Path(parentString + child);
+ statuses.add(toFileStatus(metadata.archive.get(p)));
}
}
/**
* Combine the status stored in the index and the underlying status.
* @param h status stored in the index
- * @param cache caching the underlying file statuses
* @return the combined file status
* @throws IOException
*/
- private FileStatus toFileStatus(HarStatus h,
- Map cache) throws IOException {
- FileStatus underlying = null;
- if (cache != null) {
- underlying = cache.get(h.partName);
- }
- if (underlying == null) {
- final Path p = h.isDir? archivePath: new Path(archivePath, h.partName);
- underlying = fs.getFileStatus(p);
- if (cache != null) {
- cache.put(h.partName, underlying);
- }
- }
+ private FileStatus toFileStatus(HarStatus h) throws IOException {
+ final Path p = h.isDir ? archivePath : new Path(archivePath, h.partName);
+ FileStatus underlying = metadata.getPartFileStatus(p);
long modTime = 0;
int version = metadata.getVersion();
@@ -658,7 +640,7 @@ public long getModificationTime() {
@Override
public FileStatus getFileStatus(Path f) throws IOException {
HarStatus hstatus = getFileHarStatus(f);
- return toFileStatus(hstatus, null);
+ return toFileStatus(hstatus);
}
private HarStatus getFileHarStatus(Path f) throws IOException {
@@ -676,6 +658,11 @@ private HarStatus getFileHarStatus(Path f) throws IOException {
return hstatus;
}
+ @Override
+ public void msync() throws IOException, UnsupportedOperationException {
+ fs.msync();
+ }
+
/**
* @return null since no checksum algorithm is implemented.
*/
@@ -810,7 +797,7 @@ public FileStatus[] listStatus(Path f) throws IOException {
if (hstatus.isDir()) {
fileStatusesInIndex(hstatus, statuses);
} else {
- statuses.add(toFileStatus(hstatus, null));
+ statuses.add(toFileStatus(hstatus));
}
return statuses.toArray(new FileStatus[statuses.size()]);
@@ -1138,7 +1125,8 @@ private class HarMetaData {
List stores = new ArrayList();
Map archive = new HashMap();
- private Map partFileStatuses = new HashMap();
+ // keys are always the internal har path.
+ private Map partFileStatuses = new ConcurrentHashMap<>();
public HarMetaData(FileSystem fs, Path masterIndexPath, Path archiveIndexPath) {
this.fs = fs;
@@ -1146,16 +1134,23 @@ public HarMetaData(FileSystem fs, Path masterIndexPath, Path archiveIndexPath) {
this.archiveIndexPath = archiveIndexPath;
}
- public FileStatus getPartFileStatus(Path partPath) throws IOException {
+ public FileStatus getPartFileStatus(Path path) throws IOException {
+ Path partPath = getPathInHar(path);
FileStatus status;
status = partFileStatuses.get(partPath);
if (status == null) {
- status = fs.getFileStatus(partPath);
+ status = fs.getFileStatus(path);
partFileStatuses.put(partPath, status);
}
return status;
}
+ private void addPartFileStatuses(Path path) throws IOException {
+ for (FileStatus stat : fs.listStatus(path)) {
+ partFileStatuses.put(getPathInHar(stat.getPath()), stat);
+ }
+ }
+
public long getMasterIndexTimestamp() {
return masterIndexTimestamp;
}
@@ -1212,16 +1207,22 @@ private void parseMetaData() throws IOException {
try {
FileStatus archiveStat = fs.getFileStatus(archiveIndexPath);
archiveIndexTimestamp = archiveStat.getModificationTime();
- LineReader aLin;
+
+ // pre-populate part cache.
+ addPartFileStatuses(archiveIndexPath.getParent());
+ LineReader aLin = null;
// now start reading the real index file
+ long pos = -1;
for (Store s: stores) {
- read = 0;
- aIn.seek(s.begin);
- aLin = new LineReader(aIn, getConf());
- while (read + s.begin < s.end) {
- int tmp = aLin.readLine(line);
- read += tmp;
+ if (pos != s.begin) {
+ pos = s.begin;
+ aIn.seek(s.begin);
+ aLin = new LineReader(aIn, getConf());
+ }
+
+ while (pos < s.end) {
+ pos += aLin.readLine(line);
String lineFeed = line.toString();
String[] parsed = lineFeed.split(" ");
parsed[0] = decodeFileName(parsed[0]);
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HardLink.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HardLink.java
index 8b47dfeb9a7ce..30f793dadfec3 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HardLink.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HardLink.java
@@ -29,7 +29,7 @@
import org.apache.hadoop.util.Shell.ExitCodeException;
import org.apache.hadoop.util.Shell.ShellCommandExecutor;
-import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
import static java.nio.file.Files.createLink;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/InternalOperations.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/InternalOperations.java
new file mode 100644
index 0000000000000..2db33eead9288
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/InternalOperations.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+
+/**
+ * This method allows access to Package-scoped operations from classes
+ * in org.apache.hadoop.fs.impl and other file system implementations
+ * in the hadoop modules.
+ * This is absolutely not for used by any other application or library.
+ */
+@InterfaceAudience.Private
+public class InternalOperations {
+
+ @SuppressWarnings("deprecation") // rename w/ OVERWRITE
+ public void rename(FileSystem fs, final Path src, final Path dst,
+ final Options.Rename...options) throws IOException {
+ fs.rename(src, dst, options);
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MultipartUploader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MultipartUploader.java
index 7ed987eed90dd..dcb76b50b3429 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MultipartUploader.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MultipartUploader.java
@@ -1,4 +1,4 @@
-/**
+/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
@@ -15,45 +15,32 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
+
package org.apache.hadoop.fs;
import java.io.Closeable;
import java.io.IOException;
import java.io.InputStream;
import java.util.Map;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import java.util.concurrent.CompletableFuture;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
-
-import static com.google.common.base.Preconditions.checkArgument;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
/**
* MultipartUploader is an interface for copying files multipart and across
- * multiple nodes. Users should:
- *
- * - Initialize an upload.
- * - Upload parts in any order.
- * - Complete the upload in order to have it materialize in the destination
- * FS.
- *
+ * multiple nodes.
+ *
+ * The interface extends {@link IOStatisticsSource} so that there is no
+ * need to cast an instance to see if is a source of statistics.
+ * However, implementations MAY return null for their actual statistics.
*/
-@InterfaceAudience.Private
+@InterfaceAudience.Public
@InterfaceStability.Unstable
-public abstract class MultipartUploader implements Closeable {
- public static final Logger LOG =
- LoggerFactory.getLogger(MultipartUploader.class);
+public interface MultipartUploader extends Closeable,
+ IOStatisticsSource {
- /**
- * Perform any cleanup.
- * The upload is not required to support any operations after this.
- * @throws IOException problems on close.
- */
- @Override
- public void close() throws IOException {
- }
/**
* Initialize a multipart upload.
@@ -61,94 +48,64 @@ public void close() throws IOException {
* @return unique identifier associating part uploads.
* @throws IOException IO failure
*/
- public abstract UploadHandle initialize(Path filePath) throws IOException;
+ CompletableFuture startUpload(Path filePath)
+ throws IOException;
/**
* Put part as part of a multipart upload.
* It is possible to have parts uploaded in any order (or in parallel).
- * @param filePath Target path for upload (same as {@link #initialize(Path)}).
+ * @param uploadId Identifier from {@link #startUpload(Path)}.
+ * @param partNumber Index of the part relative to others.
+ * @param filePath Target path for upload (as {@link #startUpload(Path)}).
* @param inputStream Data for this part. Implementations MUST close this
* stream after reading in the data.
- * @param partNumber Index of the part relative to others.
- * @param uploadId Identifier from {@link #initialize(Path)}.
* @param lengthInBytes Target length to read from the stream.
* @return unique PartHandle identifier for the uploaded part.
* @throws IOException IO failure
*/
- public abstract PartHandle putPart(Path filePath, InputStream inputStream,
- int partNumber, UploadHandle uploadId, long lengthInBytes)
+ CompletableFuture putPart(
+ UploadHandle uploadId,
+ int partNumber,
+ Path filePath,
+ InputStream inputStream,
+ long lengthInBytes)
throws IOException;
/**
* Complete a multipart upload.
- * @param filePath Target path for upload (same as {@link #initialize(Path)}.
+ * @param uploadId Identifier from {@link #startUpload(Path)}.
+ * @param filePath Target path for upload (as {@link #startUpload(Path)}.
* @param handles non-empty map of part number to part handle.
- * from {@link #putPart(Path, InputStream, int, UploadHandle, long)}.
- * @param multipartUploadId Identifier from {@link #initialize(Path)}.
+ * from {@link #putPart(UploadHandle, int, Path, InputStream, long)}.
* @return unique PathHandle identifier for the uploaded file.
* @throws IOException IO failure
*/
- public abstract PathHandle complete(Path filePath,
- Map handles,
- UploadHandle multipartUploadId)
+ CompletableFuture complete(
+ UploadHandle uploadId,
+ Path filePath,
+ Map handles)
throws IOException;
/**
* Aborts a multipart upload.
- * @param filePath Target path for upload (same as {@link #initialize(Path)}.
- * @param multipartUploadId Identifier from {@link #initialize(Path)}.
+ * @param uploadId Identifier from {@link #startUpload(Path)}.
+ * @param filePath Target path for upload (same as {@link #startUpload(Path)}.
* @throws IOException IO failure
+ * @return a future; the operation will have completed
*/
- public abstract void abort(Path filePath, UploadHandle multipartUploadId)
+ CompletableFuture abort(UploadHandle uploadId, Path filePath)
throws IOException;
/**
- * Utility method to validate uploadIDs.
- * @param uploadId Upload ID
- * @throws IllegalArgumentException invalid ID
- */
- protected void checkUploadId(byte[] uploadId)
- throws IllegalArgumentException {
- checkArgument(uploadId != null, "null uploadId");
- checkArgument(uploadId.length > 0,
- "Empty UploadId is not valid");
- }
-
- /**
- * Utility method to validate partHandles.
- * @param partHandles handles
- * @throws IllegalArgumentException if the parts are invalid
+ * Best effort attempt to aborts multipart uploads under a path.
+ * Not all implementations support this, and those which do may
+ * be vulnerable to eventually consistent listings of current uploads
+ * -some may be missed.
+ * @param path path to abort uploads under.
+ * @return a future to the number of entries aborted;
+ * -1 if aborting is unsupported
+ * @throws IOException IO failure
*/
- protected void checkPartHandles(Map partHandles) {
- checkArgument(!partHandles.isEmpty(),
- "Empty upload");
- partHandles.keySet()
- .stream()
- .forEach(key ->
- checkArgument(key > 0,
- "Invalid part handle index %s", key));
- }
+ CompletableFuture abortUploadsUnderPath(Path path) throws IOException;
- /**
- * Check all the arguments to the
- * {@link #putPart(Path, InputStream, int, UploadHandle, long)} operation.
- * @param filePath Target path for upload (same as {@link #initialize(Path)}).
- * @param inputStream Data for this part. Implementations MUST close this
- * stream after reading in the data.
- * @param partNumber Index of the part relative to others.
- * @param uploadId Identifier from {@link #initialize(Path)}.
- * @param lengthInBytes Target length to read from the stream.
- * @throws IllegalArgumentException invalid argument
- */
- protected void checkPutArguments(Path filePath,
- InputStream inputStream,
- int partNumber,
- UploadHandle uploadId,
- long lengthInBytes) throws IllegalArgumentException {
- checkArgument(filePath != null, "null filePath");
- checkArgument(inputStream != null, "null inputStream");
- checkArgument(partNumber > 0, "Invalid part number: %d", partNumber);
- checkArgument(uploadId != null, "null uploadId");
- checkArgument(lengthInBytes >= 0, "Invalid part length: %d", lengthInBytes);
- }
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MultipartUploaderBuilder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MultipartUploaderBuilder.java
new file mode 100644
index 0000000000000..381bfaa07f6d1
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MultipartUploaderBuilder.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+
+import org.apache.hadoop.fs.permission.FsPermission;
+
+/**
+ * Builder interface for Multipart readers.
+ * @param
+ * @param
+ */
+public interface MultipartUploaderBuilder>
+ extends FSBuilder {
+
+ /**
+ * Set permission for the file.
+ */
+ B permission(@Nonnull FsPermission perm);
+
+ /**
+ * Set the size of the buffer to be used.
+ */
+ B bufferSize(int bufSize);
+
+ /**
+ * Set replication factor.
+ */
+ B replication(short replica);
+
+ /**
+ * Set block size.
+ */
+ B blockSize(long blkSize);
+
+ /**
+ * Create an FSDataOutputStream at the specified path.
+ */
+ B create();
+
+ /**
+ * Set to true to overwrite the existing file.
+ * Set it to false, an exception will be thrown when calling {@link #build()}
+ * if the file exists.
+ */
+ B overwrite(boolean overwrite);
+
+ /**
+ * Append to an existing file (optional operation).
+ */
+ B append();
+
+ /**
+ * Set checksum opt.
+ */
+ B checksumOpt(@Nonnull Options.ChecksumOpt chksumOpt);
+
+ /**
+ * Create the FSDataOutputStream to write on the file system.
+ *
+ * @throws IllegalArgumentException if the parameters are not valid.
+ * @throws IOException on errors when file system creates or appends the file.
+ */
+ S build() throws IllegalArgumentException, IOException;
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MultipartUploaderFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MultipartUploaderFactory.java
deleted file mode 100644
index e35b6bf18bbd6..0000000000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MultipartUploaderFactory.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.fs;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.ServiceLoader;
-
-/**
- * {@link ServiceLoader}-driven uploader API for storage services supporting
- * multipart uploads.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public abstract class MultipartUploaderFactory {
- public static final Logger LOG =
- LoggerFactory.getLogger(MultipartUploaderFactory.class);
-
- /**
- * Multipart Uploaders listed as services.
- */
- private static ServiceLoader serviceLoader =
- ServiceLoader.load(MultipartUploaderFactory.class,
- MultipartUploaderFactory.class.getClassLoader());
-
- // Iterate through the serviceLoader to avoid lazy loading.
- // Lazy loading would require synchronization in concurrent use cases.
- static {
- Iterator iterServices = serviceLoader.iterator();
- while (iterServices.hasNext()) {
- iterServices.next();
- }
- }
-
- /**
- * Get the multipart loader for a specific filesystem.
- * @param fs filesystem
- * @param conf configuration
- * @return an uploader, or null if one was found.
- * @throws IOException failure during the creation process.
- */
- public static MultipartUploader get(FileSystem fs, Configuration conf)
- throws IOException {
- MultipartUploader mpu = null;
- for (MultipartUploaderFactory factory : serviceLoader) {
- mpu = factory.createMultipartUploader(fs, conf);
- if (mpu != null) {
- break;
- }
- }
- return mpu;
- }
-
- protected abstract MultipartUploader createMultipartUploader(FileSystem fs,
- Configuration conf) throws IOException;
-}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/PartialListing.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/PartialListing.java
index 043f84612dc8b..80d173e905ed5 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/PartialListing.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/PartialListing.java
@@ -17,7 +17,7 @@
*/
package org.apache.hadoop.fs;
-import com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import org.apache.commons.lang3.builder.ToStringBuilder;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
index cf2210575da15..21c69b78ca3f6 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
@@ -19,7 +19,7 @@
package org.apache.hadoop.fs;
-import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
import java.io.BufferedOutputStream;
import java.io.DataOutput;
@@ -40,13 +40,20 @@
import java.nio.file.attribute.FileTime;
import java.util.Arrays;
import java.util.EnumSet;
+import java.util.Locale;
import java.util.Optional;
import java.util.StringTokenizer;
+import java.util.concurrent.atomic.AtomicLong;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.impl.StoreImplementationUtils;
import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+import org.apache.hadoop.fs.statistics.BufferedIOStatisticsOutputStream;
+import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.nativeio.NativeIO;
import org.apache.hadoop.util.Progressable;
@@ -54,6 +61,14 @@
import org.apache.hadoop.util.StringUtils;
import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs;
+import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BYTES;
+import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_EXCEPTIONS;
+import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS;
+import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_SKIP_BYTES;
+import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_SKIP_OPERATIONS;
+import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_WRITE_BYTES;
+import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_WRITE_EXCEPTIONS;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore;
/****************************************************************
* Implement the FileSystem API for the raw local filesystem.
@@ -64,6 +79,7 @@
public class RawLocalFileSystem extends FileSystem {
static final URI NAME = URI.create("file:///");
private Path workingDir;
+ private long defaultBlockSize;
// Temporary workaround for HADOOP-9652.
private static boolean useDeprecatedFileStatus = true;
@@ -100,17 +116,36 @@ public File pathToFile(Path path) {
public void initialize(URI uri, Configuration conf) throws IOException {
super.initialize(uri, conf);
setConf(conf);
+ defaultBlockSize = getDefaultBlockSize(new Path(uri));
}
/*******************************************************
* For open()'s FSInputStream.
*******************************************************/
- class LocalFSFileInputStream extends FSInputStream implements HasFileDescriptor {
+ class LocalFSFileInputStream extends FSInputStream implements
+ HasFileDescriptor, IOStatisticsSource, StreamCapabilities {
private FileInputStream fis;
private long position;
+ /**
+ * Minimal set of counters.
+ */
+ private final IOStatisticsStore ioStatistics = iostatisticsStore()
+ .withCounters(
+ STREAM_READ_BYTES,
+ STREAM_READ_EXCEPTIONS,
+ STREAM_READ_SEEK_OPERATIONS,
+ STREAM_READ_SKIP_OPERATIONS,
+ STREAM_READ_SKIP_BYTES)
+ .build();
+
+ /** Reference to the bytes read counter for slightly faster counting. */
+ private final AtomicLong bytesRead;
+
public LocalFSFileInputStream(Path f) throws IOException {
fis = new FileInputStream(pathToFile(f));
+ bytesRead = ioStatistics.getCounterReference(
+ STREAM_READ_BYTES);
}
@Override
@@ -133,8 +168,8 @@ public boolean seekToNewSource(long targetPos) throws IOException {
return false;
}
- /*
- * Just forward to the fis
+ /**
+ * Just forward to the fis.
*/
@Override
public int available() throws IOException { return fis.available(); }
@@ -150,9 +185,11 @@ public int read() throws IOException {
if (value >= 0) {
this.position++;
statistics.incrementBytesRead(1);
+ bytesRead.addAndGet(1);
}
return value;
} catch (IOException e) { // unexpected exception
+ ioStatistics.incrementCounter(STREAM_READ_EXCEPTIONS);
throw new FSError(e); // assume native fs error
}
}
@@ -166,9 +203,11 @@ public int read(byte[] b, int off, int len) throws IOException {
if (value > 0) {
this.position += value;
statistics.incrementBytesRead(value);
+ bytesRead.addAndGet(value);
}
return value;
} catch (IOException e) { // unexpected exception
+ ioStatistics.incrementCounter(STREAM_READ_EXCEPTIONS);
throw new FSError(e); // assume native fs error
}
}
@@ -187,18 +226,22 @@ public int read(long position, byte[] b, int off, int len)
int value = fis.getChannel().read(bb, position);
if (value > 0) {
statistics.incrementBytesRead(value);
+ ioStatistics.incrementCounter(STREAM_READ_BYTES, value);
}
return value;
} catch (IOException e) {
+ ioStatistics.incrementCounter(STREAM_READ_EXCEPTIONS);
throw new FSError(e);
}
}
@Override
public long skip(long n) throws IOException {
+ ioStatistics.incrementCounter(STREAM_READ_SKIP_OPERATIONS);
long value = fis.skip(n);
if (value > 0) {
this.position += value;
+ ioStatistics.incrementCounter(STREAM_READ_SKIP_BYTES, value);
}
return value;
}
@@ -207,6 +250,23 @@ public long skip(long n) throws IOException {
public FileDescriptor getFileDescriptor() throws IOException {
return fis.getFD();
}
+
+ @Override
+ public boolean hasCapability(String capability) {
+ // a bit inefficient, but intended to make it easier to add
+ // new capabilities.
+ switch (capability.toLowerCase(Locale.ENGLISH)) {
+ case StreamCapabilities.IOSTATISTICS:
+ return true;
+ default:
+ return false;
+ }
+ }
+
+ @Override
+ public IOStatistics getIOStatistics() {
+ return ioStatistics;
+ }
}
@Override
@@ -231,9 +291,19 @@ public FSDataInputStream open(PathHandle fd, int bufferSize)
/*********************************************************
* For create()'s FSOutputStream.
*********************************************************/
- class LocalFSFileOutputStream extends OutputStream {
+ final class LocalFSFileOutputStream extends OutputStream implements
+ IOStatisticsSource, StreamCapabilities, Syncable {
private FileOutputStream fos;
-
+
+ /**
+ * Minimal set of counters.
+ */
+ private final IOStatisticsStore ioStatistics = iostatisticsStore()
+ .withCounters(
+ STREAM_WRITE_BYTES,
+ STREAM_WRITE_EXCEPTIONS)
+ .build();
+
private LocalFSFileOutputStream(Path f, boolean append,
FsPermission permission) throws IOException {
File file = pathToFile(f);
@@ -273,7 +343,9 @@ private LocalFSFileOutputStream(Path f, boolean append,
public void write(byte[] b, int off, int len) throws IOException {
try {
fos.write(b, off, len);
+ ioStatistics.incrementCounter(STREAM_WRITE_BYTES, len);
} catch (IOException e) { // unexpected exception
+ ioStatistics.incrementCounter(STREAM_WRITE_EXCEPTIONS);
throw new FSError(e); // assume native fs error
}
}
@@ -282,10 +354,44 @@ public void write(byte[] b, int off, int len) throws IOException {
public void write(int b) throws IOException {
try {
fos.write(b);
+ ioStatistics.incrementCounter(STREAM_WRITE_BYTES);
} catch (IOException e) { // unexpected exception
+ ioStatistics.incrementCounter(STREAM_WRITE_EXCEPTIONS);
throw new FSError(e); // assume native fs error
}
}
+
+ @Override
+ public void hflush() throws IOException {
+ flush();
+ }
+
+ /**
+ * HSync calls sync on fhe file descriptor after a local flush() call.
+ * @throws IOException failure
+ */
+ @Override
+ public void hsync() throws IOException {
+ flush();
+ fos.getFD().sync();
+ }
+
+ @Override
+ public boolean hasCapability(String capability) {
+ // a bit inefficient, but intended to make it easier to add
+ // new capabilities.
+ switch (capability.toLowerCase(Locale.ENGLISH)) {
+ case StreamCapabilities.IOSTATISTICS:
+ return true;
+ default:
+ return StoreImplementationUtils.isProbeForSyncable(capability);
+ }
+ }
+
+ @Override
+ public IOStatistics getIOStatistics() {
+ return ioStatistics;
+ }
}
@Override
@@ -318,8 +424,8 @@ private FSDataOutputStream create(Path f, boolean overwrite,
if (parent != null && !mkdirs(parent)) {
throw new IOException("Mkdirs failed to create " + parent.toString());
}
- return new FSDataOutputStream(new BufferedOutputStream(
- createOutputStreamWithMode(f, false, permission), bufferSize),
+ return new FSDataOutputStream(new BufferedIOStatisticsOutputStream(
+ createOutputStreamWithMode(f, false, permission), bufferSize, true),
statistics);
}
@@ -340,8 +446,8 @@ public FSDataOutputStream createNonRecursive(Path f, FsPermission permission,
if (exists(f) && !flags.contains(CreateFlag.OVERWRITE)) {
throw new FileAlreadyExistsException("File already exists: " + f);
}
- return new FSDataOutputStream(new BufferedOutputStream(
- createOutputStreamWithMode(f, false, permission), bufferSize),
+ return new FSDataOutputStream(new BufferedIOStatisticsOutputStream(
+ createOutputStreamWithMode(f, false, permission), bufferSize, true),
statistics);
}
@@ -518,7 +624,12 @@ public FileStatus[] listStatus(Path f) throws IOException {
}
return new FileStatus[] {
new DeprecatedRawLocalFileStatus(localf,
- getDefaultBlockSize(f), this) };
+ defaultBlockSize, this) };
+ }
+
+ @Override
+ public boolean exists(Path f) throws IOException {
+ return pathToFile(f).exists();
}
protected boolean mkOneDir(File p2f) throws IOException {
@@ -663,7 +774,7 @@ private FileStatus deprecatedGetFileStatus(Path f) throws IOException {
File path = pathToFile(f);
if (path.exists()) {
return new DeprecatedRawLocalFileStatus(pathToFile(f),
- getDefaultBlockSize(f), this);
+ defaultBlockSize, this);
} else {
throw new FileNotFoundException("File " + f + " does not exist");
}
@@ -1051,7 +1162,7 @@ private FileStatus deprecatedGetFileLinkStatusInternal(final Path f)
private FileStatus getNativeFileLinkStatus(final Path f,
boolean dereference) throws IOException {
checkPath(f);
- Stat stat = new Stat(f, getDefaultBlockSize(f), dereference, this);
+ Stat stat = new Stat(f, defaultBlockSize, dereference, this);
FileStatus status = stat.getFileStatus();
return status;
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Stat.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Stat.java
index 5e80a140175e6..d2300872eb146 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Stat.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Stat.java
@@ -30,7 +30,7 @@
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.util.Shell;
-import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* Wrapper for the Unix stat(1) command. Used to workaround the lack of
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageStatistics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageStatistics.java
index 74631b5695537..2efe4566344ee 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageStatistics.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageStatistics.java
@@ -19,6 +19,7 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.statistics.StoreStatisticNames;
import java.util.Iterator;
@@ -27,15 +28,16 @@
* instance.
*/
@InterfaceAudience.Public
+@InterfaceStability.Stable
public abstract class StorageStatistics {
/**
* These are common statistic names.
- *
+ *
* The following names are considered general and preserved across different
* StorageStatistics classes. When implementing a new StorageStatistics, it is
* highly recommended to use the common statistic names.
- *
+ *
* When adding new common statistic name constants, please make them unique.
* By convention, they are implicitly unique:
*
@@ -43,39 +45,46 @@ public abstract class StorageStatistics {
* underscores.
* - the value of the constants are lowercase of the constant names.
*
+ * See {@link StoreStatisticNames} for the field names used here
+ * and elsewhere.
*/
@InterfaceStability.Evolving
public interface CommonStatisticNames {
// The following names are for file system operation invocations
- String OP_APPEND = "op_append";
- String OP_COPY_FROM_LOCAL_FILE = "op_copy_from_local_file";
- String OP_CREATE = "op_create";
- String OP_CREATE_NON_RECURSIVE = "op_create_non_recursive";
- String OP_DELETE = "op_delete";
- String OP_EXISTS = "op_exists";
- String OP_GET_CONTENT_SUMMARY = "op_get_content_summary";
- String OP_GET_DELEGATION_TOKEN = "op_get_delegation_token";
- String OP_GET_FILE_CHECKSUM = "op_get_file_checksum";
- String OP_GET_FILE_STATUS = "op_get_file_status";
- String OP_GET_STATUS = "op_get_status";
- String OP_GLOB_STATUS = "op_glob_status";
- String OP_IS_FILE = "op_is_file";
- String OP_IS_DIRECTORY = "op_is_directory";
- String OP_LIST_FILES = "op_list_files";
- String OP_LIST_LOCATED_STATUS = "op_list_located_status";
- String OP_LIST_STATUS = "op_list_status";
- String OP_MKDIRS = "op_mkdirs";
- String OP_MODIFY_ACL_ENTRIES = "op_modify_acl_entries";
- String OP_OPEN = "op_open";
- String OP_REMOVE_ACL = "op_remove_acl";
- String OP_REMOVE_ACL_ENTRIES = "op_remove_acl_entries";
- String OP_REMOVE_DEFAULT_ACL = "op_remove_default_acl";
- String OP_RENAME = "op_rename";
- String OP_SET_ACL = "op_set_acl";
- String OP_SET_OWNER = "op_set_owner";
- String OP_SET_PERMISSION = "op_set_permission";
- String OP_SET_TIMES = "op_set_times";
- String OP_TRUNCATE = "op_truncate";
+ String OP_APPEND = StoreStatisticNames.OP_APPEND;
+ String OP_COPY_FROM_LOCAL_FILE =
+ StoreStatisticNames.OP_COPY_FROM_LOCAL_FILE;
+ String OP_CREATE = StoreStatisticNames.OP_CREATE;
+ String OP_CREATE_NON_RECURSIVE =
+ StoreStatisticNames.OP_CREATE_NON_RECURSIVE;
+ String OP_DELETE = StoreStatisticNames.OP_DELETE;
+ String OP_EXISTS = StoreStatisticNames.OP_EXISTS;
+ String OP_GET_CONTENT_SUMMARY =
+ StoreStatisticNames.OP_GET_CONTENT_SUMMARY;
+ String OP_GET_DELEGATION_TOKEN =
+ StoreStatisticNames.OP_GET_DELEGATION_TOKEN;
+ String OP_GET_FILE_CHECKSUM = StoreStatisticNames.OP_GET_FILE_CHECKSUM;
+ String OP_GET_FILE_STATUS = StoreStatisticNames.OP_GET_FILE_STATUS;
+ String OP_GET_STATUS = StoreStatisticNames.OP_GET_STATUS;
+ String OP_GLOB_STATUS = StoreStatisticNames.OP_GLOB_STATUS;
+ String OP_IS_FILE = StoreStatisticNames.OP_IS_FILE;
+ String OP_IS_DIRECTORY = StoreStatisticNames.OP_IS_DIRECTORY;
+ String OP_LIST_FILES = StoreStatisticNames.OP_LIST_FILES;
+ String OP_LIST_LOCATED_STATUS =
+ StoreStatisticNames.OP_LIST_LOCATED_STATUS;
+ String OP_LIST_STATUS = StoreStatisticNames.OP_LIST_STATUS;
+ String OP_MKDIRS = StoreStatisticNames.OP_MKDIRS;
+ String OP_MODIFY_ACL_ENTRIES = StoreStatisticNames.OP_MODIFY_ACL_ENTRIES;
+ String OP_OPEN = StoreStatisticNames.OP_OPEN;
+ String OP_REMOVE_ACL = StoreStatisticNames.OP_REMOVE_ACL;
+ String OP_REMOVE_ACL_ENTRIES = StoreStatisticNames.OP_REMOVE_ACL_ENTRIES;
+ String OP_REMOVE_DEFAULT_ACL = StoreStatisticNames.OP_REMOVE_DEFAULT_ACL;
+ String OP_RENAME = StoreStatisticNames.OP_RENAME;
+ String OP_SET_ACL = StoreStatisticNames.OP_SET_ACL;
+ String OP_SET_OWNER = StoreStatisticNames.OP_SET_OWNER;
+ String OP_SET_PERMISSION = StoreStatisticNames.OP_SET_PERMISSION;
+ String OP_SET_TIMES = StoreStatisticNames.OP_SET_TIMES;
+ String OP_TRUNCATE = StoreStatisticNames.OP_TRUNCATE;
}
/**
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StreamCapabilities.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StreamCapabilities.java
index e68e7b351ed78..861178019505e 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StreamCapabilities.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StreamCapabilities.java
@@ -34,7 +34,11 @@
public interface StreamCapabilities {
/**
* Stream hflush capability implemented by {@link Syncable#hflush()}.
+ *
+ * Use the {@link #HSYNC} probe to check for the support of Syncable;
+ * it's that presence of {@code hsync()} which matters.
*/
+ @Deprecated
String HFLUSH = "hflush";
/**
@@ -71,6 +75,18 @@ public interface StreamCapabilities {
*/
String PREADBYTEBUFFER = "in:preadbytebuffer";
+ /**
+ * IOStatisticsSource API.
+ */
+ String IOSTATISTICS = "iostatistics";
+
+ /**
+ * Stream abort() capability implemented by {@link Abortable#abort()}.
+ * This matches the Path Capability
+ * {@link CommonPathCapabilities#ABORTABLE_STREAM}.
+ */
+ String ABORTABLE_STREAM = CommonPathCapabilities.ABORTABLE_STREAM;
+
/**
* Capabilities that a stream can support and be queried for.
*/
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Syncable.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Syncable.java
index 7ec3509ce1df6..9cd458592ca22 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Syncable.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Syncable.java
@@ -23,20 +23,24 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
-/** This interface for flush/sync operation. */
+/**
+ * This is the interface for flush/sync operations.
+ * Consult the Hadoop filesystem specification for the definition of the
+ * semantics of these operations.
+ */
@InterfaceAudience.Public
-@InterfaceStability.Evolving
+@InterfaceStability.Stable
public interface Syncable {
-
+
/** Flush out the data in client's user buffer. After the return of
* this call, new readers will see the data.
* @throws IOException if any error occurs
*/
- public void hflush() throws IOException;
-
+ void hflush() throws IOException;
+
/** Similar to posix fsync, flush out the data in client's user buffer
* all the way to the disk device (but the disk may have it in its cache).
* @throws IOException if error occurs
*/
- public void hsync() throws IOException;
+ void hsync() throws IOException;
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicyDefault.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicyDefault.java
index 18972ea3ecf79..7682992d42590 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicyDefault.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicyDefault.java
@@ -38,7 +38,7 @@
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.util.Time;
-import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/UnionStorageStatistics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/UnionStorageStatistics.java
index 3d5b6af794682..2497ded48e7e9 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/UnionStorageStatistics.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/UnionStorageStatistics.java
@@ -20,7 +20,7 @@
import java.util.Iterator;
import java.util.NoSuchElementException;
-import com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/XAttrCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/XAttrCodec.java
index e15968dd6d273..bfd4daffc7cad 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/XAttrCodec.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/XAttrCodec.java
@@ -25,7 +25,7 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
-import com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
/**
* The value of XAttr is byte[], this class is to
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ftp/FTPFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ftp/FTPFileSystem.java
index 4b144bfddf6c6..6899bb8d87426 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ftp/FTPFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ftp/FTPFileSystem.java
@@ -20,11 +20,12 @@
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InputStream;
+import java.io.OutputStream;
import java.net.ConnectException;
import java.net.URI;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import org.apache.commons.net.ftp.FTP;
import org.apache.commons.net.ftp.FTPClient;
import org.apache.commons.net.ftp.FTPFile;
@@ -41,6 +42,7 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.util.Progressable;
import org.slf4j.Logger;
@@ -110,7 +112,9 @@ public void initialize(URI uri, Configuration conf) throws IOException { // get
// get port information from uri, (overrides info in conf)
int port = uri.getPort();
- port = (port == -1) ? FTP.DEFAULT_PORT : port;
+ if(port == -1){
+ port = conf.getInt(FS_FTP_HOST_PORT, FTP.DEFAULT_PORT);
+ }
conf.setInt(FS_FTP_HOST_PORT, port);
// get user/password information from URI (overrides info in conf)
@@ -340,8 +344,19 @@ public FSDataOutputStream create(Path file, FsPermission permission,
// file. The FTP client connection is closed when close() is called on the
// FSDataOutputStream.
client.changeWorkingDirectory(parent.toUri().getPath());
- FSDataOutputStream fos = new FSDataOutputStream(client.storeFileStream(file
- .getName()), statistics) {
+ OutputStream outputStream = client.storeFileStream(file.getName());
+
+ if (!FTPReply.isPositivePreliminary(client.getReplyCode())) {
+ // The ftpClient is an inconsistent state. Must close the stream
+ // which in turn will logout and disconnect from FTP server
+ if (outputStream != null) {
+ IOUtils.closeStream(outputStream);
+ }
+ disconnect(client);
+ throw new IOException("Unable to create file: " + file + ", Aborting");
+ }
+
+ FSDataOutputStream fos = new FSDataOutputStream(outputStream, statistics) {
@Override
public void close() throws IOException {
super.close();
@@ -356,12 +371,6 @@ public void close() throws IOException {
}
}
};
- if (!FTPReply.isPositivePreliminary(client.getReplyCode())) {
- // The ftpClient is an inconsistent state. Must close the stream
- // which in turn will logout and disconnect from FTP server
- fos.close();
- throw new IOException("Unable to create file: " + file + ", Aborting");
- }
return fos;
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/AbstractFSBuilderImpl.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/AbstractFSBuilderImpl.java
index 5fc92e97be76c..9cf8b3dc4d203 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/AbstractFSBuilderImpl.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/AbstractFSBuilderImpl.java
@@ -26,7 +26,7 @@
import java.util.Optional;
import java.util.Set;
-import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
@@ -36,8 +36,8 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathHandle;
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkArgument;
+import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull;
/**
* Builder for filesystem/filecontext operations of various kinds,
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/AbstractMultipartUploader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/AbstractMultipartUploader.java
new file mode 100644
index 0000000000000..ed4bcc84e96f9
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/AbstractMultipartUploader.java
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.impl;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+
+import org.apache.hadoop.fs.MultipartUploader;
+import org.apache.hadoop.fs.PartHandle;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.UploadHandle;
+
+import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkArgument;
+
+/**
+ * Standard base class for Multipart Uploaders.
+ */
+public abstract class AbstractMultipartUploader implements MultipartUploader {
+
+ /**
+ * Base path of upload.
+ */
+ private final Path basePath;
+
+ /**
+ * Instantiate.
+ * @param basePath base path
+ */
+ protected AbstractMultipartUploader(final Path basePath) {
+ this.basePath = Objects.requireNonNull(basePath, "null path");
+ }
+
+ /**
+ * Perform any cleanup.
+ * The upload is not required to support any operations after this.
+ * @throws IOException problems on close.
+ */
+ @Override
+ public void close() throws IOException {
+ }
+
+ protected Path getBasePath() {
+ return basePath;
+ }
+
+ /**
+ * Validate a path.
+ * @param path path to check.
+ */
+ protected void checkPath(Path path) {
+ Objects.requireNonNull(path, "null path");
+ Preconditions.checkArgument(path.toString().startsWith(basePath.toString()),
+ "Path %s is not under %s", path, basePath);
+ }
+
+ /**
+ * Utility method to validate uploadIDs.
+ * @param uploadId Upload ID
+ * @throws IllegalArgumentException invalid ID
+ */
+ protected void checkUploadId(byte[] uploadId)
+ throws IllegalArgumentException {
+ checkArgument(uploadId != null, "null uploadId");
+ checkArgument(uploadId.length > 0,
+ "Empty UploadId is not valid");
+ }
+
+ /**
+ * Utility method to validate partHandles.
+ * @param partHandles handles
+ * @throws IllegalArgumentException if the parts are invalid
+ */
+ protected void checkPartHandles(Map partHandles) {
+ checkArgument(!partHandles.isEmpty(),
+ "Empty upload");
+ partHandles.keySet()
+ .stream()
+ .forEach(key ->
+ checkArgument(key > 0,
+ "Invalid part handle index %s", key));
+ }
+
+ /**
+ * Check all the arguments to the
+ * {@link MultipartUploader#putPart(UploadHandle, int, Path, InputStream, long)}
+ * operation.
+ * @param filePath Target path for upload (as {@link #startUpload(Path)}).
+ * @param inputStream Data for this part. Implementations MUST close this
+ * stream after reading in the data.
+ * @param partNumber Index of the part relative to others.
+ * @param uploadId Identifier from {@link #startUpload(Path)}.
+ * @param lengthInBytes Target length to read from the stream.
+ * @throws IllegalArgumentException invalid argument
+ */
+ protected void checkPutArguments(Path filePath,
+ InputStream inputStream,
+ int partNumber,
+ UploadHandle uploadId,
+ long lengthInBytes) throws IllegalArgumentException {
+ checkPath(filePath);
+ checkArgument(inputStream != null, "null inputStream");
+ checkArgument(partNumber > 0, "Invalid part number: %d", partNumber);
+ checkArgument(uploadId != null, "null uploadId");
+ checkArgument(lengthInBytes >= 0, "Invalid part length: %d", lengthInBytes);
+ }
+
+ /**
+ * {@inheritDoc}.
+ * @param path path to abort uploads under.
+ * @return a future to -1.
+ * @throws IOException
+ */
+ public CompletableFuture abortUploadsUnderPath(Path path)
+ throws IOException {
+ checkPath(path);
+ CompletableFuture f = new CompletableFuture<>();
+ f.complete(-1);
+ return f;
+ }
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystemMultipartUploader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FileSystemMultipartUploader.java
similarity index 51%
rename from hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystemMultipartUploader.java
rename to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FileSystemMultipartUploader.java
index b77c244220a9e..7c5a5d949a072 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystemMultipartUploader.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FileSystemMultipartUploader.java
@@ -14,24 +14,42 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.hadoop.fs;
+
+package org.apache.hadoop.fs.impl;
import java.io.IOException;
import java.io.InputStream;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Comparator;
+import java.util.HashSet;
import java.util.List;
import java.util.Map;
+import java.util.Set;
import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
import java.util.stream.Collectors;
-import com.google.common.base.Charsets;
+import org.apache.hadoop.thirdparty.com.google.common.base.Charsets;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
import org.apache.commons.compress.utils.IOUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BBPartHandle;
+import org.apache.hadoop.fs.BBUploadHandle;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSDataOutputStreamBuilder;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.InternalOperations;
+import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.PartHandle;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathHandle;
+import org.apache.hadoop.fs.UploadHandle;
import org.apache.hadoop.fs.permission.FsPermission;
import static org.apache.hadoop.fs.Path.mergePaths;
@@ -50,40 +68,82 @@
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
-public class FileSystemMultipartUploader extends MultipartUploader {
+public class FileSystemMultipartUploader extends AbstractMultipartUploader {
+
+ private static final Logger LOG = LoggerFactory.getLogger(
+ FileSystemMultipartUploader.class);
private final FileSystem fs;
- public FileSystemMultipartUploader(FileSystem fs) {
+ private final FileSystemMultipartUploaderBuilder builder;
+
+ private final FsPermission permission;
+
+ private final long blockSize;
+
+ private final Options.ChecksumOpt checksumOpt;
+
+ public FileSystemMultipartUploader(
+ final FileSystemMultipartUploaderBuilder builder,
+ FileSystem fs) {
+ super(builder.getPath());
+ this.builder = builder;
this.fs = fs;
+ blockSize = builder.getBlockSize();
+ checksumOpt = builder.getChecksumOpt();
+ permission = builder.getPermission();
}
@Override
- public UploadHandle initialize(Path filePath) throws IOException {
- Path collectorPath = createCollectorPath(filePath);
- fs.mkdirs(collectorPath, FsPermission.getDirDefault());
+ public CompletableFuture startUpload(Path filePath)
+ throws IOException {
+ checkPath(filePath);
+ return FutureIOSupport.eval(() -> {
+ Path collectorPath = createCollectorPath(filePath);
+ fs.mkdirs(collectorPath, FsPermission.getDirDefault());
- ByteBuffer byteBuffer = ByteBuffer.wrap(
- collectorPath.toString().getBytes(Charsets.UTF_8));
- return BBUploadHandle.from(byteBuffer);
+ ByteBuffer byteBuffer = ByteBuffer.wrap(
+ collectorPath.toString().getBytes(Charsets.UTF_8));
+ return BBUploadHandle.from(byteBuffer);
+ });
}
@Override
- public PartHandle putPart(Path filePath, InputStream inputStream,
- int partNumber, UploadHandle uploadId, long lengthInBytes)
+ public CompletableFuture putPart(UploadHandle uploadId,
+ int partNumber, Path filePath,
+ InputStream inputStream,
+ long lengthInBytes)
throws IOException {
checkPutArguments(filePath, inputStream, partNumber, uploadId,
lengthInBytes);
+ return FutureIOSupport.eval(() -> innerPutPart(filePath,
+ inputStream, partNumber, uploadId, lengthInBytes));
+ }
+
+ private PartHandle innerPutPart(Path filePath,
+ InputStream inputStream,
+ int partNumber,
+ UploadHandle uploadId,
+ long lengthInBytes)
+ throws IOException {
byte[] uploadIdByteArray = uploadId.toByteArray();
checkUploadId(uploadIdByteArray);
Path collectorPath = new Path(new String(uploadIdByteArray, 0,
uploadIdByteArray.length, Charsets.UTF_8));
Path partPath =
mergePaths(collectorPath, mergePaths(new Path(Path.SEPARATOR),
- new Path(Integer.toString(partNumber) + ".part")));
- try(FSDataOutputStream fsDataOutputStream =
- fs.createFile(partPath).build()) {
- IOUtils.copy(inputStream, fsDataOutputStream, 4096);
+ new Path(partNumber + ".part")));
+ final FSDataOutputStreamBuilder fileBuilder = fs.createFile(partPath);
+ if (checksumOpt != null) {
+ fileBuilder.checksumOpt(checksumOpt);
+ }
+ if (permission != null) {
+ fileBuilder.permission(permission);
+ }
+ try (FSDataOutputStream fsDataOutputStream =
+ fileBuilder.blockSize(blockSize).build()) {
+ IOUtils.copy(inputStream, fsDataOutputStream,
+ this.builder.getBufferSize());
} finally {
cleanupWithLogger(LOG, inputStream);
}
@@ -106,16 +166,36 @@ private PathHandle getPathHandle(Path filePath) throws IOException {
private long totalPartsLen(List partHandles) throws IOException {
long totalLen = 0;
- for (Path p: partHandles) {
+ for (Path p : partHandles) {
totalLen += fs.getFileStatus(p).getLen();
}
return totalLen;
}
@Override
- @SuppressWarnings("deprecation") // rename w/ OVERWRITE
- public PathHandle complete(Path filePath, Map handleMap,
- UploadHandle multipartUploadId) throws IOException {
+ public CompletableFuture complete(
+ UploadHandle uploadId,
+ Path filePath,
+ Map handleMap) throws IOException {
+
+ checkPath(filePath);
+ return FutureIOSupport.eval(() ->
+ innerComplete(uploadId, filePath, handleMap));
+ }
+
+ /**
+ * The upload complete operation.
+ * @param multipartUploadId the ID of the upload
+ * @param filePath path
+ * @param handleMap map of handles
+ * @return the path handle
+ * @throws IOException failure
+ */
+ private PathHandle innerComplete(
+ UploadHandle multipartUploadId, Path filePath,
+ Map handleMap) throws IOException {
+
+ checkPath(filePath);
checkUploadId(multipartUploadId.toByteArray());
@@ -133,6 +213,13 @@ public PathHandle complete(Path filePath, Map handleMap,
})
.collect(Collectors.toList());
+ int count = partHandles.size();
+ // built up to identify duplicates -if the size of this set is
+ // below that of the number of parts, then there's a duplicate entry.
+ Set values = new HashSet<>(count);
+ values.addAll(partHandles);
+ Preconditions.checkArgument(values.size() == count,
+ "Duplicate PartHandles");
byte[] uploadIdByteArray = multipartUploadId.toByteArray();
Path collectorPath = new Path(new String(uploadIdByteArray, 0,
uploadIdByteArray.length, Charsets.UTF_8));
@@ -146,35 +233,30 @@ public PathHandle complete(Path filePath, Map handleMap,
fs.create(filePathInsideCollector).close();
fs.concat(filePathInsideCollector,
partHandles.toArray(new Path[handles.size()]));
- fs.rename(filePathInsideCollector, filePath, Options.Rename.OVERWRITE);
+ new InternalOperations()
+ .rename(fs, filePathInsideCollector, filePath,
+ Options.Rename.OVERWRITE);
}
fs.delete(collectorPath, true);
return getPathHandle(filePath);
}
@Override
- public void abort(Path filePath, UploadHandle uploadId) throws IOException {
+ public CompletableFuture abort(UploadHandle uploadId,
+ Path filePath)
+ throws IOException {
+ checkPath(filePath);
byte[] uploadIdByteArray = uploadId.toByteArray();
checkUploadId(uploadIdByteArray);
Path collectorPath = new Path(new String(uploadIdByteArray, 0,
uploadIdByteArray.length, Charsets.UTF_8));
- // force a check for a file existing; raises FNFE if not found
- fs.getFileStatus(collectorPath);
- fs.delete(collectorPath, true);
- }
-
- /**
- * Factory for creating MultipartUploaderFactory objects for file://
- * filesystems.
- */
- public static class Factory extends MultipartUploaderFactory {
- protected MultipartUploader createMultipartUploader(FileSystem fs,
- Configuration conf) {
- if (fs.getScheme().equals("file")) {
- return new FileSystemMultipartUploader(fs);
- }
+ return FutureIOSupport.eval(() -> {
+ // force a check for a file existing; raises FNFE if not found
+ fs.getFileStatus(collectorPath);
+ fs.delete(collectorPath, true);
return null;
- }
+ });
}
+
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FileSystemMultipartUploaderBuilder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FileSystemMultipartUploaderBuilder.java
new file mode 100644
index 0000000000000..7c4d995c69d1b
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FileSystemMultipartUploaderBuilder.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.impl;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.util.EnumSet;
+
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+/**
+ * Builder for {@link FileSystemMultipartUploader}.
+ */
+public class FileSystemMultipartUploaderBuilder extends
+ MultipartUploaderBuilderImpl {
+
+ public FileSystemMultipartUploaderBuilder(
+ @Nonnull final FileSystem fileSystem,
+ @Nonnull final Path path) {
+ super(fileSystem, path);
+ }
+
+ @Override
+ public FileSystemMultipartUploaderBuilder getThisBuilder() {
+ return this;
+ }
+
+ @Override
+ public FileSystemMultipartUploader build()
+ throws IllegalArgumentException, IOException {
+ return new FileSystemMultipartUploader(this, getFS());
+ }
+
+ @Override
+ public FileSystem getFS() {
+ return super.getFS();
+ }
+
+ @Override
+ public FsPermission getPermission() {
+ return super.getPermission();
+ }
+
+ @Override
+ public int getBufferSize() {
+ return super.getBufferSize();
+ }
+
+ @Override
+ public short getReplication() {
+ return super.getReplication();
+ }
+
+ @Override
+ public EnumSet getFlags() {
+ return super.getFlags();
+ }
+
+ @Override
+ public Options.ChecksumOpt getChecksumOpt() {
+ return super.getChecksumOpt();
+ }
+
+ @Override
+ protected long getBlockSize() {
+ return super.getBlockSize();
+ }
+
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FsLinkResolution.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FsLinkResolution.java
index f5ef8c4923328..8d4bebda15096 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FsLinkResolution.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FsLinkResolution.java
@@ -20,7 +20,7 @@
import java.io.IOException;
-import com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FunctionsRaisingIOE.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FunctionsRaisingIOE.java
index 7bbb34622647d..551cf9cff3d6f 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FunctionsRaisingIOE.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FunctionsRaisingIOE.java
@@ -24,7 +24,8 @@
import org.apache.hadoop.classification.InterfaceStability;
/**
- * Evolving support for functional programming/lambda-expressions.
+ * Support for functional programming/lambda-expressions.
+ * @deprecated use {@code org.apache.hadoop.util.functional}
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
@@ -37,6 +38,7 @@ private FunctionsRaisingIOE() {
* Function of arity 1 which may raise an IOException.
* @param type of arg1
* @param type of return value.
+ * @deprecated use {@link org.apache.hadoop.util.functional.FunctionRaisingIOE}
*/
@FunctionalInterface
public interface FunctionRaisingIOE {
@@ -49,6 +51,7 @@ public interface FunctionRaisingIOE {
* @param type of arg1
* @param type of arg2
* @param type of return value.
+ * @deprecated use {@link org.apache.hadoop.util.functional.BiFunctionRaisingIOE}
*/
@FunctionalInterface
public interface BiFunctionRaisingIOE {
@@ -59,6 +62,7 @@ public interface BiFunctionRaisingIOE {
/**
* This is a callable which only raises an IOException.
* @param return type
+ * @deprecated use {@link org.apache.hadoop.util.functional.CallableRaisingIOE}
*/
@FunctionalInterface
public interface CallableRaisingIOE {
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureIOSupport.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureIOSupport.java
index 26856e5b935e0..fe112d59352f5 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureIOSupport.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureIOSupport.java
@@ -21,6 +21,7 @@
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.Map;
+import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
@@ -31,9 +32,16 @@
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSBuilder;
+import org.apache.hadoop.util.functional.CallableRaisingIOE;
+import org.apache.hadoop.util.functional.FutureIO;
/**
* Support for future IO and the FS Builder subclasses.
+ * If methods in here are needed for applications, promote
+ * to {@link FutureIO} for public use -with the original
+ * method relaying to it. This is to ensure that external
+ * filesystem implementations can safely use these methods
+ * without linkage problems surfacing.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
@@ -52,16 +60,9 @@ private FutureIOSupport() {
* @throws IOException if something went wrong
* @throws RuntimeException any nested RTE thrown
*/
- public static T awaitFuture(final Future future)
+ public static T awaitFuture(final Future future)
throws InterruptedIOException, IOException, RuntimeException {
- try {
- return future.get();
- } catch (InterruptedException e) {
- throw (InterruptedIOException)new InterruptedIOException(e.toString())
- .initCause(e);
- } catch (ExecutionException e) {
- return raiseInnerCause(e);
- }
+ return FutureIO.awaitFuture(future);
}
@@ -81,18 +82,9 @@ public static T awaitFuture(final Future future,
final TimeUnit unit)
throws InterruptedIOException, IOException, RuntimeException,
TimeoutException {
-
- try {
- return future.get(timeout, unit);
- } catch (InterruptedException e) {
- throw (InterruptedIOException)new InterruptedIOException(e.toString())
- .initCause(e);
- } catch (ExecutionException e) {
- return raiseInnerCause(e);
- }
+ return FutureIO.awaitFuture(future, timeout, unit);
}
-
/**
* From the inner cause of an execution exception, extract the inner cause
* if it is an IOE or RTE.
@@ -109,7 +101,7 @@ public static T awaitFuture(final Future future,
*/
public static T raiseInnerCause(final ExecutionException e)
throws IOException {
- throw unwrapInnerException(e);
+ return FutureIO.raiseInnerCause(e);
}
/**
@@ -124,41 +116,7 @@ public static T raiseInnerCause(final ExecutionException e)
*/
public static T raiseInnerCause(final CompletionException e)
throws IOException {
- throw unwrapInnerException(e);
- }
-
- /**
- * From the inner cause of an execution exception, extract the inner cause.
- * If it is an RTE: throw immediately.
- * If it is an IOE: Return.
- * If it is a WrappedIOException: Unwrap and return
- * Else: create a new IOException.
- *
- * Recursively handles wrapped Execution and Completion Exceptions in
- * case something very complicated has happened.
- * @param e exception.
- * @return an IOException extracted or built from the cause.
- * @throws RuntimeException if that is the inner cause.
- */
- private static IOException unwrapInnerException(final Throwable e) {
- Throwable cause = e.getCause();
- if (cause instanceof IOException) {
- return (IOException) cause;
- } else if (cause instanceof WrappedIOException) {
- return ((WrappedIOException) cause).getCause();
- } else if (cause instanceof CompletionException) {
- return unwrapInnerException(cause);
- } else if (cause instanceof ExecutionException) {
- return unwrapInnerException(cause);
- } else if (cause instanceof RuntimeException) {
- throw (RuntimeException) cause;
- } else if (cause != null) {
- // other type: wrap with a new IOE
- return new IOException(cause);
- } else {
- // this only happens if there was no cause.
- return new IOException(e);
- }
+ return FutureIO.raiseInnerCause(e);
}
/**
@@ -224,4 +182,29 @@ public static void propagateOptions(
}
}
}
+
+ /**
+ * Evaluate a CallableRaisingIOE in the current thread,
+ * converting IOEs to RTEs and propagating.
+ * @param callable callable to invoke
+ * @param Return type.
+ * @return the evaluated result.
+ * @throws UnsupportedOperationException fail fast if unsupported
+ * @throws IllegalArgumentException invalid argument
+ */
+ public static CompletableFuture eval(
+ CallableRaisingIOE callable) {
+ CompletableFuture result = new CompletableFuture<>();
+ try {
+ result.complete(callable.apply());
+ } catch (UnsupportedOperationException | IllegalArgumentException tx) {
+ // fail fast here
+ throw tx;
+ } catch (Throwable tx) {
+ // fail lazily here to ensure callers expect all File IO operations to
+ // surface later
+ result.completeExceptionally(tx);
+ }
+ return result;
+ }
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/MultipartUploaderBuilderImpl.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/MultipartUploaderBuilderImpl.java
new file mode 100644
index 0000000000000..88c573acc4cb3
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/MultipartUploaderBuilderImpl.java
@@ -0,0 +1,215 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.impl;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.util.EnumSet;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FsServerDefaults;
+import org.apache.hadoop.fs.MultipartUploader;
+import org.apache.hadoop.fs.MultipartUploaderBuilder;
+import org.apache.hadoop.fs.Options.ChecksumOpt;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
+
+/**
+ * Builder for {@link MultipartUploader} implementations.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public abstract class MultipartUploaderBuilderImpl
+ >
+ extends AbstractFSBuilderImpl
+ implements MultipartUploaderBuilder {
+
+ private final FileSystem fs;
+
+ private FsPermission permission;
+
+ private int bufferSize;
+
+ private short replication;
+
+ private long blockSize;
+
+ private final EnumSet flags = EnumSet.noneOf(CreateFlag.class);
+
+ private ChecksumOpt checksumOpt;
+
+ /**
+ * Return the concrete implementation of the builder instance.
+ */
+ public abstract B getThisBuilder();
+
+ /**
+ * Construct from a {@link FileContext}.
+ *
+ * @param fc FileContext
+ * @param p path.
+ * @throws IOException failure
+ */
+ protected MultipartUploaderBuilderImpl(@Nonnull FileContext fc,
+ @Nonnull Path p) throws IOException {
+ super(checkNotNull(p));
+ checkNotNull(fc);
+ this.fs = null;
+
+ FsServerDefaults defaults = fc.getServerDefaults(p);
+ bufferSize = defaults.getFileBufferSize();
+ replication = defaults.getReplication();
+ blockSize = defaults.getBlockSize();
+ }
+
+ /**
+ * Constructor.
+ */
+ protected MultipartUploaderBuilderImpl(@Nonnull FileSystem fileSystem,
+ @Nonnull Path p) {
+ super(fileSystem.makeQualified(checkNotNull(p)));
+ checkNotNull(fileSystem);
+ fs = fileSystem;
+ bufferSize = fs.getConf().getInt(IO_FILE_BUFFER_SIZE_KEY,
+ IO_FILE_BUFFER_SIZE_DEFAULT);
+ replication = fs.getDefaultReplication(p);
+ blockSize = fs.getDefaultBlockSize(p);
+ }
+
+ protected FileSystem getFS() {
+ checkNotNull(fs);
+ return fs;
+ }
+
+ protected FsPermission getPermission() {
+ if (permission == null) {
+ permission = FsPermission.getFileDefault();
+ }
+ return permission;
+ }
+
+ /**
+ * Set permission for the file.
+ */
+ @Override
+ public B permission(@Nonnull final FsPermission perm) {
+ checkNotNull(perm);
+ permission = perm;
+ return getThisBuilder();
+ }
+
+ protected int getBufferSize() {
+ return bufferSize;
+ }
+
+ /**
+ * Set the size of the buffer to be used.
+ */
+ @Override
+ public B bufferSize(int bufSize) {
+ bufferSize = bufSize;
+ return getThisBuilder();
+ }
+
+ protected short getReplication() {
+ return replication;
+ }
+
+ /**
+ * Set replication factor.
+ */
+ @Override
+ public B replication(short replica) {
+ replication = replica;
+ return getThisBuilder();
+ }
+
+ protected long getBlockSize() {
+ return blockSize;
+ }
+
+ /**
+ * Set block size.
+ */
+ @Override
+ public B blockSize(long blkSize) {
+ blockSize = blkSize;
+ return getThisBuilder();
+ }
+
+ protected EnumSet getFlags() {
+ return flags;
+ }
+
+ /**
+ * Create an FSDataOutputStream at the specified path.
+ */
+ @Override
+ public B create() {
+ flags.add(CreateFlag.CREATE);
+ return getThisBuilder();
+ }
+
+ /**
+ * Set to true to overwrite the existing file.
+ * Set it to false, an exception will be thrown when calling {@link #build()}
+ * if the file exists.
+ */
+ @Override
+ public B overwrite(boolean overwrite) {
+ if (overwrite) {
+ flags.add(CreateFlag.OVERWRITE);
+ } else {
+ flags.remove(CreateFlag.OVERWRITE);
+ }
+ return getThisBuilder();
+ }
+
+ /**
+ * Append to an existing file (optional operation).
+ */
+ @Override
+ public B append() {
+ flags.add(CreateFlag.APPEND);
+ return getThisBuilder();
+ }
+
+ protected ChecksumOpt getChecksumOpt() {
+ return checksumOpt;
+ }
+
+ /**
+ * Set checksum opt.
+ */
+ @Override
+ public B checksumOpt(@Nonnull final ChecksumOpt chksumOpt) {
+ checkNotNull(chksumOpt);
+ checksumOpt = chksumOpt;
+ return getThisBuilder();
+ }
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/PathCapabilitiesSupport.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/PathCapabilitiesSupport.java
index 9332ac6e7eedb..1e3e43581dccc 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/PathCapabilitiesSupport.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/PathCapabilitiesSupport.java
@@ -25,7 +25,7 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathCapabilities;
-import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkArgument;
@InterfaceAudience.Private
@InterfaceStability.Evolving
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/StoreImplementationUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/StoreImplementationUtils.java
new file mode 100644
index 0000000000000..605a3538d8b6b
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/StoreImplementationUtils.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License
+ */
+
+package org.apache.hadoop.fs.impl;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.StreamCapabilities;
+
+import static org.apache.hadoop.fs.StreamCapabilities.HFLUSH;
+import static org.apache.hadoop.fs.StreamCapabilities.HSYNC;
+
+/**
+ * Utility classes to help implementing filesystems and streams.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public final class StoreImplementationUtils {
+
+ private StoreImplementationUtils() {
+ }
+
+ /**
+ * Check the probe capability being for {@link StreamCapabilities#HSYNC}
+ * or {@link StreamCapabilities#HFLUSH}
+ * {@code Syncable.hsync()} and {@code Syncable.hflush()} functionality.
+ * @param capability capability string.
+ * @return true if either refers to one of the Syncable operations.
+ */
+ public static boolean isProbeForSyncable(String capability) {
+ return capability.equalsIgnoreCase(HSYNC) ||
+ capability.equalsIgnoreCase(HFLUSH);
+ }
+
+ /**
+ * Probe for an object having a capability; returns true
+ * if the stream implements {@link StreamCapabilities} and its
+ * {@code hasCapabilities()} method returns true for the capability.
+ * This is a package private method intended to provided a common
+ * implementation for input and output streams.
+ * {@link StreamCapabilities#hasCapability(String)} call is for public use.
+ * @param object object to probe.
+ * @param capability capability to probe for
+ * @return true if the object implements stream capabilities and
+ * declares that it supports the capability.
+ */
+ static boolean objectHasCapability(Object object, String capability) {
+ if (object instanceof StreamCapabilities) {
+ return ((StreamCapabilities) object).hasCapability(capability);
+ }
+ return false;
+ }
+
+ /**
+ * Probe for an output stream having a capability; returns true
+ * if the stream implements {@link StreamCapabilities} and its
+ * {@code hasCapabilities()} method returns true for the capability.
+ * @param out output stream
+ * @param capability capability to probe for
+ * @return true if the stream declares that it supports the capability.
+ */
+ public static boolean hasCapability(OutputStream out, String capability) {
+ return objectHasCapability(out, capability);
+ }
+
+ /**
+ * Probe for an input stream having a capability; returns true
+ * if the stream implements {@link StreamCapabilities} and its
+ * {@code hasCapabilities()} method returns true for the capability.
+ * @param in input stream
+ * @param capability capability to probe for
+ * @return true if the stream declares that it supports the capability.
+ */
+ public static boolean hasCapability(InputStream in, String capability) {
+ return objectHasCapability(in, capability);
+ }
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/WrappedIOException.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/WrappedIOException.java
index 1de1ecb785368..2fcdee915ede9 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/WrappedIOException.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/WrappedIOException.java
@@ -19,9 +19,10 @@
package org.apache.hadoop.fs.impl;
import java.io.IOException;
+import java.io.UncheckedIOException;
import java.util.concurrent.ExecutionException;
-import com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
@@ -33,10 +34,12 @@
*
* The constructor signature guarantees the cause will be an IOException,
* and as it checks for a null-argument, non-null.
+ * @deprecated use the {@code UncheckedIOException}.
*/
+@Deprecated
@InterfaceAudience.Private
@InterfaceStability.Unstable
-public class WrappedIOException extends RuntimeException {
+public class WrappedIOException extends UncheckedIOException {
private static final long serialVersionUID = 2510210974235779294L;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclEntry.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclEntry.java
index a902488377239..04a045299b7ea 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclEntry.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclEntry.java
@@ -21,7 +21,7 @@
import java.util.Collection;
import java.util.List;
-import com.google.common.base.Objects;
+import org.apache.hadoop.thirdparty.com.google.common.base.Objects;
import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.classification.InterfaceAudience;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclStatus.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclStatus.java
index 385fed21d4194..674b88083d3dc 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclStatus.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclStatus.java
@@ -22,9 +22,9 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
-import com.google.common.base.Objects;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
+import org.apache.hadoop.thirdparty.com.google.common.base.Objects;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
/**
* An AclStatus contains the ACL information of a specific file. AclStatus
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclUtil.java
index 42492520dceaa..58b24f200429b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclUtil.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclUtil.java
@@ -23,7 +23,7 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
-import com.google.common.collect.Lists;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
/**
* AclUtil contains utility methods for manipulating ACLs.
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPFileSystem.java
index ed33357b51d2b..297ec048e50c7 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPFileSystem.java
@@ -19,12 +19,12 @@
import java.io.FileNotFoundException;
import java.io.IOException;
-import java.io.InputStream;
import java.io.OutputStream;
import java.net.URI;
import java.net.URLDecoder;
import java.util.ArrayList;
import java.util.Vector;
+import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
@@ -35,7 +35,7 @@
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.util.Progressable;
-import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
import com.jcraft.jsch.ChannelSftp;
import com.jcraft.jsch.ChannelSftp.LsEntry;
import com.jcraft.jsch.SftpATTRS;
@@ -51,6 +51,7 @@ public class SFTPFileSystem extends FileSystem {
private SFTPConnectionPool connectionPool;
private URI uri;
+ private final AtomicBoolean closed = new AtomicBoolean(false);
private static final int DEFAULT_SFTP_PORT = 22;
private static final int DEFAULT_MAX_CONNECTION = 5;
@@ -84,6 +85,7 @@ public class SFTPFileSystem extends FileSystem {
"Destination path %s already exist, cannot rename!";
public static final String E_FAILED_GETHOME = "Failed to get home directory";
public static final String E_FAILED_DISCONNECT = "Failed to disconnect";
+ public static final String E_FS_CLOSED = "FileSystem is closed!";
/**
* Set configuration from UI.
@@ -139,8 +141,9 @@ private void setConfigurationFromURI(URI uriInfo, Configuration conf)
* @throws IOException
*/
private ChannelSftp connect() throws IOException {
- Configuration conf = getConf();
+ checkNotClosed();
+ Configuration conf = getConf();
String host = conf.get(FS_SFTP_HOST, null);
int port = conf.getInt(FS_SFTP_HOST_PORT, DEFAULT_SFTP_PORT);
String user = conf.get(FS_SFTP_USER_PREFIX + host, null);
@@ -516,20 +519,21 @@ public FSDataInputStream open(Path f, int bufferSize) throws IOException {
disconnect(channel);
throw new IOException(String.format(E_PATH_DIR, f));
}
- InputStream is;
try {
// the path could be a symbolic link, so get the real path
absolute = new Path("/", channel.realpath(absolute.toUri().getPath()));
-
- is = channel.get(absolute.toUri().getPath());
} catch (SftpException e) {
throw new IOException(e);
}
- return new FSDataInputStream(new SFTPInputStream(is, statistics)){
+ return new FSDataInputStream(
+ new SFTPInputStream(channel, absolute, statistics)){
@Override
public void close() throws IOException {
- super.close();
- disconnect(channel);
+ try {
+ super.close();
+ } finally {
+ disconnect(channel);
+ }
}
};
}
@@ -703,6 +707,31 @@ public FileStatus getFileStatus(Path f) throws IOException {
}
}
+ @Override
+ public void close() throws IOException {
+ if (closed.getAndSet(true)) {
+ return;
+ }
+ try {
+ super.close();
+ } finally {
+ if (connectionPool != null) {
+ connectionPool.shutdown();
+ }
+ }
+ }
+
+ /**
+ * Verify that the input stream is open. Non blocking; this gives
+ * the last state of the volatile {@link #closed} field.
+ * @throws IOException if the connection is closed.
+ */
+ private void checkNotClosed() throws IOException {
+ if (closed.get()) {
+ throw new IOException(uri + ": " + E_FS_CLOSED);
+ }
+ }
+
@VisibleForTesting
SFTPConnectionPool getConnectionPool() {
return connectionPool;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPInputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPInputStream.java
index 7af299bd113e1..d0f9a8d0887ca 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPInputStream.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPInputStream.java
@@ -15,62 +15,107 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
+
package org.apache.hadoop.fs.sftp;
+import java.io.EOFException;
import java.io.IOException;
import java.io.InputStream;
+import com.jcraft.jsch.ChannelSftp;
+import com.jcraft.jsch.SftpATTRS;
+import com.jcraft.jsch.SftpException;
+
+import org.apache.hadoop.fs.FSExceptionMessages;
import org.apache.hadoop.fs.FSInputStream;
import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
/** SFTP FileSystem input stream. */
class SFTPInputStream extends FSInputStream {
- public static final String E_SEEK_NOTSUPPORTED = "Seek not supported";
- public static final String E_NULL_INPUTSTREAM = "Null InputStream";
- public static final String E_STREAM_CLOSED = "Stream closed";
-
+ private final ChannelSftp channel;
+ private final Path path;
private InputStream wrappedStream;
private FileSystem.Statistics stats;
private boolean closed;
private long pos;
+ private long nextPos;
+ private long contentLength;
- SFTPInputStream(InputStream stream, FileSystem.Statistics stats) {
-
- if (stream == null) {
- throw new IllegalArgumentException(E_NULL_INPUTSTREAM);
+ SFTPInputStream(ChannelSftp channel, Path path, FileSystem.Statistics stats)
+ throws IOException {
+ try {
+ this.channel = channel;
+ this.path = path;
+ this.stats = stats;
+ this.wrappedStream = channel.get(path.toUri().getPath());
+ SftpATTRS stat = channel.lstat(path.toString());
+ this.contentLength = stat.getSize();
+ } catch (SftpException e) {
+ throw new IOException(e);
}
- this.wrappedStream = stream;
- this.stats = stats;
+ }
- this.pos = 0;
- this.closed = false;
+ @Override
+ public synchronized void seek(long position) throws IOException {
+ checkNotClosed();
+ if (position < 0) {
+ throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK);
+ }
+ nextPos = position;
}
@Override
- public void seek(long position) throws IOException {
- throw new IOException(E_SEEK_NOTSUPPORTED);
+ public synchronized int available() throws IOException {
+ checkNotClosed();
+ long remaining = contentLength - nextPos;
+ if (remaining > Integer.MAX_VALUE) {
+ return Integer.MAX_VALUE;
+ }
+ return (int) remaining;
+ }
+
+ private void seekInternal() throws IOException {
+ if (pos == nextPos) {
+ return;
+ }
+ if (nextPos > pos) {
+ long skipped = wrappedStream.skip(nextPos - pos);
+ pos = pos + skipped;
+ }
+ if (nextPos < pos) {
+ wrappedStream.close();
+ try {
+ wrappedStream = channel.get(path.toUri().getPath());
+ pos = wrappedStream.skip(nextPos);
+ } catch (SftpException e) {
+ throw new IOException(e);
+ }
+ }
}
@Override
public boolean seekToNewSource(long targetPos) throws IOException {
- throw new IOException(E_SEEK_NOTSUPPORTED);
+ return false;
}
@Override
- public long getPos() throws IOException {
- return pos;
+ public synchronized long getPos() throws IOException {
+ return nextPos;
}
@Override
public synchronized int read() throws IOException {
- if (closed) {
- throw new IOException(E_STREAM_CLOSED);
+ checkNotClosed();
+ if (this.contentLength == 0 || (nextPos >= contentLength)) {
+ return -1;
}
-
+ seekInternal();
int byteRead = wrappedStream.read();
if (byteRead >= 0) {
pos++;
+ nextPos++;
}
if (stats != null & byteRead >= 0) {
stats.incrementBytesRead(1);
@@ -78,23 +123,6 @@ public synchronized int read() throws IOException {
return byteRead;
}
- public synchronized int read(byte[] buf, int off, int len)
- throws IOException {
- if (closed) {
- throw new IOException(E_STREAM_CLOSED);
- }
-
- int result = wrappedStream.read(buf, off, len);
- if (result > 0) {
- pos += result;
- }
- if (stats != null & result > 0) {
- stats.incrementBytesRead(result);
- }
-
- return result;
- }
-
public synchronized void close() throws IOException {
if (closed) {
return;
@@ -103,4 +131,12 @@ public synchronized void close() throws IOException {
wrappedStream.close();
closed = true;
}
+
+ private void checkNotClosed() throws IOException {
+ if (closed) {
+ throw new IOException(
+ path.toUri() + ": " + FSExceptionMessages.STREAM_IS_CLOSED
+ );
+ }
+ }
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/AclCommands.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/AclCommands.java
index 5a60ef2ae9b03..dcff0094eccf5 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/AclCommands.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/AclCommands.java
@@ -22,7 +22,7 @@
import java.util.LinkedList;
import java.util.List;
-import com.google.common.collect.Lists;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.classification.InterfaceAudience;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandWithDestination.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandWithDestination.java
index ca9961aeb65a0..90a709dffc0c1 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandWithDestination.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandWithDestination.java
@@ -54,6 +54,7 @@
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
import static org.apache.hadoop.fs.CreateFlag.CREATE;
import static org.apache.hadoop.fs.CreateFlag.LAZY_PERSIST;
+import static org.apache.hadoop.fs.CreateFlag.OVERWRITE;
/**
* Provides: argument processing to ensure the destination is valid
@@ -515,7 +516,8 @@ FSDataOutputStream create(PathData item, boolean lazyPersist)
defaultBlockSize = getDefaultBlockSize(item.path);
}
- EnumSet createFlags = EnumSet.of(CREATE, LAZY_PERSIST);
+ EnumSet createFlags =
+ EnumSet.of(CREATE, LAZY_PERSIST, OVERWRITE);
return create(item.path,
FsPermission.getFileDefault().applyUMask(
FsPermission.getUMask(getConf())),
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Concat.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Concat.java
new file mode 100644
index 0000000000000..f25b689e7ed17
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Concat.java
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.shell;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.LinkedList;
+
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathIOException;
+
+/**
+ * Concat the given files.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class Concat extends FsCommand {
+ public static void registerCommands(CommandFactory factory) {
+ factory.addClass(Concat.class, "-concat");
+ }
+
+ public static final String NAME = "concat";
+ public static final String USAGE = " ...";
+ public static final String DESCRIPTION = "Concatenate existing source files"
+ + " into the target file. Target file and source files should be in the"
+ + " same directory.";
+ private static FileSystem testFs; // test only.
+
+ @Override
+ protected void processArguments(LinkedList args)
+ throws IOException {
+ if (args.size() < 1) {
+ throw new IOException("Target path not specified. " + USAGE);
+ }
+ if (args.size() < 3) {
+ throw new IOException(
+ "The number of source paths is less than 2. " + USAGE);
+ }
+ PathData target = args.removeFirst();
+ LinkedList srcList = args;
+ if (!target.exists || !target.stat.isFile()) {
+ throw new FileNotFoundException(String
+ .format("Target path %s does not exist or is" + " not file.",
+ target.path));
+ }
+ Path[] srcArray = new Path[srcList.size()];
+ for (int i = 0; i < args.size(); i++) {
+ PathData src = srcList.get(i);
+ if (!src.exists || !src.stat.isFile()) {
+ throw new FileNotFoundException(
+ String.format("%s does not exist or is not file.", src.path));
+ }
+ srcArray[i] = src.path;
+ }
+ FileSystem fs = target.fs;
+ if (testFs != null) {
+ fs = testFs;
+ }
+ try {
+ fs.concat(target.path, srcArray);
+ } catch (UnsupportedOperationException exception) {
+ throw new PathIOException("Dest filesystem '" + fs.getUri().getScheme()
+ + "' doesn't support concat.", exception);
+ }
+ }
+
+ @VisibleForTesting
+ static void setTestFs(FileSystem fs) {
+ testFs = fs;
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java
index 4622c75fbd410..77f63170593ab 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java
@@ -30,7 +30,7 @@
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.TimeUnit;
-import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.FSDataInputStream;
@@ -239,26 +239,35 @@ protected void processOptions(LinkedList args)
* Copy local files to a remote filesystem
*/
public static class Put extends CommandWithDestination {
+ private ThreadPoolExecutor executor = null;
+ private int numThreads = 1;
+
+ private static final int MAX_THREADS =
+ Runtime.getRuntime().availableProcessors() * 2;
+
public static final String NAME = "put";
public static final String USAGE =
- "[-f] [-p] [-l] [-d] ... ";
+ "[-f] [-p] [-l] [-d] [-t ] ... ";
public static final String DESCRIPTION =
- "Copy files from the local file system " +
- "into fs. Copying fails if the file already " +
- "exists, unless the -f flag is given.\n" +
- "Flags:\n" +
- " -p : Preserves access and modification times, ownership and the mode.\n" +
- " -f : Overwrites the destination if it already exists.\n" +
- " -l : Allow DataNode to lazily persist the file to disk. Forces\n" +
- " replication factor of 1. This flag will result in reduced\n" +
- " durability. Use with care.\n" +
+ "Copy files from the local file system " +
+ "into fs. Copying fails if the file already " +
+ "exists, unless the -f flag is given.\n" +
+ "Flags:\n" +
+ " -p : Preserves timestamps, ownership and the mode.\n" +
+ " -f : Overwrites the destination if it already exists.\n" +
+ " -t : Number of threads to be used, default is 1.\n" +
+ " -l : Allow DataNode to lazily persist the file to disk. Forces" +
+ " replication factor of 1. This flag will result in reduced" +
+ " durability. Use with care.\n" +
" -d : Skip creation of temporary file(._COPYING_).\n";
@Override
protected void processOptions(LinkedList args) throws IOException {
CommandFormat cf =
new CommandFormat(1, Integer.MAX_VALUE, "f", "p", "l", "d");
+ cf.addOptionWithValue("t");
cf.parse(args);
+ setNumberThreads(cf.getOptValue("t"));
setOverwrite(cf.getOpt("f"));
setPreserve(cf.getOpt("p"));
setLazyPersist(cf.getOpt("l"));
@@ -288,32 +297,22 @@ protected void processArguments(LinkedList args)
copyStreamToTarget(System.in, getTargetPath(args.get(0)));
return;
}
- super.processArguments(args);
- }
- }
- public static class CopyFromLocal extends Put {
- private ThreadPoolExecutor executor = null;
- private int numThreads = 1;
+ executor = new ThreadPoolExecutor(numThreads, numThreads, 1,
+ TimeUnit.SECONDS, new ArrayBlockingQueue<>(1024),
+ new ThreadPoolExecutor.CallerRunsPolicy());
+ super.processArguments(args);
- private static final int MAX_THREADS =
- Runtime.getRuntime().availableProcessors() * 2;
- public static final String NAME = "copyFromLocal";
- public static final String USAGE =
- "[-f] [-p] [-l] [-d] [-t ] ... ";
- public static final String DESCRIPTION =
- "Copy files from the local file system " +
- "into fs. Copying fails if the file already " +
- "exists, unless the -f flag is given.\n" +
- "Flags:\n" +
- " -p : Preserves access and modification times, ownership and the" +
- " mode.\n" +
- " -f : Overwrites the destination if it already exists.\n" +
- " -t : Number of threads to be used, default is 1.\n" +
- " -l : Allow DataNode to lazily persist the file to disk. Forces" +
- " replication factor of 1. This flag will result in reduced" +
- " durability. Use with care.\n" +
- " -d : Skip creation of temporary file(._COPYING_).\n";
+ // issue the command and then wait for it to finish
+ executor.shutdown();
+ try {
+ executor.awaitTermination(Long.MAX_VALUE, TimeUnit.MINUTES);
+ } catch (InterruptedException e) {
+ executor.shutdownNow();
+ displayError(e);
+ Thread.currentThread().interrupt();
+ }
+ }
private void setNumberThreads(String numberThreadsString) {
if (numberThreadsString == null) {
@@ -330,22 +329,6 @@ private void setNumberThreads(String numberThreadsString) {
}
}
- @Override
- protected void processOptions(LinkedList args) throws IOException {
- CommandFormat cf =
- new CommandFormat(1, Integer.MAX_VALUE, "f", "p", "l", "d");
- cf.addOptionWithValue("t");
- cf.parse(args);
- setNumberThreads(cf.getOptValue("t"));
- setOverwrite(cf.getOpt("f"));
- setPreserve(cf.getOpt("p"));
- setLazyPersist(cf.getOpt("l"));
- setDirectWrite(cf.getOpt("d"));
- getRemoteDestination(args);
- // should have a -r option
- setRecursive(true);
- }
-
private void copyFile(PathData src, PathData target) throws IOException {
if (isPathRecursable(src)) {
throw new PathIsDirectoryException(src.toString());
@@ -372,25 +355,6 @@ protected void copyFileToTarget(PathData src, PathData target)
executor.submit(task);
}
- @Override
- protected void processArguments(LinkedList args)
- throws IOException {
- executor = new ThreadPoolExecutor(numThreads, numThreads, 1,
- TimeUnit.SECONDS, new ArrayBlockingQueue<>(1024),
- new ThreadPoolExecutor.CallerRunsPolicy());
- super.processArguments(args);
-
- // issue the command and then wait for it to finish
- executor.shutdown();
- try {
- executor.awaitTermination(Long.MAX_VALUE, TimeUnit.MINUTES);
- } catch (InterruptedException e) {
- executor.shutdownNow();
- displayError(e);
- Thread.currentThread().interrupt();
- }
- }
-
@VisibleForTesting
public int getNumThreads() {
return numThreads;
@@ -401,6 +365,12 @@ public ThreadPoolExecutor getExecutor() {
return executor;
}
}
+
+ public static class CopyFromLocal extends Put {
+ public static final String NAME = "copyFromLocal";
+ public static final String USAGE = Put.USAGE;
+ public static final String DESCRIPTION = "Identical to the -put command.";
+ }
public static class CopyToLocal extends Get {
public static final String NAME = "copyToLocal";
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsCommand.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsCommand.java
index 784bbf33f7826..9cafbb0f151a9 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsCommand.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsCommand.java
@@ -70,6 +70,7 @@ public static void registerCommands(CommandFactory factory) {
factory.registerCommands(Truncate.class);
factory.registerCommands(SnapshotCommands.class);
factory.registerCommands(XAttrCommands.class);
+ factory.registerCommands(Concat.class);
}
protected FsCommand() {}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsUsage.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsUsage.java
index 6596527738058..64aade3df9539 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsUsage.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsUsage.java
@@ -128,7 +128,8 @@ private void addToUsagesTable(URI uri, FsStatus fsStatus,
@Override
protected void processPath(PathData item) throws IOException {
- if (ViewFileSystemUtil.isViewFileSystem(item.fs)) {
+ if (ViewFileSystemUtil.isViewFileSystem(item.fs)
+ || ViewFileSystemUtil.isViewFileSystemOverloadScheme(item.fs)) {
ViewFileSystem viewFileSystem = (ViewFileSystem) item.fs;
Map fsStatusMap =
ViewFileSystemUtil.getStatus(viewFileSystem, item.path);
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Ls.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Ls.java
index efc541ccf81ee..b50eb69a26d70 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Ls.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Ls.java
@@ -25,7 +25,7 @@
import java.util.Date;
import java.util.LinkedList;
-import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.util.StringUtils;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/MoveCommands.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/MoveCommands.java
index 5ef42775ea58b..c20293e1a5adb 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/MoveCommands.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/MoveCommands.java
@@ -25,7 +25,7 @@
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.PathIOException;
import org.apache.hadoop.fs.PathExistsException;
-import org.apache.hadoop.fs.shell.CopyCommands.Put;
+import org.apache.hadoop.fs.shell.CopyCommands.CopyFromLocal;
/** Various commands for moving files */
@InterfaceAudience.Private
@@ -41,12 +41,22 @@ public static void registerCommands(CommandFactory factory) {
/**
* Move local files to a remote filesystem
*/
- public static class MoveFromLocal extends Put {
+ public static class MoveFromLocal extends CopyFromLocal {
public static final String NAME = "moveFromLocal";
- public static final String USAGE = " ... ";
+ public static final String USAGE =
+ "[-f] [-p] [-l] [-d] ... ";
public static final String DESCRIPTION =
- "Same as -put, except that the source is " +
- "deleted after it's copied.";
+ "Same as -put, except that the source is " +
+ "deleted after it's copied\n" +
+ "and -t option has not yet implemented.";
+
+ @Override
+ protected void processOptions(LinkedList args) throws IOException {
+ if(args.contains("-t")) {
+ throw new CommandFormat.UnknownOptionException("-t");
+ }
+ super.processOptions(args);
+ }
@Override
protected void processPath(PathData src, PathData target) throws IOException {
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/SnapshotCommands.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/SnapshotCommands.java
index 4bd596a40d678..75dc86ec87c18 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/SnapshotCommands.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/SnapshotCommands.java
@@ -26,7 +26,7 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathIsNotDirectoryException;
-import com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
/**
* Snapshot related operations
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Tail.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Tail.java
index 8a75a60f435ca..22dd32bce8512 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Tail.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Tail.java
@@ -28,7 +28,7 @@
import org.apache.hadoop.fs.PathIsDirectoryException;
import org.apache.hadoop.io.IOUtils;
-import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* Get a listing of all files in that match the file patterns.
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/TouchCommands.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/TouchCommands.java
index be174b5e9cf68..872de306d287a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/TouchCommands.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/TouchCommands.java
@@ -31,7 +31,7 @@
import org.apache.hadoop.fs.PathNotFoundException;
import org.apache.hadoop.util.StringUtils;
-import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* Unix touch like commands
@@ -102,8 +102,8 @@ public static class Touch extends TouchCommands {
public static final String NAME = "touch";
public static final String USAGE = "[-" + OPTION_CHANGE_ONLY_ACCESS_TIME
+ "] [-" + OPTION_CHANGE_ONLY_MODIFICATION_TIME + "] [-"
- + OPTION_USE_TIMESTAMP + " TIMESTAMP ] [-" + OPTION_DO_NOT_CREATE_FILE
- + "] ...";
+ + OPTION_USE_TIMESTAMP + " TIMESTAMP (yyyyMMdd:HHmmss) ] "
+ + "[-" + OPTION_DO_NOT_CREATE_FILE + "] ...";
public static final String DESCRIPTION =
"Updates the access and modification times of the file specified by the"
+ " to the current time. If the file does not exist, then a zero"
@@ -114,7 +114,8 @@ public static class Touch extends TouchCommands {
+ OPTION_CHANGE_ONLY_MODIFICATION_TIME
+ " Change only the modification time \n" + "-"
+ OPTION_USE_TIMESTAMP + " TIMESTAMP"
- + " Use specified timestamp (in format yyyyMMddHHmmss) instead of current time \n"
+ + " Use specified timestamp instead of current time\n"
+ + " TIMESTAMP format yyyyMMdd:HHmmss\n"
+ "-" + OPTION_DO_NOT_CREATE_FILE + " Do not create any files";
private boolean changeModTime = false;
@@ -137,7 +138,7 @@ protected void processOptions(LinkedList args) {
CommandFormat cf = new CommandFormat(1, Integer.MAX_VALUE,
OPTION_USE_TIMESTAMP, OPTION_CHANGE_ONLY_ACCESS_TIME,
- OPTION_CHANGE_ONLY_MODIFICATION_TIME);
+ OPTION_CHANGE_ONLY_MODIFICATION_TIME, OPTION_DO_NOT_CREATE_FILE);
cf.parse(args);
this.changeModTime = cf.getOpt(OPTION_CHANGE_ONLY_MODIFICATION_TIME);
this.changeAccessTime = cf.getOpt(OPTION_CHANGE_ONLY_ACCESS_TIME);
@@ -183,7 +184,8 @@ private void updateTime(PathData item) throws IOException {
time = dateFormat.parse(timestamp).getTime();
} catch (ParseException e) {
throw new IllegalArgumentException(
- "Unable to parse the specified timestamp " + timestamp, e);
+ "Unable to parse the specified timestamp "+ timestamp
+ + ". The expected format is " + dateFormat.toPattern(), e);
}
}
if (changeModTime ^ changeAccessTime) {
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/XAttrCommands.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/XAttrCommands.java
index 630177611940e..2fe7c858e4e66 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/XAttrCommands.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/XAttrCommands.java
@@ -23,7 +23,7 @@
import java.util.Map;
import java.util.Map.Entry;
-import com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.classification.InterfaceAudience;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/BufferedIOStatisticsInputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/BufferedIOStatisticsInputStream.java
new file mode 100644
index 0000000000000..bdc432570542b
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/BufferedIOStatisticsInputStream.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.statistics;
+
+import java.io.BufferedInputStream;
+import java.io.InputStream;
+
+import org.apache.hadoop.fs.StreamCapabilities;
+
+import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics;
+
+/**
+ * An extension of {@code BufferedInputStream} which implements
+ * {@link IOStatisticsSource} and forwards requests for the
+ * {@link IOStatistics} to the wrapped stream.
+ *
+ * This should be used when any input stream needs buffering while
+ * allowing the inner stream to be a source of statistics.
+ *
+ * It also implements {@link StreamCapabilities} and forwards the probe
+ * to the inner stream, if possible.
+ */
+public class BufferedIOStatisticsInputStream
+ extends BufferedInputStream
+ implements IOStatisticsSource, StreamCapabilities {
+
+ /**
+ * Buffer an input stream with the default buffer size of 8k.
+ * @param in input stream
+ */
+ public BufferedIOStatisticsInputStream(final InputStream in) {
+ super(in);
+ }
+
+ /**
+ * Buffer an input stream with the chosen buffer size.
+ * @param in input stream
+ * @param size buffer size
+ */
+ public BufferedIOStatisticsInputStream(final InputStream in, final int size) {
+ super(in, size);
+ }
+
+ /**
+ * Return any IOStatistics offered by the inner stream.
+ * @return inner IOStatistics or null
+ */
+ @Override
+ public IOStatistics getIOStatistics() {
+ return retrieveIOStatistics(in);
+ }
+
+ /**
+ * If the inner stream supports {@link StreamCapabilities},
+ * forward the probe to it.
+ * Otherwise: return false.
+ *
+ * @param capability string to query the stream support for.
+ * @return true if a capability is known to be supported.
+ */
+ @Override
+ public boolean hasCapability(final String capability) {
+ if (in instanceof StreamCapabilities) {
+ return ((StreamCapabilities) in).hasCapability(capability);
+ } else {
+ return false;
+ }
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/BufferedIOStatisticsOutputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/BufferedIOStatisticsOutputStream.java
new file mode 100644
index 0000000000000..88e73a0629b1d
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/BufferedIOStatisticsOutputStream.java
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.statistics;
+
+import java.io.BufferedOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+
+import org.apache.hadoop.fs.StreamCapabilities;
+import org.apache.hadoop.fs.Syncable;
+
+import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics;
+
+/**
+ * An extension of {@code BufferedOutputStream} which implements
+ * {@link IOStatisticsSource} and forwards requests for the
+ * {@link IOStatistics} to the wrapped stream.
+ *
+ * This should be used when any output stream needs buffering while
+ * allowing the inner stream to be a source of statistics.
+ *
+ * It also implements {@link StreamCapabilities}
+ * and {@link Syncable} and forwards to to the inner stream,
+ * if possible.
+ */
+public class BufferedIOStatisticsOutputStream
+ extends BufferedOutputStream
+ implements IOStatisticsSource, Syncable, StreamCapabilities {
+
+ /**
+ * Should calls to Syncable downgrade to flush if the underlying
+ * stream does not support it?
+ * While that breaks a core contract requirement of Syncable:
+ * "Sync.sync() guarantees durability", downgrading is
+ * the default behavior of FsDataOutputStream.
+ */
+ private final boolean downgradeSyncable;
+
+ /**
+ * Construct with default buffer size.
+ * @param out output stream to buffer
+ * @param downgradeSyncable should Syncable calls downgrade?
+ */
+ public BufferedIOStatisticsOutputStream(
+ final OutputStream out,
+ final boolean downgradeSyncable) {
+ super(out);
+ this.downgradeSyncable = downgradeSyncable;
+ }
+
+ /**
+ * Construct with custom buffer size.
+ *
+ * @param out output stream to buffer
+ * @param size buffer.
+ * @param downgradeSyncable should Syncable calls downgrade?
+ */
+ public BufferedIOStatisticsOutputStream(
+ final OutputStream out,
+ final int size,
+ final boolean downgradeSyncable) {
+ super(out, size);
+ this.downgradeSyncable = downgradeSyncable;
+ }
+
+ /**
+ * Ask the inner stream for their IOStatistics.
+ * @return any IOStatistics offered by the inner stream.
+ */
+ @Override
+ public IOStatistics getIOStatistics() {
+ return retrieveIOStatistics(out);
+ }
+
+ /**
+ * If the inner stream supports {@link StreamCapabilities},
+ * forward the probe to it.
+ * Otherwise: return false.
+ *
+ * @param capability string to query the stream support for.
+ * @return true if a capability is known to be supported.
+ */
+ @Override
+ public boolean hasCapability(final String capability) {
+ if (out instanceof StreamCapabilities) {
+ return ((StreamCapabilities) out).hasCapability(capability);
+ } else {
+ return false;
+ }
+ }
+
+ /**
+ * If the inner stream is Syncable, flush the buffer and then
+ * invoke the inner stream's hflush() operation.
+ *
+ * Otherwise: throw an exception, unless the stream was constructed with
+ * {@link #downgradeSyncable} set to true, in which case the stream
+ * is just flushed.
+ * @throws IOException IO Problem
+ * @throws UnsupportedOperationException if the inner class is not syncable
+ */
+ @Override
+ public void hflush() throws IOException {
+ if (out instanceof Syncable) {
+ flush();
+ ((Syncable) out).hflush();
+ } else {
+ if (!downgradeSyncable) {
+ throw new UnsupportedOperationException("hflush not supported by "
+ + out);
+ } else {
+ flush();
+ }
+ }
+ }
+
+ /**
+ * If the inner stream is Syncable, flush the buffer and then
+ * invoke the inner stream's hsync() operation.
+ *
+ * Otherwise: throw an exception, unless the stream was constructed with
+ * {@link #downgradeSyncable} set to true, in which case the stream
+ * is just flushed.
+ * @throws IOException IO Problem
+ * @throws UnsupportedOperationException if the inner class is not syncable
+ */
+ @Override
+ public void hsync() throws IOException {
+ if (out instanceof Syncable) {
+ flush();
+ ((Syncable) out).hsync();
+ } else {
+ if (!downgradeSyncable) {
+ throw new UnsupportedOperationException("hsync not supported by "
+ + out);
+ } else {
+ flush();
+ }
+ }
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/DurationStatisticSummary.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/DurationStatisticSummary.java
new file mode 100644
index 0000000000000..e1335d77d792a
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/DurationStatisticSummary.java
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.statistics;
+
+import javax.annotation.Nullable;
+import java.io.Serializable;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_FAILURES;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MAX;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MEAN;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MIN;
+
+/**
+ * Summary of duration tracking statistics
+ * as extracted from an IOStatistics instance.
+ *
+ * This is for reporting and testing.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public final class DurationStatisticSummary implements Serializable {
+
+ private static final long serialVersionUID = 6776381340896518486L;
+
+ /** Statistic key. */
+ private final String key;
+
+ /** Are these success or failure statistics. */
+ private final boolean success;
+
+ /** Count of operation invocations. */
+ private final long count;
+
+ /** Max duration; -1 if unknown. */
+ private final long max;
+
+ /** Min duration; -1 if unknown. */
+ private final long min;
+
+ /** Mean duration -may be null. */
+ private final MeanStatistic mean;
+
+ /**
+ * Constructor.
+ * @param key Statistic key.
+ * @param success Are these success or failure statistics.
+ * @param count Count of operation invocations.
+ * @param max Max duration; -1 if unknown.
+ * @param min Min duration; -1 if unknown.
+ * @param mean Mean duration -may be null. (will be cloned)
+ */
+ public DurationStatisticSummary(final String key,
+ final boolean success,
+ final long count,
+ final long max,
+ final long min,
+ @Nullable final MeanStatistic mean) {
+ this.key = key;
+ this.success = success;
+ this.count = count;
+ this.max = max;
+ this.min = min;
+ this.mean = mean == null ? null : mean.clone();
+ }
+
+ public String getKey() {
+ return key;
+ }
+
+ public boolean isSuccess() {
+ return success;
+ }
+
+ public long getCount() {
+ return count;
+ }
+
+ public long getMax() {
+ return max;
+ }
+
+ public long getMin() {
+ return min;
+ }
+
+ public MeanStatistic getMean() {
+ return mean;
+ }
+
+ @Override
+ public String toString() {
+ return "DurationStatisticSummary{" +
+ "key='" + key + '\'' +
+ ", success=" + success +
+ ", counter=" + count +
+ ", max=" + max +
+ ", mean=" + mean +
+ '}';
+ }
+
+ /**
+ * Fetch the duration timing summary of success or failure operations
+ * from an IO Statistics source.
+ * If the duration key is unknown, the summary will be incomplete.
+ * @param source source of data
+ * @param key duration statistic key
+ * @param success fetch success statistics, or if false, failure stats.
+ * @return a summary of the statistics.
+ */
+ public static DurationStatisticSummary fetchDurationSummary(
+ IOStatistics source,
+ String key,
+ boolean success) {
+ String fullkey = success ? key : key + SUFFIX_FAILURES;
+ return new DurationStatisticSummary(key, success,
+ source.counters().getOrDefault(fullkey, 0L),
+ source.maximums().getOrDefault(fullkey + SUFFIX_MAX, -1L),
+ source.minimums().getOrDefault(fullkey + SUFFIX_MIN, -1L),
+ source.meanStatistics()
+ .get(fullkey + SUFFIX_MEAN));
+ }
+
+ /**
+ * Fetch the duration timing summary from an IOStatistics source.
+ * If the duration key is unknown, the summary will be incomplete.
+ * @param source source of data
+ * @param key duration statistic key
+ * @return a summary of the statistics.
+ */
+ public static DurationStatisticSummary fetchSuccessSummary(
+ IOStatistics source,
+ String key) {
+ return fetchDurationSummary(source, key, true);
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/DurationTracker.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/DurationTracker.java
new file mode 100644
index 0000000000000..5a15c7ad66c4f
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/DurationTracker.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.statistics;
+
+import java.time.Duration;
+
+/**
+ * Interface to be implemented by objects which can track duration.
+ * It extends AutoCloseable to fit into a try-with-resources statement,
+ * but then strips out the {@code throws Exception} aspect of the signature
+ * so it doesn't force code to add extra handling for any failures.
+ *
+ * If a duration is declared as "failed()" then the failure counters
+ * will be updated.
+ */
+public interface DurationTracker extends AutoCloseable {
+
+ /**
+ * The operation failed. Failure statistics will be updated.
+ */
+ void failed();
+
+ /**
+ * Finish tracking: update the statistics with the timings.
+ */
+ void close();
+
+ /**
+ * Get the duration of an operation as a java Duration
+ * instance. If the duration tracker hasn't completed,
+ * or its duration tracking doesn't actually measure duration,
+ * returns Duration.ZERO.
+ * @return a duration, value of ZERO until close().
+ */
+ default Duration asDuration() {
+ return Duration.ZERO;
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/DurationTrackerFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/DurationTrackerFactory.java
new file mode 100644
index 0000000000000..641d7e8368bb1
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/DurationTrackerFactory.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.statistics;
+
+import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.stubDurationTracker;
+
+/**
+ * Interface for a source of duration tracking.
+ *
+ * This is intended for uses where it can be passed into classes
+ * which update operation durations, without tying those
+ * classes to internal implementation details.
+ */
+public interface DurationTrackerFactory {
+
+ /**
+ * Initiate a duration tracking operation by creating/returning
+ * an object whose {@code close()} call will
+ * update the statistics.
+ *
+ * The statistics counter with the key name will be incremented
+ * by the given count.
+ *
+ * The expected use is within a try-with-resources clause.
+ *
+ * The default implementation returns a stub duration tracker.
+ * @param key statistic key prefix
+ * @param count #of times to increment the matching counter in this
+ * operation.
+ * @return an object to close after an operation completes.
+ */
+ default DurationTracker trackDuration(String key, long count) {
+ return stubDurationTracker();
+ }
+
+ /**
+ * Initiate a duration tracking operation by creating/returning
+ * an object whose {@code close()} call will
+ * update the statistics.
+ * The expected use is within a try-with-resources clause.
+ * @param key statistic key
+ * @return an object to close after an operation completes.
+ */
+ default DurationTracker trackDuration(String key) {
+ return trackDuration(key, 1);
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatistics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatistics.java
new file mode 100644
index 0000000000000..75d9965128101
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatistics.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.statistics;
+
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * IO Statistics.
+ *
+ * These are low-cost per-instance statistics provided by any Hadoop
+ * I/O class instance.
+ *
+ * Consult the filesystem specification document for the requirements
+ * of an implementation of this interface.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public interface IOStatistics {
+
+ /**
+ * Map of counters.
+ * @return the current map of counters.
+ */
+ Map counters();
+
+ /**
+ * Map of gauges.
+ * @return the current map of gauges.
+ */
+ Map gauges();
+
+ /**
+ * Map of minimums.
+ * @return the current map of minimums.
+ */
+ Map minimums();
+
+ /**
+ * Map of maximums.
+ * @return the current map of maximums.
+ */
+ Map maximums();
+
+ /**
+ * Map of meanStatistics.
+ * @return the current map of MeanStatistic statistics.
+ */
+ Map meanStatistics();
+
+ /**
+ * Value when a minimum value has never been set.
+ */
+ long MIN_UNSET_VALUE = -1;
+
+ /**
+ * Value when a max value has never been set.
+ */
+ long MAX_UNSET_VALUE = -1;
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsAggregator.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsAggregator.java
new file mode 100644
index 0000000000000..1c5451c6f0e83
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsAggregator.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.statistics;
+
+import javax.annotation.Nullable;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Interface exported by classes which support
+ * aggregation of {@link IOStatistics}.
+ * Implementations MAY aggregate all statistics
+ * exported by the IOStatistics reference passed in to
+ * {@link #aggregate(IOStatistics)}, or they
+ * may selectively aggregate specific values/classes
+ * of statistics.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface IOStatisticsAggregator {
+
+ /**
+ * Aggregate the supplied statistics into the current
+ * set.
+ *
+ * @param statistics statistics; may be null
+ * @return true if the statistics reference was not null and
+ * so aggregated.
+ */
+ boolean aggregate(@Nullable IOStatistics statistics);
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsLogging.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsLogging.java
new file mode 100644
index 0000000000000..c7230e25c3434
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsLogging.java
@@ -0,0 +1,301 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.statistics;
+
+import javax.annotation.Nullable;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.function.Predicate;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding;
+
+import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics;
+
+/**
+ * Utility operations convert IO Statistics sources/instances
+ * to strings, especially for robustly logging.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public final class IOStatisticsLogging {
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(IOStatisticsLogging.class);
+
+ private IOStatisticsLogging() {
+ }
+
+ /**
+ * Extract the statistics from a source object -or ""
+ * if it is not an instance of {@link IOStatistics},
+ * {@link IOStatisticsSource} or the retrieved
+ * statistics are null.
+ *
+ * Exceptions are caught and downgraded to debug logging.
+ * @param source source of statistics.
+ * @return a string for logging.
+ */
+ public static String ioStatisticsSourceToString(@Nullable Object source) {
+ try {
+ return ioStatisticsToString(retrieveIOStatistics(source));
+ } catch (RuntimeException e) {
+ LOG.debug("Ignoring", e);
+ return "";
+ }
+ }
+
+ /**
+ * Convert IOStatistics to a string form.
+ * @param statistics A statistics instance.
+ * @return string value or the empty string if null
+ */
+ public static String ioStatisticsToString(
+ @Nullable final IOStatistics statistics) {
+ if (statistics != null) {
+ StringBuilder sb = new StringBuilder();
+ mapToString(sb, "counters", statistics.counters(), " ");
+ mapToString(sb, "gauges", statistics.gauges(), " ");
+ mapToString(sb, "minimums", statistics.minimums(), " ");
+ mapToString(sb, "maximums", statistics.maximums(), " ");
+ mapToString(sb, "means", statistics.meanStatistics(), " ");
+
+ return sb.toString();
+ } else {
+ return "";
+ }
+ }
+
+ /**
+ * Convert IOStatistics to a string form, with all the metrics sorted
+ * and empty value stripped.
+ * This is more expensive than the simple conversion, so should only
+ * be used for logging/output where it's known/highly likely that the
+ * caller wants to see the values. Not for debug logging.
+ * @param statistics A statistics instance.
+ * @return string value or the empty string if null
+ */
+ public static String ioStatisticsToPrettyString(
+ @Nullable final IOStatistics statistics) {
+ if (statistics != null) {
+ StringBuilder sb = new StringBuilder();
+ mapToSortedString(sb, "counters", statistics.counters(),
+ p -> p == 0);
+ mapToSortedString(sb, "\ngauges", statistics.gauges(),
+ p -> p == 0);
+ mapToSortedString(sb, "\nminimums", statistics.minimums(),
+ p -> p < 0);
+ mapToSortedString(sb, "\nmaximums", statistics.maximums(),
+ p -> p < 0);
+ mapToSortedString(sb, "\nmeans", statistics.meanStatistics(),
+ MeanStatistic::isEmpty);
+
+ return sb.toString();
+ } else {
+ return "";
+ }
+ }
+
+ /**
+ * Given a map, add its entryset to the string.
+ * The entries are only sorted if the source entryset
+ * iterator is sorted, such as from a TreeMap.
+ * @param sb string buffer to append to
+ * @param type type (for output)
+ * @param map map to evaluate
+ * @param separator separator
+ * @param type of values of the map
+ */
+ private static void mapToString(StringBuilder sb,
+ final String type,
+ final Map map,
+ final String separator) {
+ int count = 0;
+ sb.append(type);
+ sb.append("=(");
+ for (Map.Entry entry : map.entrySet()) {
+ if (count > 0) {
+ sb.append(separator);
+ }
+ count++;
+ sb.append(IOStatisticsBinding.entryToString(
+ entry.getKey(), entry.getValue()));
+ }
+ sb.append(");\n");
+ }
+
+ /**
+ * Given a map, produce a string with all the values, sorted.
+ * Needs to create a treemap and insert all the entries.
+ * @param sb string buffer to append to
+ * @param type type (for output)
+ * @param map map to evaluate
+ * @param type of values of the map
+ */
+ private static void mapToSortedString(StringBuilder sb,
+ final String type,
+ final Map map,
+ final Predicate isEmpty) {
+ mapToString(sb, type, sortedMap(map, isEmpty), "\n");
+ }
+
+ /**
+ * Create a sorted (tree) map from an unsorted map.
+ * This incurs the cost of creating a map and that
+ * of inserting every object into the tree.
+ * @param source source map
+ * @param value type
+ * @return a treemap with all the entries.
+ */
+ private static Map sortedMap(
+ final Map source,
+ final Predicate isEmpty) {
+ Map tm = new TreeMap<>();
+ for (Map.Entry entry : source.entrySet()) {
+ if (!isEmpty.test(entry.getValue())) {
+ tm.put(entry.getKey(), entry.getValue());
+ }
+ }
+ return tm;
+ }
+
+ /**
+ * On demand stringifier of an IOStatisticsSource instance.
+ *
+ * Whenever this object's toString() method is called, it evaluates the
+ * statistics.
+ *
+ * This is designed to affordable to use in log statements.
+ * @param source source of statistics -may be null.
+ * @return an object whose toString() operation returns the current values.
+ */
+ public static Object demandStringifyIOStatisticsSource(
+ @Nullable IOStatisticsSource source) {
+ return new SourceToString(source);
+ }
+
+ /**
+ * On demand stringifier of an IOStatistics instance.
+ *
+ * Whenever this object's toString() method is called, it evaluates the
+ * statistics.
+ *
+ * This is for use in log statements where for the cost of creation
+ * of this entry is low; it is affordable to use in log statements.
+ * @param statistics statistics to stringify -may be null.
+ * @return an object whose toString() operation returns the current values.
+ */
+ public static Object demandStringifyIOStatistics(
+ @Nullable IOStatistics statistics) {
+ return new StatisticsToString(statistics);
+ }
+
+ /**
+ * Extract any statistics from the source and log at debug, if
+ * the log is set to log at debug.
+ * No-op if logging is not at debug or the source is null/of
+ * the wrong type/doesn't provide statistics.
+ * @param log log to log to
+ * @param message message for log -this must contain "{}" for the
+ * statistics report to actually get logged.
+ * @param source source object
+ */
+ public static void logIOStatisticsAtDebug(
+ Logger log,
+ String message,
+ Object source) {
+ if (log.isDebugEnabled()) {
+ // robust extract and convert to string
+ String stats = ioStatisticsSourceToString(source);
+ if (!stats.isEmpty()) {
+ log.debug(message, stats);
+ }
+ }
+ }
+
+ /**
+ * Extract any statistics from the source and log to
+ * this class's log at debug, if
+ * the log is set to log at debug.
+ * No-op if logging is not at debug or the source is null/of
+ * the wrong type/doesn't provide statistics.
+ * @param message message for log -this must contain "{}" for the
+ * statistics report to actually get logged.
+ * @param source source object
+ */
+ public static void logIOStatisticsAtDebug(
+ String message,
+ Object source) {
+ logIOStatisticsAtDebug(LOG, message, source);
+ }
+
+ /**
+ * On demand stringifier.
+ *
+ * Whenever this object's toString() method is called, it
+ * retrieves the latest statistics instance and re-evaluates it.
+ */
+ private static final class SourceToString {
+
+ private final IOStatisticsSource source;
+
+ private SourceToString(@Nullable IOStatisticsSource source) {
+ this.source = source;
+ }
+
+ @Override
+ public String toString() {
+ return source != null
+ ? ioStatisticsSourceToString(source)
+ : IOStatisticsBinding.NULL_SOURCE;
+ }
+ }
+
+ /**
+ * Stringifier of statistics: low cost to instantiate and every
+ * toString/logging will re-evaluate the statistics.
+ */
+ private static final class StatisticsToString {
+
+ private final IOStatistics statistics;
+
+ /**
+ * Constructor.
+ * @param statistics statistics
+ */
+ private StatisticsToString(@Nullable IOStatistics statistics) {
+ this.statistics = statistics;
+ }
+
+ /**
+ * Evaluate and stringify the statistics.
+ * @return a string value.
+ */
+ @Override
+ public String toString() {
+ return statistics != null
+ ? ioStatisticsToString(statistics)
+ : IOStatisticsBinding.NULL_SOURCE;
+ }
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSnapshot.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSnapshot.java
new file mode 100644
index 0000000000000..5b8b2e284cc11
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSnapshot.java
@@ -0,0 +1,285 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.statistics;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Collectors;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding;
+import org.apache.hadoop.util.JsonSerialization;
+
+import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToString;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.aggregateMaps;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.snapshotMap;
+
+/**
+ * Snapshot of statistics from a different source.
+ *
+ * It is serializable so that frameworks which can use java serialization
+ * to propagate data (Spark, Flink...) can send the statistics
+ * back. For this reason, TreeMaps are explicitly used as field types,
+ * even though IDEs can recommend use of Map instead.
+ * For security reasons, untrusted java object streams should never be
+ * deserialized. If for some reason this is required, use
+ * {@link #requiredSerializationClasses()} to get the list of classes
+ * used when deserializing instances of this object.
+ *
+ *
+ * It is annotated for correct serializations with jackson2.
+ *
+ */
+@SuppressWarnings("CollectionDeclaredAsConcreteClass")
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class IOStatisticsSnapshot
+ implements IOStatistics, Serializable, IOStatisticsAggregator {
+
+ private static final long serialVersionUID = -1762522703841538084L;
+
+ /**
+ * List of chasses needed to deserialize.
+ */
+ private static final Class[] DESERIALIZATION_CLASSES = {
+ IOStatisticsSnapshot.class,
+ TreeMap.class,
+ Long.class,
+ MeanStatistic.class,
+ };
+
+ /**
+ * Counters.
+ */
+ @JsonProperty
+ private transient Map counters;
+
+ /**
+ * Gauges.
+ */
+ @JsonProperty
+ private transient Map gauges;
+
+ /**
+ * Minimum values.
+ */
+ @JsonProperty
+ private transient Map minimums;
+
+ /**
+ * Maximum values.
+ */
+ @JsonProperty
+ private transient Map maximums;
+
+ /**
+ * mean statistics. The JSON key is all lower case..
+ */
+ @JsonProperty("meanstatistics")
+ private transient Map meanStatistics;
+
+ /**
+ * Construct.
+ */
+ public IOStatisticsSnapshot() {
+ createMaps();
+ }
+
+ /**
+ * Construct, taking a snapshot of the source statistics data
+ * if the source is non-null.
+ * If the source is null, the empty maps are created
+ * @param source statistics source. Nullable.
+ */
+ public IOStatisticsSnapshot(IOStatistics source) {
+ if (source != null) {
+ snapshot(source);
+ } else {
+ createMaps();
+ }
+ }
+
+ /**
+ * Create the maps.
+ */
+ private synchronized void createMaps() {
+ counters = new ConcurrentHashMap<>();
+ gauges = new ConcurrentHashMap<>();
+ minimums = new ConcurrentHashMap<>();
+ maximums = new ConcurrentHashMap<>();
+ meanStatistics = new ConcurrentHashMap<>();
+ }
+
+ /**
+ * Clear all the maps.
+ */
+ public synchronized void clear() {
+ counters.clear();
+ gauges.clear();
+ minimums.clear();
+ maximums.clear();
+ meanStatistics.clear();
+ }
+
+ /**
+ * Take a snapshot.
+ *
+ * This completely overwrites the map data with the statistics
+ * from the source.
+ * @param source statistics source.
+ */
+ public synchronized void snapshot(IOStatistics source) {
+ checkNotNull(source);
+ counters = snapshotMap(source.counters());
+ gauges = snapshotMap(source.gauges());
+ minimums = snapshotMap(source.minimums());
+ maximums = snapshotMap(source.maximums());
+ meanStatistics = snapshotMap(source.meanStatistics(),
+ MeanStatistic::copy);
+ }
+
+ /**
+ * Aggregate the current statistics with the
+ * source reference passed in.
+ *
+ * The operation is synchronized.
+ * @param source source; may be null
+ * @return true if a merge took place.
+ */
+ @Override
+ public synchronized boolean aggregate(
+ @Nullable IOStatistics source) {
+ if (source == null) {
+ return false;
+ }
+ aggregateMaps(counters, source.counters(),
+ IOStatisticsBinding::aggregateCounters,
+ IOStatisticsBinding::passthroughFn);
+ aggregateMaps(gauges, source.gauges(),
+ IOStatisticsBinding::aggregateGauges,
+ IOStatisticsBinding::passthroughFn);
+ aggregateMaps(minimums, source.minimums(),
+ IOStatisticsBinding::aggregateMinimums,
+ IOStatisticsBinding::passthroughFn);
+ aggregateMaps(maximums, source.maximums(),
+ IOStatisticsBinding::aggregateMaximums,
+ IOStatisticsBinding::passthroughFn);
+ aggregateMaps(meanStatistics, source.meanStatistics(),
+ IOStatisticsBinding::aggregateMeanStatistics, MeanStatistic::copy);
+ return true;
+ }
+
+ @Override
+ public synchronized Map counters() {
+ return counters;
+ }
+
+ @Override
+ public synchronized Map gauges() {
+ return gauges;
+ }
+
+ @Override
+ public synchronized Map minimums() {
+ return minimums;
+ }
+
+ @Override
+ public synchronized Map maximums() {
+ return maximums;
+ }
+
+ @Override
+ public synchronized Map meanStatistics() {
+ return meanStatistics;
+ }
+
+ @Override
+ public String toString() {
+ return ioStatisticsToString(this);
+ }
+
+ /**
+ * Get a JSON serializer for this class.
+ * @return a serializer.
+ */
+ public static JsonSerialization serializer() {
+ return new JsonSerialization<>(IOStatisticsSnapshot.class, false, true);
+ }
+
+ /**
+ * Serialize by converting each map to a TreeMap, and saving that
+ * to the stream.
+ */
+ private synchronized void writeObject(ObjectOutputStream s)
+ throws IOException {
+ // Write out the core
+ s.defaultWriteObject();
+ s.writeObject(new TreeMap(counters));
+ s.writeObject(new TreeMap(gauges));
+ s.writeObject(new TreeMap(minimums));
+ s.writeObject(new TreeMap(maximums));
+ s.writeObject(new TreeMap(meanStatistics));
+ }
+
+ /**
+ * Deserialize by loading each TreeMap, and building concurrent
+ * hash maps from them.
+ */
+ private void readObject(final ObjectInputStream s)
+ throws IOException, ClassNotFoundException {
+ // read in core
+ s.defaultReadObject();
+ // and rebuild a concurrent hashmap from every serialized tree map
+ // read back from the stream.
+ counters = new ConcurrentHashMap<>(
+ (TreeMap) s.readObject());
+ gauges = new ConcurrentHashMap<>(
+ (TreeMap) s.readObject());
+ minimums = new ConcurrentHashMap<>(
+ (TreeMap) s.readObject());
+ maximums = new ConcurrentHashMap<>(
+ (TreeMap) s.readObject());
+ meanStatistics = new ConcurrentHashMap<>(
+ (TreeMap) s.readObject());
+ }
+
+ /**
+ * What classes are needed to deserialize this class?
+ * Needed to securely unmarshall this from untrusted sources.
+ * @return a list of required classes to deserialize the data.
+ */
+ public static List requiredSerializationClasses() {
+ return Arrays.stream(DESERIALIZATION_CLASSES)
+ .collect(Collectors.toList());
+ }
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSource.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSource.java
new file mode 100644
index 0000000000000..67bf51fc0c3ae
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSource.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.statistics;
+
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A source of IO statistics.
+ *
+ * These statistics MUST be instance specific, not thread local.
+ *
+ */
+
+@InterfaceStability.Unstable
+public interface IOStatisticsSource {
+
+ /**
+ * Return a statistics instance.
+ *
+ * It is not a requirement that the same instance is returned every time.
+ * {@link IOStatisticsSource}.
+ *
+ * If the object implementing this is Closeable, this method
+ * may return null if invoked on a closed object, even if
+ * it returns a valid instance when called earlier.
+ * @return an IOStatistics instance or null
+ */
+ default IOStatistics getIOStatistics() {
+ return null;
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSupport.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSupport.java
new file mode 100644
index 0000000000000..75977047c0f2a
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSupport.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.statistics;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.statistics.impl.StubDurationTracker;
+import org.apache.hadoop.fs.statistics.impl.StubDurationTrackerFactory;
+
+/**
+ * Support for working with IOStatistics.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public final class IOStatisticsSupport {
+
+ private IOStatisticsSupport() {
+ }
+
+ /**
+ * Take a snapshot of the current statistics state.
+ *
+ * This is not an atomic option.
+ *
+ * The instance can be serialized, and its
+ * {@code toString()} method lists all the values.
+ * @param statistics statistics
+ * @return a snapshot of the current values.
+ */
+ public static IOStatisticsSnapshot
+ snapshotIOStatistics(IOStatistics statistics) {
+
+ return new IOStatisticsSnapshot(statistics);
+ }
+
+ /**
+ * Create a snapshot statistics instance ready to aggregate data.
+ *
+ * The instance can be serialized, and its
+ * {@code toString()} method lists all the values.
+ * @return an empty snapshot
+ */
+ public static IOStatisticsSnapshot
+ snapshotIOStatistics() {
+
+ return new IOStatisticsSnapshot();
+ }
+
+ /**
+ * Get the IOStatistics of the source, casting it
+ * if it is of the relevant type, otherwise,
+ * if it implements {@link IOStatisticsSource}
+ * extracting the value.
+ *
+ * Returns null if the source isn't of the write type
+ * or the return value of
+ * {@link IOStatisticsSource#getIOStatistics()} was null.
+ * @return an IOStatistics instance or null
+ */
+
+ public static IOStatistics retrieveIOStatistics(
+ final Object source) {
+ if (source instanceof IOStatistics) {
+ return (IOStatistics) source;
+ } else if (source instanceof IOStatisticsSource) {
+ return ((IOStatisticsSource) source).getIOStatistics();
+ } else {
+ // null source or interface not implemented
+ return null;
+ }
+ }
+
+ /**
+ * Return a stub duration tracker factory whose returned trackers
+ * are always no-ops.
+ *
+ * As singletons are returned, this is very low-cost to use.
+ * @return a duration tracker factory.
+ */
+ public static DurationTrackerFactory stubDurationTrackerFactory() {
+ return StubDurationTrackerFactory.STUB_DURATION_TRACKER_FACTORY;
+ }
+
+ /**
+ * Get a stub duration tracker.
+ * @return a stub tracker.
+ */
+ public static DurationTracker stubDurationTracker() {
+ return StubDurationTracker.STUB_DURATION_TRACKER;
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/MeanStatistic.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/MeanStatistic.java
new file mode 100644
index 0000000000000..d9ff0c25c6a21
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/MeanStatistic.java
@@ -0,0 +1,290 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.statistics;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A mean statistic represented as the sum and the sample count;
+ * the mean is calculated on demand.
+ *
+ * It can be used to accrue values so as to dynamically update
+ * the mean. If so, know that there is no synchronization
+ * on the methods.
+ *
+ *
+ * If a statistic has 0 samples then it is considered to be empty.
+ *
+ *
+ * All 'empty' statistics are equivalent, independent of the sum value.
+ *
+ *
+ * For non-empty statistics, sum and sample values must match
+ * for equality.
+ *
+ *
+ * It is serializable and annotated for correct serializations with jackson2.
+ *
+ *
+ * Thread safety. The operations to add/copy sample data, are thread safe.
+ *
+ *
+ * - {@link #add(MeanStatistic)}
+ * - {@link #addSample(long)}
+ * - {@link #clear()}
+ * - {@link #setSamplesAndSum(long, long)}
+ * - {@link #set(MeanStatistic)}
+ * - {@link #setSamples(long)} and {@link #setSum(long)}
+ *
+ *
+ * So is the {@link #mean()} method. This ensures that when
+ * used to aggregated statistics, the aggregate value and sample
+ * count are set and evaluated consistently.
+ *
+ *
+ * Other methods marked as synchronized because Findbugs overreacts
+ * to the idea that some operations to update sum and sample count
+ * are synchronized, but that things like equals are not.
+ *
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class MeanStatistic implements Serializable, Cloneable {
+
+ private static final long serialVersionUID = 567888327998615425L;
+
+ /**
+ * Number of samples used to calculate
+ * the mean.
+ */
+ private long samples;
+
+ /**
+ * sum of the values.
+ */
+ private long sum;
+
+ /**
+ * Constructor, with some resilience against invalid sample counts.
+ * If the sample count is 0 or less, the sum is set to 0 and
+ * the sample count to 0.
+ * @param samples sample count.
+ * @param sum sum value
+ */
+ public MeanStatistic(final long samples, final long sum) {
+ if (samples > 0) {
+ this.sum = sum;
+ this.samples = samples;
+ }
+ }
+
+ /**
+ * Create from another statistic.
+ * @param that source
+ */
+ public MeanStatistic(MeanStatistic that) {
+ synchronized (that) {
+ set(that);
+ }
+ }
+
+ /**
+ * Create an empty statistic.
+ */
+ public MeanStatistic() {
+ }
+
+ /**
+ * Get the sum of samples.
+ * @return the sum
+ */
+ public synchronized long getSum() {
+ return sum;
+ }
+
+ /**
+ * Get the sample count.
+ * @return the sample count; 0 means empty
+ */
+ public synchronized long getSamples() {
+ return samples;
+ }
+
+ /**
+ * Is a statistic empty?
+ * @return true if the sample count is 0
+ */
+ @JsonIgnore
+ public synchronized boolean isEmpty() {
+ return samples == 0;
+ }
+
+ /**
+ * Set the values to 0.
+ */
+ public void clear() {
+ setSamplesAndSum(0, 0);
+ }
+
+ /**
+ * Set the sum and samples.
+ * Synchronized.
+ * @param sampleCount new sample count.
+ * @param newSum new sum
+ */
+ public synchronized void setSamplesAndSum(long sampleCount,
+ long newSum) {
+ setSamples(sampleCount);
+ setSum(newSum);
+ }
+
+ /**
+ * Set the statistic to the values of another.
+ * Synchronized.
+ * @param other the source.
+ */
+ public void set(final MeanStatistic other) {
+ setSamplesAndSum(other.getSamples(), other.getSum());
+ }
+
+ /**
+ * Set the sum.
+ * @param sum new sum
+ */
+ public synchronized void setSum(final long sum) {
+ this.sum = sum;
+ }
+
+ /**
+ * Set the sample count.
+ *
+ * If this is less than zero, it is set to zero.
+ * This stops an ill-formed JSON entry from
+ * breaking deserialization, or get an invalid sample count
+ * into an entry.
+ * @param samples sample count.
+ */
+ public synchronized void setSamples(final long samples) {
+ if (samples < 0) {
+ this.samples = 0;
+ } else {
+ this.samples = samples;
+ }
+ }
+
+ /**
+ * Get the arithmetic mean value.
+ * @return the mean
+ */
+ public synchronized double mean() {
+ return samples > 0
+ ? ((double) sum) / samples
+ : 0.0d;
+ }
+
+ /**
+ * Add another MeanStatistic.
+ * @param other other value
+ */
+ public synchronized MeanStatistic add(final MeanStatistic other) {
+ if (other.isEmpty()) {
+ return this;
+ }
+ long otherSamples;
+ long otherSum;
+ synchronized (other) {
+ otherSamples = other.samples;
+ otherSum = other.sum;
+ }
+ if (isEmpty()) {
+ samples = otherSamples;
+ sum = otherSum;
+ return this;
+ }
+ samples += otherSamples;
+ sum += otherSum;
+ return this;
+ }
+
+ /**
+ * Add a sample.
+ * Thread safe.
+ * @param value value to add to the sum
+ */
+ public synchronized void addSample(long value) {
+ samples++;
+ sum += value;
+ }
+
+ /**
+ * The hash code is derived from the mean
+ * and sample count: if either is changed
+ * the statistic cannot be used as a key
+ * for hash tables/maps.
+ * @return a hash value
+ */
+ @Override
+ public synchronized int hashCode() {
+ return Objects.hash(sum, samples);
+ }
+
+ @Override
+ public synchronized boolean equals(final Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ MeanStatistic that = (MeanStatistic) o;
+ if (isEmpty()) {
+ // if we are empty, then so must the other.
+ return that.isEmpty();
+ }
+ return getSum() == that.getSum() &&
+ getSamples() == that.getSamples();
+ }
+
+ @Override
+ public MeanStatistic clone() {
+ return copy();
+ }
+
+ /**
+ * Create a copy of this instance.
+ * @return copy.
+ *
+ */
+ public MeanStatistic copy() {
+ return new MeanStatistic(this);
+ }
+
+ @Override
+ public String toString() {
+ return String.format("(samples=%d, sum=%d, mean=%.4f)",
+ samples, sum, mean());
+ }
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java
new file mode 100644
index 0000000000000..ef04feca6917f
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java
@@ -0,0 +1,355 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.statistics;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Common statistic names for object store operations..
+ *
+ * When adding new common statistic name constants, please make them unique.
+ * By convention:
+ *
+ *
+ * - the name of the constants are uppercase, words separated by
+ * underscores.
+ * - the value of the constants are lowercase of the constant names.
+ *
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class StoreStatisticNames {
+
+ /** {@value}. */
+ public static final String OP_ABORT = "op_abort";
+
+ /** {@value}. */
+ public static final String OP_APPEND = "op_append";
+
+ /** {@value}. */
+ public static final String OP_COPY_FROM_LOCAL_FILE =
+ "op_copy_from_local_file";
+
+ /** {@value}. */
+ public static final String OP_CREATE = "op_create";
+
+ /** {@value}. */
+ public static final String OP_CREATE_NON_RECURSIVE =
+ "op_create_non_recursive";
+
+ /** {@value}. */
+ public static final String OP_DELETE = "op_delete";
+
+ /** {@value}. */
+ public static final String OP_EXISTS = "op_exists";
+
+ /** {@value}. */
+ public static final String OP_GET_CONTENT_SUMMARY =
+ "op_get_content_summary";
+
+ /** {@value}. */
+ public static final String OP_GET_DELEGATION_TOKEN =
+ "op_get_delegation_token";
+
+ /** {@value}. */
+ public static final String OP_GET_FILE_CHECKSUM =
+ "op_get_file_checksum";
+
+ /** {@value}. */
+ public static final String OP_GET_FILE_STATUS = "op_get_file_status";
+
+ /** {@value}. */
+ public static final String OP_GET_STATUS = "op_get_status";
+
+ /** {@value}. */
+ public static final String OP_GLOB_STATUS = "op_glob_status";
+
+ /** {@value}. */
+ public static final String OP_IS_FILE = "op_is_file";
+
+ /** {@value}. */
+ public static final String OP_IS_DIRECTORY = "op_is_directory";
+
+ /** {@value}. */
+ public static final String OP_LIST_FILES = "op_list_files";
+
+ /** {@value}. */
+ public static final String OP_LIST_LOCATED_STATUS =
+ "op_list_located_status";
+
+ /** {@value}. */
+ public static final String OP_LIST_STATUS = "op_list_status";
+
+ /** {@value}. */
+ public static final String OP_MKDIRS = "op_mkdirs";
+
+ /** {@value}. */
+ public static final String OP_MODIFY_ACL_ENTRIES = "op_modify_acl_entries";
+
+ /** {@value}. */
+ public static final String OP_OPEN = "op_open";
+
+ /** {@value}. */
+ public static final String OP_REMOVE_ACL = "op_remove_acl";
+
+ /** {@value}. */
+ public static final String OP_REMOVE_ACL_ENTRIES = "op_remove_acl_entries";
+
+ /** {@value}. */
+ public static final String OP_REMOVE_DEFAULT_ACL = "op_remove_default_acl";
+
+ /** {@value}. */
+ public static final String OP_RENAME = "op_rename";
+
+ /** {@value}. */
+ public static final String OP_SET_ACL = "op_set_acl";
+
+ /** {@value}. */
+ public static final String OP_SET_OWNER = "op_set_owner";
+
+ /** {@value}. */
+ public static final String OP_SET_PERMISSION = "op_set_permission";
+
+ /** {@value}. */
+ public static final String OP_SET_TIMES = "op_set_times";
+
+ /** {@value}. */
+ public static final String OP_TRUNCATE = "op_truncate";
+
+ /* The XAttr API */
+
+ /** Invoke {@code getXAttrs(Path path)}: {@value}. */
+ public static final String OP_XATTR_GET_MAP = "op_xattr_get_map";
+
+ /** Invoke {@code getXAttr(Path, String)}: {@value}. */
+ public static final String OP_XATTR_GET_NAMED = "op_xattr_get_named";
+
+ /**
+ * Invoke {@code getXAttrs(Path path, List names)}: {@value}.
+ */
+ public static final String OP_XATTR_GET_NAMED_MAP =
+ "op_xattr_get_named_map";
+
+ /** Invoke {@code listXAttrs(Path path)}: {@value}. */
+ public static final String OP_XATTR_LIST = "op_xattr_list";
+
+
+ /** {@value}. */
+ public static final String DELEGATION_TOKENS_ISSUED
+ = "delegation_tokens_issued";
+
+ /** Requests throttled and retried: {@value}. */
+ public static final String STORE_IO_THROTTLED
+ = "store_io_throttled";
+
+ /** Requests made of a store: {@value}. */
+ public static final String STORE_IO_REQUEST
+ = "store_io_request";
+
+ /**
+ * IO retried: {@value}.
+ */
+ public static final String STORE_IO_RETRY
+ = "store_io_retry";
+
+ /**
+ * A store's equivalent of a paged LIST request was initiated: {@value}.
+ */
+ public static final String OBJECT_LIST_REQUEST
+ = "object_list_request";
+
+ /**
+ * Number of continued object listings made.
+ * Value :{@value}.
+ */
+ public static final String OBJECT_CONTINUE_LIST_REQUEST =
+ "object_continue_list_request";
+
+ /**
+ * A bulk DELETE request was made: {@value}.
+ * A separate statistic from {@link #OBJECT_DELETE_REQUEST}
+ * so that metrics on duration of the operations can
+ * be distinguished.
+ */
+ public static final String OBJECT_BULK_DELETE_REQUEST
+ = "object_bulk_delete_request";
+
+ /**
+ * A store's equivalent of a DELETE request was made: {@value}.
+ * This may be an HTTP DELETE verb, or it may be some custom
+ * operation which takes a list of objects to delete.
+ */
+ public static final String OBJECT_DELETE_REQUEST
+ = "object_delete_request";
+
+ /**
+ * The count of objects deleted in delete requests.
+ */
+ public static final String OBJECT_DELETE_OBJECTS
+ = "object_delete_objects";
+
+ /**
+ * Object multipart upload initiated.
+ * Value :{@value}.
+ */
+ public static final String OBJECT_MULTIPART_UPLOAD_INITIATED =
+ "object_multipart_initiated";
+
+ /**
+ * Object multipart upload aborted.
+ * Value :{@value}.
+ */
+ public static final String OBJECT_MULTIPART_UPLOAD_ABORTED =
+ "object_multipart_aborted";
+
+ /**
+ * Object put/multipart upload count.
+ * Value :{@value}.
+ */
+ public static final String OBJECT_PUT_REQUEST =
+ "object_put_request";
+
+ /**
+ * Object put/multipart upload completed count.
+ * Value :{@value}.
+ */
+ public static final String OBJECT_PUT_REQUEST_COMPLETED =
+ "object_put_request_completed";
+
+ /**
+ * Current number of active put requests.
+ * Value :{@value}.
+ */
+ public static final String OBJECT_PUT_REQUEST_ACTIVE =
+ "object_put_request_active";
+
+ /**
+ * number of bytes uploaded.
+ * Value :{@value}.
+ */
+ public static final String OBJECT_PUT_BYTES =
+ "object_put_bytes";
+
+ /**
+ * number of bytes queued for upload/being actively uploaded.
+ * Value :{@value}.
+ */
+ public static final String OBJECT_PUT_BYTES_PENDING =
+ "object_put_bytes_pending";
+
+ /**
+ * Count of S3 Select (or similar) requests issued.
+ * Value :{@value}.
+ */
+ public static final String OBJECT_SELECT_REQUESTS =
+ "object_select_requests";
+
+ /**
+ * Suffix to use for a minimum value when
+ * the same key is shared across min/mean/max
+ * statistics.
+ *
+ * Value {@value}.
+ */
+ public static final String SUFFIX_MIN = ".min";
+
+ /**
+ * Suffix to use for a maximum value when
+ * the same key is shared across max/mean/max
+ * statistics.
+ *
+ * Value {@value}.
+ */
+ public static final String SUFFIX_MAX = ".max";
+
+ /**
+ * Suffix to use for a mean statistic value when
+ * the same key is shared across mean/mean/max
+ * statistics.
+ *
+ * Value {@value}.
+ */
+ public static final String SUFFIX_MEAN = ".mean";
+
+ /**
+ * String to add to counters and other stats to track failures.
+ * This comes before the .min/.mean//max suffixes.
+ *
+ * Value {@value}.
+ */
+ public static final String SUFFIX_FAILURES = ".failures";
+
+ /**
+ * The name of the statistic collected for executor acquisition if
+ * a duration tracker factory is passed in to the constructor.
+ * {@value}.
+ */
+ public static final String ACTION_EXECUTOR_ACQUIRED =
+ "action_executor_acquired";
+
+ /**
+ * An HTTP HEAD request was made: {@value}.
+ */
+ public static final String ACTION_HTTP_HEAD_REQUEST
+ = "action_http_head_request";
+
+ /**
+ * An HTTP GET request was made: {@value}.
+ */
+ public static final String ACTION_HTTP_GET_REQUEST
+ = "action_http_get_request";
+
+ /**
+ * An HTTP HEAD request was made: {@value}.
+ */
+ public static final String OBJECT_METADATA_REQUESTS
+ = "object_metadata_request";
+
+ public static final String OBJECT_COPY_REQUESTS
+ = "object_copy_requests";
+
+ public static final String STORE_IO_THROTTLE_RATE
+ = "store_io_throttle_rate";
+
+ public static final String MULTIPART_UPLOAD_INSTANTIATED
+ = "multipart_instantiated";
+
+ public static final String MULTIPART_UPLOAD_PART_PUT
+ = "multipart_upload_part_put";
+
+ public static final String MULTIPART_UPLOAD_PART_PUT_BYTES
+ = "multipart_upload_part_put_bytes";
+
+ public static final String MULTIPART_UPLOAD_ABORTED
+ = "multipart_upload_aborted";
+
+ public static final String MULTIPART_UPLOAD_ABORT_UNDER_PATH_INVOKED
+ = "multipart_upload_abort_under_path_invoked";
+
+ public static final String MULTIPART_UPLOAD_COMPLETED
+ = "multipart_upload_completed";
+
+ public static final String MULTIPART_UPLOAD_STARTED
+ = "multipart_upload_started";
+
+ private StoreStatisticNames() {
+ }
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StreamStatisticNames.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StreamStatisticNames.java
new file mode 100644
index 0000000000000..bbb8517118e16
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StreamStatisticNames.java
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.statistics;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * These are common statistic names.
+ *
+ * When adding new common statistic name constants, please make them unique.
+ * By convention, they are implicitly unique:
+ *
+ * -
+ * The name of the constants are uppercase, words separated by
+ * underscores.
+ *
+ * -
+ * The value of the constants are lowercase of the constant names.
+ *
+ *
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class StreamStatisticNames {
+
+ /**
+ * Count of times the TCP stream was aborted.
+ * Value: {@value}.
+ */
+ public static final String STREAM_READ_ABORTED = "stream_aborted";
+
+ /**
+ * Bytes read from an input stream in read() calls.
+ * Does not include bytes read and then discarded in seek/close etc.
+ * These are the bytes returned to the caller.
+ * Value: {@value}.
+ */
+ public static final String STREAM_READ_BYTES
+ = "stream_read_bytes";
+
+ /**
+ * Count of bytes discarded by aborting an input stream .
+ * Value: {@value}.
+ */
+ public static final String STREAM_READ_BYTES_DISCARDED_ABORT
+ = "stream_read_bytes_discarded_in_abort";
+
+ /**
+ * Count of bytes read and discarded when closing an input stream.
+ * Value: {@value}.
+ */
+ public static final String STREAM_READ_BYTES_DISCARDED_CLOSE
+ = "stream_read_bytes_discarded_in_close";
+
+ /**
+ * Count of times the TCP stream was closed.
+ * Value: {@value}.
+ */
+ public static final String STREAM_READ_CLOSED = "stream_read_closed";
+
+ /**
+ * Total count of times an attempt to close an input stream was made
+ * Value: {@value}.
+ */
+ public static final String STREAM_READ_CLOSE_OPERATIONS
+ = "stream_read_close_operations";
+
+ /**
+ * Total count of times an input stream to was opened.
+ * For object stores, that means the count a GET request was initiated.
+ * Value: {@value}.
+ */
+ public static final String STREAM_READ_OPENED = "stream_read_opened";
+
+ /**
+ * Count of exceptions raised during input stream reads.
+ * Value: {@value}.
+ */
+ public static final String STREAM_READ_EXCEPTIONS =
+ "stream_read_exceptions";
+
+ /**
+ * Count of readFully() operations in an input stream.
+ * Value: {@value}.
+ */
+ public static final String STREAM_READ_FULLY_OPERATIONS
+ = "stream_read_fully_operations";
+
+ /**
+ * Count of read() operations in an input stream.
+ * Value: {@value}.
+ */
+ public static final String STREAM_READ_OPERATIONS =
+ "stream_read_operations";
+
+ /**
+ * Count of incomplete read() operations in an input stream,
+ * that is, when the bytes returned were less than that requested.
+ * Value: {@value}.
+ */
+ public static final String STREAM_READ_OPERATIONS_INCOMPLETE
+ = "stream_read_operations_incomplete";
+
+ /**
+ * Count of version mismatches encountered while reading an input stream.
+ * Value: {@value}.
+ */
+ public static final String STREAM_READ_VERSION_MISMATCHES
+ = "stream_read_version_mismatches";
+
+ /**
+ * Count of executed seek operations which went backwards in a stream.
+ * Value: {@value}.
+ */
+ public static final String STREAM_READ_SEEK_BACKWARD_OPERATIONS =
+ "stream_read_seek_backward_operations";
+
+ /**
+ * Count of bytes moved backwards during seek operations
+ * in an input stream.
+ * Value: {@value}.
+ */
+ public static final String STREAM_READ_SEEK_BYTES_BACKWARDS
+ = "stream_read_bytes_backwards_on_seek";
+
+ /**
+ * Count of bytes read and discarded during seek() in an input stream.
+ * Value: {@value}.
+ */
+ public static final String STREAM_READ_SEEK_BYTES_DISCARDED =
+ "stream_read_seek_bytes_discarded";
+
+ /**
+ * Count of bytes skipped during forward seek operations.
+ * Value: {@value}.
+ */
+ public static final String STREAM_READ_SEEK_BYTES_SKIPPED
+ = "stream_read_seek_bytes_skipped";
+
+ /**
+ * Count of executed seek operations which went forward in
+ * an input stream.
+ * Value: {@value}.
+ */
+ public static final String STREAM_READ_SEEK_FORWARD_OPERATIONS
+ = "stream_read_seek_forward_operations";
+
+ /**
+ * Count of times the seek policy was dynamically changed
+ * in an input stream.
+ * Value: {@value}.
+ */
+ public static final String STREAM_READ_SEEK_POLICY_CHANGED =
+ "stream_read_seek_policy_changed";
+
+ /**
+ * Count of seek operations in an input stream.
+ * Value: {@value}.
+ */
+ public static final String STREAM_READ_SEEK_OPERATIONS =
+ "stream_read_seek_operations";
+
+ /**
+ * Count of {@code InputStream.skip()} calls.
+ * Value: {@value}.
+ */
+ public static final String STREAM_READ_SKIP_OPERATIONS =
+ "stream_read_skip_operations";
+
+ /**
+ * Count bytes skipped in {@code InputStream.skip()} calls.
+ * Value: {@value}.
+ */
+ public static final String STREAM_READ_SKIP_BYTES =
+ "stream_read_skip_bytes";
+
+ /**
+ * Total count of bytes read from an input stream.
+ * Value: {@value}.
+ */
+ public static final String STREAM_READ_TOTAL_BYTES
+ = "stream_read_total_bytes";
+
+ /**
+ * Count of calls of {@code CanUnbuffer.unbuffer()}.
+ * Value: {@value}.
+ */
+ public static final String STREAM_READ_UNBUFFERED
+ = "stream_read_unbuffered";
+
+ /**
+ * "Count of stream write failures reported.
+ * Value: {@value}.
+ */
+ public static final String STREAM_WRITE_EXCEPTIONS =
+ "stream_write_exceptions";
+
+ /**
+ * Count of failures when finalizing a multipart upload:
+ * {@value}.
+ */
+ public static final String STREAM_WRITE_EXCEPTIONS_COMPLETING_UPLOADS =
+ "stream_write_exceptions_completing_upload";
+
+ /**
+ * Count of block/partition uploads complete.
+ * Value: {@value}.
+ */
+ public static final String STREAM_WRITE_BLOCK_UPLOADS
+ = "stream_write_block_uploads";
+
+ /**
+ * Count of number of block uploads aborted.
+ * Value: {@value}.
+ */
+ public static final String STREAM_WRITE_BLOCK_UPLOADS_ABORTED
+ = "stream_write_block_uploads_aborted";
+
+ /**
+ * Count of block/partition uploads active.
+ * Value: {@value}.
+ */
+ public static final String STREAM_WRITE_BLOCK_UPLOADS_ACTIVE
+ = "stream_write_block_uploads_active";
+
+ /**
+ * Gauge of data queued to be written.
+ * Value: {@value}.
+ */
+ public static final String STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING =
+ "stream_write_block_uploads_data_pending";
+
+ /**
+ * Count of number of block uploads committed.
+ * Value: {@value}.
+ */
+ public static final String STREAM_WRITE_BLOCK_UPLOADS_COMMITTED
+ = "stream_write_block_uploads_committed";
+
+ /**
+ * Gauge of block/partitions uploads queued to be written.
+ * Value: {@value}.
+ */
+ public static final String STREAM_WRITE_BLOCK_UPLOADS_PENDING
+ = "stream_write_block_uploads_pending";
+
+
+ /**
+ * "Count of bytes written to output stream including all not yet uploaded.
+ * {@value}.
+ */
+ public static final String STREAM_WRITE_BYTES
+ = "stream_write_bytes";
+
+ /**
+ * Count of total time taken for uploads to complete.
+ * {@value}.
+ */
+ public static final String STREAM_WRITE_TOTAL_TIME
+ = "stream_write_total_time";
+
+ /**
+ * Total queue duration of all block uploads.
+ * {@value}.
+ */
+ public static final String STREAM_WRITE_QUEUE_DURATION
+ = "stream_write_queue_duration";
+
+ public static final String STREAM_WRITE_TOTAL_DATA
+ = "stream_write_total_data";
+
+ /**
+ * Number of bytes to upload from an OutputStream.
+ */
+ public static final String BYTES_TO_UPLOAD
+ = "bytes_upload";
+
+ /**
+ * Number of bytes uploaded successfully to the object store.
+ */
+ public static final String BYTES_UPLOAD_SUCCESSFUL
+ = "bytes_upload_successfully";
+
+ /**
+ * Number of bytes failed to upload to the object store.
+ */
+ public static final String BYTES_UPLOAD_FAILED
+ = "bytes_upload_failed";
+
+ /**
+ * Total time spent on waiting for a task to complete.
+ */
+ public static final String TIME_SPENT_ON_TASK_WAIT
+ = "time_spent_task_wait";
+
+ /**
+ * Number of task queue shrunk operations.
+ */
+ public static final String QUEUE_SHRUNK_OPS
+ = "queue_shrunk_ops";
+
+ /**
+ * Number of times current buffer is written to the service.
+ */
+ public static final String WRITE_CURRENT_BUFFER_OPERATIONS
+ = "write_current_buffer_ops";
+
+ /**
+ * Total time spent on completing a PUT request.
+ */
+ public static final String TIME_SPENT_ON_PUT_REQUEST
+ = "time_spent_on_put_request";
+
+ /**
+ * Number of seeks in buffer.
+ */
+ public static final String SEEK_IN_BUFFER
+ = "seek_in_buffer";
+
+ /**
+ * Number of bytes read from the buffer.
+ */
+ public static final String BYTES_READ_BUFFER
+ = "bytes_read_buffer";
+
+ /**
+ * Total number of remote read operations performed.
+ */
+ public static final String REMOTE_READ_OP
+ = "remote_read_op";
+
+ /**
+ * Total number of bytes read from readAhead.
+ */
+ public static final String READ_AHEAD_BYTES_READ
+ = "read_ahead_bytes_read";
+
+ /**
+ * Total number of bytes read from remote operations.
+ */
+ public static final String REMOTE_BYTES_READ
+ = "remote_bytes_read";
+
+ private StreamStatisticNames() {
+ }
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/AbstractIOStatisticsImpl.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/AbstractIOStatisticsImpl.java
new file mode 100644
index 0000000000000..c701a509d8951
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/AbstractIOStatisticsImpl.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.statistics.impl;
+
+import org.apache.hadoop.fs.statistics.IOStatistics;
+
+/**
+ * Base implementation in case common methods/fields need to be added
+ * in future.
+ */
+public abstract class AbstractIOStatisticsImpl implements IOStatistics {
+
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/DynamicIOStatistics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/DynamicIOStatistics.java
new file mode 100644
index 0000000000000..50c2625c3513d
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/DynamicIOStatistics.java
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.statistics.impl;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.function.Function;
+
+import org.apache.hadoop.fs.statistics.MeanStatistic;
+
+/**
+ * These statistics are dynamically evaluated by the supplied
+ * String -> type functions.
+ *
+ * This allows statistic sources to supply a list of callbacks used to
+ * generate the statistics on demand; similar to some of the Coda Hale metrics.
+ *
+ * The evaluation actually takes place during the iteration's {@code next()}
+ * call.
+ */
+final class DynamicIOStatistics
+ extends AbstractIOStatisticsImpl {
+
+ /**
+ * Counter evaluators.
+ */
+ private final EvaluatingStatisticsMap counters
+ = new EvaluatingStatisticsMap<>();
+
+ private final EvaluatingStatisticsMap gauges
+ = new EvaluatingStatisticsMap<>();
+
+ private final EvaluatingStatisticsMap minimums
+ = new EvaluatingStatisticsMap<>();
+
+ private final EvaluatingStatisticsMap maximums
+ = new EvaluatingStatisticsMap<>();
+
+ private final EvaluatingStatisticsMap meanStatistics
+ = new EvaluatingStatisticsMap<>(MeanStatistic::copy);
+
+ DynamicIOStatistics() {
+ }
+
+ @Override
+ public Map counters() {
+ return Collections.unmodifiableMap(counters);
+ }
+
+ @Override
+ public Map gauges() {
+ return Collections.unmodifiableMap(gauges);
+ }
+
+ @Override
+ public Map minimums() {
+ return Collections.unmodifiableMap(minimums);
+ }
+
+ @Override
+ public Map maximums() {
+ return Collections.unmodifiableMap(maximums);
+ }
+
+ @Override
+ public Map meanStatistics() {
+ return Collections.unmodifiableMap(meanStatistics);
+ }
+
+ /**
+ * add a mapping of a key to a counter function.
+ * @param key the key
+ * @param eval the evaluator
+ */
+ void addCounterFunction(String key, Function eval) {
+ counters.addFunction(key, eval);
+ }
+
+ /**
+ * add a mapping of a key to a gauge function.
+ * @param key the key
+ * @param eval the evaluator
+ */
+ void addGaugeFunction(String key, Function eval) {
+ gauges.addFunction(key, eval);
+ }
+
+ /**
+ * add a mapping of a key to a minimum function.
+ * @param key the key
+ * @param eval the evaluator
+ */
+ void addMinimumFunction(String key, Function eval) {
+ minimums.addFunction(key, eval);
+ }
+
+ /**
+ * add a mapping of a key to a maximum function.
+ * @param key the key
+ * @param eval the evaluator
+ */
+ void addMaximumFunction(String key, Function eval) {
+ maximums.addFunction(key, eval);
+ }
+
+ /**
+ * add a mapping of a key to a meanStatistic function.
+ * @param key the key
+ * @param eval the evaluator
+ */
+ void addMeanStatisticFunction(String key,
+ Function eval) {
+ meanStatistics.addFunction(key, eval);
+ }
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/DynamicIOStatisticsBuilder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/DynamicIOStatisticsBuilder.java
new file mode 100644
index 0000000000000..47a317076dcf2
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/DynamicIOStatisticsBuilder.java
@@ -0,0 +1,248 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.statistics.impl;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Function;
+import java.util.function.ToLongFunction;
+
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.MeanStatistic;
+import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+
+import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkState;
+
+/**
+ * Builder of {@link DynamicIOStatistics}.
+ *
+ * Instantiate through
+ * {@link IOStatisticsBinding#dynamicIOStatistics()}.
+ */
+public class DynamicIOStatisticsBuilder {
+
+ /**
+ * the instance being built up. Will be null after the (single)
+ * call to {@link #build()}.
+ */
+ private DynamicIOStatistics instance = new DynamicIOStatistics();
+
+ /**
+ * Build the IOStatistics instance.
+ * @return an instance.
+ * @throws IllegalStateException if the builder has already been built.
+ */
+ public IOStatistics build() {
+ final DynamicIOStatistics stats = activeInstance();
+ // stop the builder from working any more.
+ instance = null;
+ return stats;
+ }
+
+ /**
+ * Get the statistics instance.
+ * @return the instance to build/return
+ * @throws IllegalStateException if the builder has already been built.
+ */
+ private DynamicIOStatistics activeInstance() {
+ checkState(instance != null, "Already built");
+ return instance;
+ }
+
+ /**
+ * Add a new evaluator to the counter statistics.
+ * @param key key of this statistic
+ * @param eval evaluator for the statistic
+ * @return the builder.
+ */
+ public DynamicIOStatisticsBuilder withLongFunctionCounter(String key,
+ ToLongFunction eval) {
+ activeInstance().addCounterFunction(key, eval::applyAsLong);
+ return this;
+ }
+
+ /**
+ * Add a counter statistic to dynamically return the
+ * latest value of the source.
+ * @param key key of this statistic
+ * @param source atomic long counter
+ * @return the builder.
+ */
+ public DynamicIOStatisticsBuilder withAtomicLongCounter(String key,
+ AtomicLong source) {
+ withLongFunctionCounter(key, s -> source.get());
+ return this;
+ }
+
+ /**
+ * Add a counter statistic to dynamically return the
+ * latest value of the source.
+ * @param key key of this statistic
+ * @param source atomic int counter
+ * @return the builder.
+ */
+ public DynamicIOStatisticsBuilder withAtomicIntegerCounter(String key,
+ AtomicInteger source) {
+ withLongFunctionCounter(key, s -> source.get());
+ return this;
+ }
+
+ /**
+ * Build a dynamic counter statistic from a
+ * {@link MutableCounterLong}.
+ * @param key key of this statistic
+ * @param source mutable long counter
+ * @return the builder.
+ */
+ public DynamicIOStatisticsBuilder withMutableCounter(String key,
+ MutableCounterLong source) {
+ withLongFunctionCounter(key, s -> source.value());
+ return this;
+ }
+
+ /**
+ * Add a new evaluator to the gauge statistics.
+ * @param key key of this statistic
+ * @param eval evaluator for the statistic
+ * @return the builder.
+ */
+ public DynamicIOStatisticsBuilder withLongFunctionGauge(String key,
+ ToLongFunction eval) {
+ activeInstance().addGaugeFunction(key, eval::applyAsLong);
+ return this;
+ }
+
+ /**
+ * Add a gauge statistic to dynamically return the
+ * latest value of the source.
+ * @param key key of this statistic
+ * @param source atomic long gauge
+ * @return the builder.
+ */
+ public DynamicIOStatisticsBuilder withAtomicLongGauge(String key,
+ AtomicLong source) {
+ withLongFunctionGauge(key, s -> source.get());
+ return this;
+ }
+
+ /**
+ * Add a gauge statistic to dynamically return the
+ * latest value of the source.
+ * @param key key of this statistic
+ * @param source atomic int gauge
+ * @return the builder.
+ */
+ public DynamicIOStatisticsBuilder withAtomicIntegerGauge(String key,
+ AtomicInteger source) {
+ withLongFunctionGauge(key, s -> source.get());
+ return this;
+ }
+
+ /**
+ * Add a new evaluator to the minimum statistics.
+ * @param key key of this statistic
+ * @param eval evaluator for the statistic
+ * @return the builder.
+ */
+ public DynamicIOStatisticsBuilder withLongFunctionMinimum(String key,
+ ToLongFunction eval) {
+ activeInstance().addMinimumFunction(key, eval::applyAsLong);
+ return this;
+ }
+
+ /**
+ * Add a minimum statistic to dynamically return the
+ * latest value of the source.
+ * @param key key of this statistic
+ * @param source atomic long minimum
+ * @return the builder.
+ */
+ public DynamicIOStatisticsBuilder withAtomicLongMinimum(String key,
+ AtomicLong source) {
+ withLongFunctionMinimum(key, s -> source.get());
+ return this;
+ }
+
+ /**
+ * Add a minimum statistic to dynamically return the
+ * latest value of the source.
+ * @param key key of this statistic
+ * @param source atomic int minimum
+ * @return the builder.
+ */
+ public DynamicIOStatisticsBuilder withAtomicIntegerMinimum(String key,
+ AtomicInteger source) {
+ withLongFunctionMinimum(key, s -> source.get());
+ return this;
+ }
+
+
+ /**
+ * Add a new evaluator to the maximum statistics.
+ * @param key key of this statistic
+ * @param eval evaluator for the statistic
+ * @return the builder.
+ */
+ public DynamicIOStatisticsBuilder withLongFunctionMaximum(String key,
+ ToLongFunction eval) {
+ activeInstance().addMaximumFunction(key, eval::applyAsLong);
+ return this;
+ }
+
+ /**
+ * Add a maximum statistic to dynamically return the
+ * latest value of the source.
+ * @param key key of this statistic
+ * @param source atomic long maximum
+ * @return the builder.
+ */
+ public DynamicIOStatisticsBuilder withAtomicLongMaximum(String key,
+ AtomicLong source) {
+ withLongFunctionMaximum(key, s -> source.get());
+ return this;
+ }
+
+ /**
+ * Add a maximum statistic to dynamically return the
+ * latest value of the source.
+ * @param key key of this statistic
+ * @param source atomic int maximum
+ * @return the builder.
+ */
+ public DynamicIOStatisticsBuilder withAtomicIntegerMaximum(String key,
+ AtomicInteger source) {
+ withLongFunctionMaximum(key, s -> source.get());
+ return this;
+ }
+
+ /**
+ * Add a new evaluator to the mean statistics.
+ *
+ * This is a function which must return the mean and the sample count.
+ * @param key key of this statistic
+ * @param eval evaluator for the statistic
+ * @return the builder.
+ */
+ public DynamicIOStatisticsBuilder withMeanStatisticFunction(String key,
+ Function eval) {
+ activeInstance().addMeanStatisticFunction(key, eval);
+ return this;
+ }
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/EmptyIOStatistics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/EmptyIOStatistics.java
new file mode 100644
index 0000000000000..f474fc209771c
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/EmptyIOStatistics.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.statistics.impl;
+
+import java.util.Map;
+
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.MeanStatistic;
+
+import static java.util.Collections.emptyMap;
+
+/**
+ * An empty IO Statistics implementation for classes which always
+ * want to return a non-null set of statistics.
+ */
+final class EmptyIOStatistics extends AbstractIOStatisticsImpl {
+
+ /**
+ * The sole instance of this class.
+ */
+ private static final EmptyIOStatistics INSTANCE = new EmptyIOStatistics();
+
+ private EmptyIOStatistics() {
+ }
+
+ @Override
+ public Map counters() {
+ return emptyMap();
+ }
+
+ @Override
+ public Map gauges() {
+ return emptyMap();
+ }
+
+ @Override
+ public Map minimums() {
+ return emptyMap();
+ }
+
+ @Override
+ public Map maximums() {
+ return emptyMap();
+ }
+
+ @Override
+ public Map meanStatistics() {
+ return emptyMap();
+ }
+
+ /**
+ * Get the single instance of this class.
+ * @return a shared, empty instance.
+ */
+ public static IOStatistics getInstance() {
+ return INSTANCE;
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/EvaluatingStatisticsMap.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/EvaluatingStatisticsMap.java
new file mode 100644
index 0000000000000..e4680f2d81fa0
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/EvaluatingStatisticsMap.java
@@ -0,0 +1,202 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.statistics.impl;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/**
+ * A map of functions which can be invoked to dynamically
+ * create the value of an entry.
+ * @param type of entry value.
+ */
+final class EvaluatingStatisticsMap implements
+ Map {
+
+ /**
+ * Functions to invoke when evaluating keys.
+ */
+ private final Map> evaluators
+ = new ConcurrentHashMap<>();
+
+ /**
+ * Function to use when copying map values.
+ */
+ private final Function copyFn;
+
+ /**
+ * Construct with the copy function being simple passthrough.
+ */
+ EvaluatingStatisticsMap() {
+ this(IOStatisticsBinding::passthroughFn);
+ }
+
+ /**
+ * Construct with the copy function being that supplied in.
+ * @param copyFn copy function.
+ */
+ EvaluatingStatisticsMap(final Function copyFn) {
+ this.copyFn = copyFn;
+ }
+
+ /**
+ * add a mapping of a key to a function.
+ * @param key the key
+ * @param eval the evaluator
+ */
+ void addFunction(String key, Function eval) {
+ evaluators.put(key, eval);
+ }
+
+ @Override
+ public int size() {
+ return evaluators.size();
+ }
+
+ @Override
+ public boolean isEmpty() {
+ return evaluators.isEmpty();
+ }
+
+ @Override
+ public boolean containsKey(final Object key) {
+ return evaluators.containsKey(key);
+ }
+
+ @Override
+ public boolean containsValue(final Object value) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public E get(final Object key) {
+ Function fn = evaluators.get(key);
+ return fn != null
+ ? fn.apply((String) key)
+ : null;
+ }
+
+ @Override
+ public E put(final String key, final E value) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public E remove(final Object key) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void putAll(final Map extends String, ? extends E> m) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void clear() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public Set keySet() {
+ return evaluators.keySet();
+ }
+
+ /**
+ * Evaluate all the entries and provide a list of the results.
+ *
+ * This is not a snapshot, so if the evaluators actually return
+ * references to mutable objects (e.g. a MeanStatistic instance)
+ * then that value may still change.
+ * @return the current list of evaluated results.
+ */
+ @Override
+ public Collection values() {
+ Set>> evalEntries =
+ evaluators.entrySet();
+ return evalEntries.parallelStream().map((e) ->
+ e.getValue().apply(e.getKey()))
+ .collect(Collectors.toList());
+ }
+
+ /**
+ * Take a snapshot.
+ * @return a map snapshot.
+ */
+ public Map snapshot() {
+ return IOStatisticsBinding.snapshotMap(this, copyFn);
+ }
+
+ /**
+ * Creating the entry set forces an evaluation of the functions.
+ *
+ * This is not a snapshot, so if the evaluators actually return
+ * references to mutable objects (e.g. a MeanStatistic instance)
+ * then that value may still change.
+ *
+ * The evaluation may be parallelized.
+ * @return an evaluated set of values
+ */
+ @Override
+ public synchronized Set> entrySet() {
+ Set>> evalEntries =
+ evaluators.entrySet();
+ Set> r = evalEntries.parallelStream().map((e) ->
+ new EntryImpl<>(e.getKey(), e.getValue().apply(e.getKey())))
+ .collect(Collectors.toSet());
+ return r;
+ }
+
+ /**
+ * Simple entry.
+ * @param entry type
+ */
+ private static final class EntryImpl implements Entry {
+
+ private String key;
+
+ private E value;
+
+ private EntryImpl(final String key, final E value) {
+ this.key = key;
+ this.value = value;
+ }
+
+ @Override
+ public String getKey() {
+ return key;
+ }
+
+ @Override
+ public E getValue() {
+ return value;
+ }
+
+ @Override
+ public E setValue(final E val) {
+ this.value = val;
+ return val;
+ }
+ }
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java
new file mode 100644
index 0000000000000..c3507dbc73ef4
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java
@@ -0,0 +1,616 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.statistics.impl;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+import org.apache.hadoop.fs.StorageStatistics;
+import org.apache.hadoop.fs.statistics.DurationTracker;
+import org.apache.hadoop.fs.statistics.DurationTrackerFactory;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+import org.apache.hadoop.fs.statistics.MeanStatistic;
+import org.apache.hadoop.util.functional.CallableRaisingIOE;
+import org.apache.hadoop.util.functional.ConsumerRaisingIOE;
+import org.apache.hadoop.util.functional.FunctionRaisingIOE;
+import org.apache.hadoop.util.functional.InvocationRaisingIOE;
+
+import static org.apache.hadoop.fs.statistics.IOStatistics.MIN_UNSET_VALUE;
+import static org.apache.hadoop.fs.statistics.impl.StubDurationTracker.STUB_DURATION_TRACKER;
+
+/**
+ * Support for implementing IOStatistics interfaces.
+ */
+public final class IOStatisticsBinding {
+
+ /** Pattern used for each entry. */
+ public static final String ENTRY_PATTERN = "(%s=%s)";
+
+ /** String to return when a source is null. */
+ @VisibleForTesting
+ public static final String NULL_SOURCE = "()";
+
+ private IOStatisticsBinding() {
+ }
+
+ /**
+ * Create IOStatistics from a storage statistics instance.
+ *
+ * This will be updated as the storage statistics change.
+ * @param storageStatistics source data.
+ * @return an IO statistics source.
+ */
+ public static IOStatistics fromStorageStatistics(
+ StorageStatistics storageStatistics) {
+ DynamicIOStatisticsBuilder builder = dynamicIOStatistics();
+ Iterator it = storageStatistics
+ .getLongStatistics();
+ while (it.hasNext()) {
+ StorageStatistics.LongStatistic next = it.next();
+ builder.withLongFunctionCounter(next.getName(),
+ k -> storageStatistics.getLong(k));
+ }
+ return builder.build();
+ }
+
+ /**
+ * Create a builder for dynamic IO Statistics.
+ * @return a builder to be completed.
+ */
+ public static DynamicIOStatisticsBuilder dynamicIOStatistics() {
+ return new DynamicIOStatisticsBuilder();
+ }
+
+ /**
+ * Get the shared instance of the immutable empty statistics
+ * object.
+ * @return an empty statistics object.
+ */
+ public static IOStatistics emptyStatistics() {
+ return EmptyIOStatistics.getInstance();
+ }
+
+ /**
+ * Take an IOStatistics instance and wrap it in a source.
+ * @param statistics statistics.
+ * @return a source which will return the values
+ */
+ public static IOStatisticsSource wrap(IOStatistics statistics) {
+ return new SourceWrappedStatistics(statistics);
+ }
+
+ /**
+ * Create a builder for an {@link IOStatisticsStore}.
+ *
+ * @return a builder instance.
+ */
+ public static IOStatisticsStoreBuilder iostatisticsStore() {
+ return new IOStatisticsStoreBuilderImpl();
+ }
+
+ /**
+ * Convert an entry to the string format used in logging.
+ *
+ * @param entry entry to evaluate
+ * @param entry type
+ * @return formatted string
+ */
+ public static String entryToString(
+ final Map.Entry entry) {
+ return entryToString(entry.getKey(), entry.getValue());
+ }
+
+ /**
+ * Convert entry values to the string format used in logging.
+ *
+ * @param name statistic name
+ * @param value stat value
+ * @return formatted string
+ */
+ public static String entryToString(
+ final String name, final E value) {
+ return String.format(
+ ENTRY_PATTERN,
+ name,
+ value);
+ }
+
+ /**
+ * Copy into the dest map all the source entries.
+ * The destination is cleared first.
+ * @param entry type
+ * @param dest destination of the copy
+ * @param source source
+ * @param copyFn function to copy entries
+ * @return the destination.
+ */
+ private static Map copyMap(
+ Map dest,
+ Map source,
+ Function copyFn) {
+ // we have to clone the values so that they aren't
+ // bound to the original values
+ dest.clear();
+ source.entrySet()
+ .forEach(entry ->
+ dest.put(entry.getKey(), copyFn.apply(entry.getValue())));
+ return dest;
+ }
+
+ /**
+ * A passthrough copy operation suitable for immutable
+ * types, including numbers.
+ * @param src source object
+ * @return the source object
+ */
+ public static E passthroughFn(E src) {
+ return src;
+ }
+
+ /**
+ * Take a snapshot of a supplied map, where the copy option simply
+ * uses the existing value.
+ *
+ * For this to be safe, the map must refer to immutable objects.
+ * @param source source map
+ * @param type of values.
+ * @return a new map referencing the same values.
+ */
+ public static Map snapshotMap(
+ Map source) {
+ return snapshotMap(source,
+ IOStatisticsBinding::passthroughFn);
+ }
+
+ /**
+ * Take a snapshot of a supplied map, using the copy function
+ * to replicate the source values.
+ * @param source source map
+ * @param copyFn function to copy the value
+ * @param type of values.
+ * @return a concurrent hash map referencing the same values.
+ */
+ public static
+ ConcurrentHashMap snapshotMap(
+ Map source,
+ Function copyFn) {
+ ConcurrentHashMap dest = new ConcurrentHashMap<>();
+ copyMap(dest, source, copyFn);
+ return dest;
+ }
+
+ /**
+ * Aggregate two maps so that the destination.
+ * @param type of values
+ * @param dest destination map.
+ * @param other other map
+ * @param aggregateFn function to aggregate the values.
+ * @param copyFn function to copy the value
+ */
+ public static void aggregateMaps(
+ Map dest,
+ Map other,
+ BiFunction aggregateFn,
+ Function copyFn) {
+ // scan through the other hand map; copy
+ // any values not in the left map,
+ // aggregate those for which there is already
+ // an entry
+ other.entrySet().forEach(entry -> {
+ String key = entry.getKey();
+ E rVal = entry.getValue();
+ E lVal = dest.get(key);
+ if (lVal == null) {
+ dest.put(key, copyFn.apply(rVal));
+ } else {
+ dest.put(key, aggregateFn.apply(lVal, rVal));
+ }
+ });
+ }
+
+ /**
+ * Aggregate two counters.
+ * @param l left value
+ * @param r right value
+ * @return the aggregate value
+ */
+ public static Long aggregateCounters(Long l, Long r) {
+ return Math.max(l, 0) + Math.max(r, 0);
+ }
+
+ /**
+ * Add two gauges.
+ * @param l left value
+ * @param r right value
+ * @return aggregate value
+ */
+ public static Long aggregateGauges(Long l, Long r) {
+ return l + r;
+ }
+
+
+ /**
+ * Aggregate two minimum values.
+ * @param l left
+ * @param r right
+ * @return the new minimum.
+ */
+ public static Long aggregateMinimums(Long l, Long r) {
+ if (l == MIN_UNSET_VALUE) {
+ return r;
+ } else if (r == MIN_UNSET_VALUE) {
+ return l;
+ } else {
+ return Math.min(l, r);
+ }
+ }
+
+ /**
+ * Aggregate two maximum values.
+ * @param l left
+ * @param r right
+ * @return the new minimum.
+ */
+ public static Long aggregateMaximums(Long l, Long r) {
+ if (l == MIN_UNSET_VALUE) {
+ return r;
+ } else if (r == MIN_UNSET_VALUE) {
+ return l;
+ } else {
+ return Math.max(l, r);
+ }
+ }
+
+ /**
+ * Aggregate the mean statistics.
+ * This returns a new instance.
+ * @param l left value
+ * @param r right value
+ * @return aggregate value
+ */
+ public static MeanStatistic aggregateMeanStatistics(
+ MeanStatistic l, MeanStatistic r) {
+ MeanStatistic res = l.copy();
+ res.add(r);
+ return res;
+ }
+
+ /**
+ * Update a maximum value tracked in an atomic long.
+ * This is thread safe -it uses compareAndSet to ensure
+ * that Thread T1 whose sample is greater than the current
+ * value never overwrites an update from thread T2 whose
+ * sample was also higher -and which completed first.
+ * @param dest destination for all changes.
+ * @param sample sample to update.
+ */
+ public static void maybeUpdateMaximum(AtomicLong dest, long sample) {
+ boolean done;
+ do {
+ long current = dest.get();
+ if (sample > current) {
+ done = dest.compareAndSet(current, sample);
+ } else {
+ done = true;
+ }
+ } while (!done);
+ }
+
+ /**
+ * Update a maximum value tracked in an atomic long.
+ * This is thread safe -it uses compareAndSet to ensure
+ * that Thread T1 whose sample is greater than the current
+ * value never overwrites an update from thread T2 whose
+ * sample was also higher -and which completed first.
+ * @param dest destination for all changes.
+ * @param sample sample to update.
+ */
+ public static void maybeUpdateMinimum(AtomicLong dest, long sample) {
+ boolean done;
+ do {
+ long current = dest.get();
+ if (current == MIN_UNSET_VALUE || sample < current) {
+ done = dest.compareAndSet(current, sample);
+ } else {
+ done = true;
+ }
+ } while (!done);
+ }
+
+ /**
+ * Given an IOException raising function/lambda expression,
+ * return a new one which wraps the inner and tracks
+ * the duration of the operation, including whether
+ * it passes/fails.
+ * @param factory factory of duration trackers
+ * @param statistic statistic key
+ * @param inputFn input function
+ * @param type of argument to the input function.
+ * @param return type.
+ * @return a new function which tracks duration and failure.
+ */
+ public static FunctionRaisingIOE trackFunctionDuration(
+ @Nullable DurationTrackerFactory factory,
+ String statistic,
+ FunctionRaisingIOE inputFn) {
+ return (x) -> {
+ // create the tracker outside try-with-resources so
+ // that failures can be set in the catcher.
+ DurationTracker tracker = createTracker(factory, statistic);
+ try {
+ // exec the input function and return its value
+ return inputFn.apply(x);
+ } catch (IOException | RuntimeException e) {
+ // input function failed: note it
+ tracker.failed();
+ // and rethrow
+ throw e;
+ } finally {
+ // update the tracker.
+ // this is called after the catch() call would have
+ // set the failed flag.
+ tracker.close();
+ }
+ };
+ }
+
+ /**
+ * Given a java function/lambda expression,
+ * return a new one which wraps the inner and tracks
+ * the duration of the operation, including whether
+ * it passes/fails.
+ * @param factory factory of duration trackers
+ * @param statistic statistic key
+ * @param inputFn input function
+ * @param type of argument to the input function.
+ * @param return type.
+ * @return a new function which tracks duration and failure.
+ */
+ public static Function trackJavaFunctionDuration(
+ @Nullable DurationTrackerFactory factory,
+ String statistic,
+ Function inputFn) {
+ return (x) -> {
+ // create the tracker outside try-with-resources so
+ // that failures can be set in the catcher.
+ DurationTracker tracker = createTracker(factory, statistic);
+ try {
+ // exec the input function and return its value
+ return inputFn.apply(x);
+ } catch (RuntimeException e) {
+ // input function failed: note it
+ tracker.failed();
+ // and rethrow
+ throw e;
+ } finally {
+ // update the tracker.
+ // this is called after the catch() call would have
+ // set the failed flag.
+ tracker.close();
+ }
+ };
+ }
+
+ /**
+ * Given an IOException raising callable/lambda expression,
+ * execute it and update the relevant statistic.
+ * @param factory factory of duration trackers
+ * @param statistic statistic key
+ * @param input input callable.
+ * @param return type.
+ * @return the result of the operation.
+ */
+ public static B trackDuration(
+ DurationTrackerFactory factory,
+ String statistic,
+ CallableRaisingIOE input) throws IOException {
+ return trackDurationOfOperation(factory, statistic, input).apply();
+ }
+
+ /**
+ * Given an IOException raising callable/lambda expression,
+ * execute it and update the relevant statistic.
+ * @param factory factory of duration trackers
+ * @param statistic statistic key
+ * @param input input callable.
+ */
+ public static void trackDurationOfInvocation(
+ DurationTrackerFactory factory,
+ String statistic,
+ InvocationRaisingIOE input) throws IOException {
+
+ // create the tracker outside try-with-resources so
+ // that failures can be set in the catcher.
+ DurationTracker tracker = createTracker(factory, statistic);
+ try {
+ // exec the input function and return its value
+ input.apply();
+ } catch (IOException | RuntimeException e) {
+ // input function failed: note it
+ tracker.failed();
+ // and rethrow
+ throw e;
+ } finally {
+ // update the tracker.
+ // this is called after the catch() call would have
+ // set the failed flag.
+ tracker.close();
+ }
+ }
+
+ /**
+ * Given an IOException raising callable/lambda expression,
+ * return a new one which wraps the inner and tracks
+ * the duration of the operation, including whether
+ * it passes/fails.
+ * @param factory factory of duration trackers
+ * @param statistic statistic key
+ * @param input input callable.
+ * @param return type.
+ * @return a new callable which tracks duration and failure.
+ */
+ public static