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..026eff427c9f7 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
@@ -24,6 +24,10 @@
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 +37,8 @@
@InterfaceAudience.Private
@InterfaceStability.Unstable
public class BufferedFSInputStream extends BufferedInputStream
-implements Seekable, PositionedReadable, HasFileDescriptor {
+implements Seekable, PositionedReadable, HasFileDescriptor,
+ IOStatisticsSource {
/**
* Creates a BufferedFSInputStream
* with the specified buffer size,
@@ -126,4 +131,9 @@ public FileDescriptor getFileDescriptor() throws IOException {
return null;
}
}
+
+ @Override
+ public IOStatistics getIOStatistics() {
+ return retrieveIOStatistics(in);
+ }
}
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..70bf5e177529e 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
@@ -38,6 +38,9 @@
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;
@@ -134,7 +137,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 +274,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 +410,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 {
private FSDataOutputStream datas;
private FSDataOutputStream sums;
private static final float CHKSUM_AS_FRACTION = 0.01f;
@@ -449,6 +465,17 @@ 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);
+ }
}
@Override
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..b63e047358c70 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,9 @@
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 org.apache.hadoop.fs.statistics.IOStatisticsSupport;
import org.apache.hadoop.io.ByteBufferPool;
import org.apache.hadoop.util.IdentityHashStore;
@@ -40,7 +43,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
@@ -267,4 +270,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..27d164b7d87ba 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,17 @@
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 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 {
private final OutputStream wrappedStream;
private static class PositionCache extends FilterOutputStream {
@@ -155,4 +159,15 @@ 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);
+ }
}
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..373120ce3078f 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
@@ -24,6 +24,9 @@
import 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/RawLocalFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
index cf2210575da15..7e26378128059 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
@@ -47,6 +47,10 @@
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
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.impl.BufferedIOStatisticsOutputStream;
+import org.apache.hadoop.fs.statistics.impl.CounterIOStatistics;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.nativeio.NativeIO;
import org.apache.hadoop.util.Progressable;
@@ -54,6 +58,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.counterIOStatistics;
/****************************************************************
* Implement the FileSystem API for the raw local filesystem.
@@ -105,10 +117,21 @@ public void initialize(URI uri, Configuration conf) throws IOException {
/*******************************************************
* For open()'s FSInputStream.
*******************************************************/
- class LocalFSFileInputStream extends FSInputStream implements HasFileDescriptor {
+ class LocalFSFileInputStream extends FSInputStream implements HasFileDescriptor,
+ IOStatisticsSource {
private FileInputStream fis;
private long position;
+ /**
+ * Minimal set of counters.
+ */
+ private final CounterIOStatistics ioStatistics = counterIOStatistics(
+ STREAM_READ_BYTES,
+ STREAM_READ_EXCEPTIONS,
+ STREAM_READ_SEEK_OPERATIONS,
+ STREAM_READ_SKIP_OPERATIONS,
+ STREAM_READ_SKIP_BYTES);
+
public LocalFSFileInputStream(Path f) throws IOException {
fis = new FileInputStream(pathToFile(f));
}
@@ -150,9 +173,11 @@ public int read() throws IOException {
if (value >= 0) {
this.position++;
statistics.incrementBytesRead(1);
+ ioStatistics.increment(STREAM_READ_BYTES, 1);
}
return value;
} catch (IOException e) { // unexpected exception
+ ioStatistics.increment(STREAM_READ_EXCEPTIONS, 1);
throw new FSError(e); // assume native fs error
}
}
@@ -166,9 +191,11 @@ public int read(byte[] b, int off, int len) throws IOException {
if (value > 0) {
this.position += value;
statistics.incrementBytesRead(value);
+ ioStatistics.increment(STREAM_READ_BYTES, value);
}
return value;
} catch (IOException e) { // unexpected exception
+ ioStatistics.increment(STREAM_READ_EXCEPTIONS, 1);
throw new FSError(e); // assume native fs error
}
}
@@ -187,18 +214,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.increment(STREAM_READ_BYTES, value);
}
return value;
} catch (IOException e) {
+ ioStatistics.increment(STREAM_READ_EXCEPTIONS, 1);
throw new FSError(e);
}
}
@Override
public long skip(long n) throws IOException {
+ ioStatistics.increment(STREAM_READ_SKIP_OPERATIONS, 1);
long value = fis.skip(n);
if (value > 0) {
this.position += value;
+ ioStatistics.increment(STREAM_READ_SKIP_BYTES, value);
}
return value;
}
@@ -207,6 +238,11 @@ public long skip(long n) throws IOException {
public FileDescriptor getFileDescriptor() throws IOException {
return fis.getFD();
}
+
+ @Override
+ public IOStatistics getIOStatistics() {
+ return ioStatistics;
+ }
}
@Override
@@ -231,9 +267,17 @@ public FSDataInputStream open(PathHandle fd, int bufferSize)
/*********************************************************
* For create()'s FSOutputStream.
*********************************************************/
- class LocalFSFileOutputStream extends OutputStream {
+ class LocalFSFileOutputStream extends OutputStream implements
+ IOStatisticsSource {
private FileOutputStream fos;
-
+
+ /**
+ * Minimal set of counters.
+ */
+ private final CounterIOStatistics ioStatistics = counterIOStatistics(
+ STREAM_WRITE_BYTES,
+ STREAM_WRITE_EXCEPTIONS);
+
private LocalFSFileOutputStream(Path f, boolean append,
FsPermission permission) throws IOException {
File file = pathToFile(f);
@@ -273,7 +317,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.increment(STREAM_WRITE_BYTES, len);
} catch (IOException e) { // unexpected exception
+ ioStatistics.increment(STREAM_WRITE_EXCEPTIONS, 1);
throw new FSError(e); // assume native fs error
}
}
@@ -282,10 +328,17 @@ public void write(byte[] b, int off, int len) throws IOException {
public void write(int b) throws IOException {
try {
fos.write(b);
+ ioStatistics.increment(STREAM_WRITE_BYTES, 1);
} catch (IOException e) { // unexpected exception
+ ioStatistics.increment(STREAM_WRITE_EXCEPTIONS, 1);
throw new FSError(e); // assume native fs error
}
}
+
+ @Override
+ public IOStatistics getIOStatistics() {
+ return ioStatistics;
+ }
}
@Override
@@ -318,7 +371,7 @@ 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(
+ return new FSDataOutputStream(new BufferedIOStatisticsOutputStream(
createOutputStreamWithMode(f, false, permission), bufferSize),
statistics);
}
@@ -340,7 +393,7 @@ 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(
+ return new FSDataOutputStream(new BufferedIOStatisticsOutputStream(
createOutputStreamWithMode(f, false, permission), bufferSize),
statistics);
}
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: *
+ * These are low-cost per-instance statistics provided by any Hadoop + * I/O class instance. + *
+ * The statistics MUST BE for the specific instance of the source; + * possibly including aggregate statistics from other objects + * created by that stores. + * For example, the statistics of a filesystem instance must be unique + * to that instant and not shared with any other. + * However, those statistics may also collect and aggregate statistics + * generated in the use of input and output streams created by that + * file system instance. + * + *
+ * The iterator is a possibly empty iterator over all monitored statistics. + *
+ * 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 ""; + } + } + + /** + * On demand stringifier. + *
+ * 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. + * @return an object whose toString() operation returns the current values. + */ + public static Object demandStringify( + @Nullable IOStatisticsSource source) { + return new SourceToString(source); + } + + /** + * On demand stringifier. + *
+ * 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 scan. + * @return an object whose toString() operation returns the current values. + */ + public static Object demandStringify(@Nullable IOStatistics statistics) { + return new StatisticsToString(statistics); + } + + /** + * 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) + : IOStatisticsImplementationUtils.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) + : IOStatisticsImplementationUtils.NULL_SOURCE; + } + } +} 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..260853d1e2f39 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSource.java @@ -0,0 +1,45 @@ +/* + * 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..07f818e44dbe8 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSupport.java @@ -0,0 +1,68 @@ +/* + * 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 org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding; + +/** + * 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
+ * When adding new common statistic name constants, please make them unique.
+ * By convention, they are implicitly unique:
+ *
+ * When adding new common statistic name constants, please make them unique.
+ * By convention, they are implicitly unique:
+ *
+ * It is serializable so that frameworks which can use java serialization
+ * to propagate data (Spark, Flink...) can send the statistics
+ * back.
+ */
+class SnapshotIOStatistics implements IOStatistics, Serializable {
+
+
+ private static final long serialVersionUID = -1762522703841538084L;
+
+ /**
+ * Treemaps sort their insertions so the iterator is ordered.
+ * They are also serializable.
+ */
+ private final TreeMap
+ * This package defines two interfaces:
+ *
+ * {@link org.apache.hadoop.fs.statistics.IOStatisticsSource}:
+ * a source of statistic data, which can be retrieved
+ * through a call to
+ * {@link org.apache.hadoop.fs.statistics.IOStatisticsSource#getIOStatistics()} .
+ *
+ * {@link org.apache.hadoop.fs.statistics.IOStatistics} the statistics retrieved
+ * from a statistics source.
+ *
+ * The retrieved statistics may be an immutable snapshot -in which case to get
+ * updated statistics another call to
+ * {@link org.apache.hadoop.fs.statistics.IOStatisticsSource#getIOStatistics()}
+ * must be made. Or they may be dynamic -in which case every time a specific
+ * statistic is retrieved, the latest version is returned. Callers should assume
+ * that if a statistics instance is dynamic, there is no atomicity when querying
+ * multiple statistics. If the statistics source was a closeable object (e.g. a
+ * stream), the statistics MUST remain valid after the stream is closed.
+ *
+ * Use pattern:
+ *
+ * An application probes an object (filesystem, stream etc) for implementation of
+ * {@code IOStatisticsSource}, and, if it is, calls {@code getIOStatistics()}
+ * to get its statistics.
+ * If this is non-null, the client has statistics on the current
+ * state of the statistics.
+ *
+ * The expectation is that a statistics source is dynamic: when a value is
+ * looked up the most recent values are returned.
+ * When iterating through the set, the values of the iterator SHOULD
+ * be frozen at the time the iterator was requested.
+ *
+ * These statistics can be used to: log operations, profile applications,
+ * and make assertions about the state of the output.
+ *
+ * The names of statistics are a matter of choice of the specific source.
+ * However, {@link org.apache.hadoop.fs.statistics.StoreStatisticNames}
+ * contains a
+ * set of names recommended for object store operations.
+ * {@link org.apache.hadoop.fs.statistics.StreamStatisticNames} declares
+ * recommended names for statistics provided forInconsistentS3ClientFactoryInconsistentS3ClientFactory
+ * input and output streams.
+ *
+ * Utility classes are includes to assist use
+ *
+ * Implementors notes:
+ *
+ * Requires both the input and output streams to offer the basic
+ * bytes read/written statistics.
+ *
+ * If the AWS stats are not null then a {@link AwsStatisticsCollector}.
+ * is created to bind to the two.
+ * Important: until this binding works properly across regions,
+ * this should be null.
+ */
@Override
public AmazonS3 createS3Client(URI name,
final String bucket,
final AWSCredentialsProvider credentials,
- final String userAgentSuffix) throws IOException {
+ final String userAgentSuffix,
+ final StatisticsFromAwsSdk statisticsFromAwsSdk) throws IOException {
Configuration conf = getConf();
final ClientConfiguration awsConf = S3AUtils
.createAwsConf(conf, bucket, Constants.AWS_SERVICE_IDENTIFIER_S3);
@@ -72,36 +99,122 @@ public AmazonS3 createS3Client(URI name,
if (!StringUtils.isEmpty(userAgentSuffix)) {
awsConf.setUserAgentSuffix(userAgentSuffix);
}
- return configureAmazonS3Client(
- newAmazonS3Client(credentials, awsConf), conf);
+ // optional metrics
+ RequestMetricCollector metrics = statisticsFromAwsSdk != null
+ ? new AwsStatisticsCollector(statisticsFromAwsSdk)
+ : null;
+
+ return newAmazonS3Client(
+ credentials,
+ awsConf,
+ metrics,
+ conf.getTrimmed(ENDPOINT, ""),
+ conf.getBoolean(PATH_STYLE_ACCESS, false));
}
/**
- * Wrapper around constructor for {@link AmazonS3} client.
+ * Create an {@link AmazonS3} client.
* Override this to provide an extended version of the client
* @param credentials credentials to use
* @param awsConf AWS configuration
- * @return new AmazonS3 client
+ * @param metrics metrics collector or null
+ * @param endpoint endpoint string; may be ""
+ * @param pathStyleAccess enable path style access?
+ * @return new AmazonS3 client
*/
protected AmazonS3 newAmazonS3Client(
- AWSCredentialsProvider credentials, ClientConfiguration awsConf) {
- return new AmazonS3Client(credentials, awsConf);
+ final AWSCredentialsProvider credentials,
+ final ClientConfiguration awsConf,
+ final RequestMetricCollector metrics,
+ final String endpoint,
+ final boolean pathStyleAccess) {
+ if (metrics != null) {
+ LOG.debug("Building S3 client using the SDK builder API");
+ return buildAmazonS3Client(credentials, awsConf, metrics, endpoint,
+ pathStyleAccess);
+ } else {
+ LOG.debug("Building S3 client using the SDK builder API");
+ return classicAmazonS3Client(credentials, awsConf, endpoint,
+ pathStyleAccess);
+ }
}
/**
- * Configure S3 client from the Hadoop configuration.
- *
+ * Use the (newer) Builder SDK to create a an AWS S3 client.
+ *
+ * This has a more complex endpoint configuration in a
+ * way which does not yet work in this code in a way
+ * which doesn't trigger regressions. So it is only used
+ * when SDK metrics are supplied.
+ * @param credentials credentials to use
+ * @param awsConf AWS configuration
+ * @param metrics metrics collector or null
+ * @param endpoint endpoint string; may be ""
+ * @param pathStyleAccess enable path style access?
+ * @return new AmazonS3 client
+ */
+ private AmazonS3 buildAmazonS3Client(
+ final AWSCredentialsProvider credentials,
+ final ClientConfiguration awsConf,
+ final RequestMetricCollector metrics,
+ final String endpoint,
+ final boolean pathStyleAccess) {
+ AmazonS3ClientBuilder b = AmazonS3Client.builder();
+ b.withCredentials(credentials);
+ b.withClientConfiguration(awsConf);
+ b.withPathStyleAccessEnabled(pathStyleAccess);
+ if (metrics != null) {
+ b.withMetricsCollector(metrics);
+ }
+
+ // endpoint set up is a PITA
+ // client.setEndpoint("") is no longer available
+ AwsClientBuilder.EndpointConfiguration epr
+ = createEndpointConfiguration(endpoint, awsConf);
+ if (epr != null) {
+ // an endpoint binding was constructed: use it.
+ b.withEndpointConfiguration(epr);
+ }
+ final AmazonS3 client = b.build();
+ return client;
+ }
+
+ /**
+ * Wrapper around constructor for {@link AmazonS3} client.
+ * Override this to provide an extended version of the client.
+ *
+ * This uses a deprecated constructor -it is currently
+ * the only one which works for us.
+ * @param credentials credentials to use
+ * @param awsConf AWS configuration
+ * @param endpoint endpoint string; may be ""
+ * @param pathStyleAccess enable path style access?
+ * @return new AmazonS3 client
+ */
+ private AmazonS3 classicAmazonS3Client(
+ AWSCredentialsProvider credentials,
+ ClientConfiguration awsConf,
+ final String endpoint,
+ final boolean pathStyleAccess) {
+ final AmazonS3 client = new AmazonS3Client(credentials, awsConf);
+ return configureAmazonS3Client(client, endpoint, pathStyleAccess);
+ }
+
+ /**
+ * Configure classic S3 client.
+ *
* This includes: endpoint, Path Access and possibly other
* options.
*
- * @param conf Hadoop configuration
+ * @param s3 S3 Client.
+ * @param pathStyleAccess enable path style access?
* @return S3 client
* @throws IllegalArgumentException if misconfigured
*/
- private static AmazonS3 configureAmazonS3Client(AmazonS3 s3,
- Configuration conf)
+ protected static AmazonS3 configureAmazonS3Client(AmazonS3 s3,
+ final String endPoint,
+ final boolean pathStyleAccess)
throws IllegalArgumentException {
- String endPoint = conf.getTrimmed(ENDPOINT, "");
if (!endPoint.isEmpty()) {
try {
s3.setEndpoint(endPoint);
@@ -111,7 +224,7 @@ private static AmazonS3 configureAmazonS3Client(AmazonS3 s3,
throw new IllegalArgumentException(msg, e);
}
}
- return applyS3ClientOptions(s3, conf);
+ return applyS3ClientOptions(s3, pathStyleAccess);
}
/**
@@ -119,23 +232,23 @@ private static AmazonS3 configureAmazonS3Client(AmazonS3 s3,
* the Hadoop configuration.
* This is different from the general AWS configuration creation as
* it is unique to S3 connections.
- *
+ *
* The {@link Constants#PATH_STYLE_ACCESS} option enables path-style access
* to S3 buckets if configured. By default, the
* behavior is to use virtual hosted-style access with URIs of the form
* {@code http://bucketname.s3.amazonaws.com}
+ *
* Enabling path-style access and a
* region-specific endpoint switches the behavior to use URIs of the form
* {@code http://s3-eu-west-1.amazonaws.com/bucketname}.
* It is common to use this when connecting to private S3 servers, as it
* avoids the need to play with DNS entries.
* @param s3 S3 client
- * @param conf Hadoop configuration
+ * @param pathStyleAccess enable path style access?
* @return the S3 client
*/
- private static AmazonS3 applyS3ClientOptions(AmazonS3 s3,
- Configuration conf) {
- final boolean pathStyleAccess = conf.getBoolean(PATH_STYLE_ACCESS, false);
+ protected static AmazonS3 applyS3ClientOptions(AmazonS3 s3,
+ final boolean pathStyleAccess) {
if (pathStyleAccess) {
LOG.debug("Enabling path style access!");
s3.setS3ClientOptions(S3ClientOptions.builder()
@@ -144,4 +257,54 @@ private static AmazonS3 applyS3ClientOptions(AmazonS3 s3,
}
return s3;
}
+
+ /**
+ * Given an endpoint string, return an endpoint config, or null, if none
+ * is needed.
+ *
+ * This is a pretty painful piece of code. It is trying to replicate
+ * what AwsClient.setEndpoint() does, because you can't
+ * call that setter on an AwsClient constructed via
+ * the builder, and you can't pass a metrics collector
+ * down except through the builder.
+ *
+ * Note also that AWS signing is a mystery which nobody fully
+ * understands, especially given all problems surface in a
+ * "400 bad request" response, which, like all security systems,
+ * provides minimal diagnostics out of fear of leaking
+ * secrets.
+ *
+ * @param endpoint possibly null endpoint.
+ * @param awsConf config to build the URI from.
+ * @return a configuration for the S3 client builder.
+ */
+ @VisibleForTesting
+ public static AwsClientBuilder.EndpointConfiguration
+ createEndpointConfiguration(
+ final String endpoint, final ClientConfiguration awsConf) {
+ LOG.debug("Creating endpoint configuration for {}", endpoint);
+ if (endpoint == null || endpoint.isEmpty()) {
+ // the default endpoint...we should be using null at this point.
+ LOG.debug("Using default endpoint -no need to generate a configuration");
+ return null;
+ }
+
+ final URI epr = RuntimeHttpUtils.toUri(endpoint, awsConf);
+ LOG.debug("Endpoint URI = {}", epr);
+
+ String region;
+ if (!ServiceUtils.isS3USStandardEndpoint(endpoint)) {
+ LOG.debug("Endpoint {} is not the default; parsing", epr);
+ region = AwsHostNameUtils.parseRegion(
+ epr.getHost(),
+ S3_SERVICE_NAME);
+ } else {
+ // US-east, set region == null.
+ LOG.debug("Endpoint {} is the standard one; declare region as null", epr);
+ region = null;
+ }
+ LOG.debug("Region for endpoint {}, URI {} is determined as {}",
+ endpoint, epr, region);
+ return new AwsClientBuilder.EndpointConfiguration(endpoint, region);
+ }
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java
index 932c472f5bea2..e99ec1ab28f75 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java
@@ -20,6 +20,7 @@
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.metrics.RequestMetricCollector;
import com.amazonaws.services.s3.AmazonS3;
import org.apache.hadoop.classification.InterfaceAudience;
@@ -40,12 +41,21 @@ public class InconsistentS3ClientFactory extends DefaultS3ClientFactory {
* Logs a warning that this is being done.
* @param credentials credentials to use
* @param awsConf AWS configuration
+ * @param metrics
+ * @param endpoint
+ * @param pathStyleAccess
* @return an inconsistent client.
*/
@Override
protected AmazonS3 newAmazonS3Client(AWSCredentialsProvider credentials,
- ClientConfiguration awsConf) {
+ ClientConfiguration awsConf,
+ final RequestMetricCollector metrics,
+ final String endpoint,
+ final boolean pathStyleAccess) {
LOG.warn("** FAILURE INJECTION ENABLED. Do not run in production! **");
- return new InconsistentAmazonS3Client(credentials, awsConf, getConf());
+ InconsistentAmazonS3Client s3
+ = new InconsistentAmazonS3Client(credentials, awsConf, getConf());
+ configureAmazonS3Client(s3, endpoint, pathStyleAccess);
+ return s3;
}
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
index 15f69bde3f119..d9b774a11f1aa 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
@@ -49,10 +49,15 @@
import org.apache.hadoop.fs.StreamCapabilities;
import org.apache.hadoop.fs.s3a.commit.CommitConstants;
import org.apache.hadoop.fs.s3a.commit.PutTracker;
+import org.apache.hadoop.fs.s3a.impl.statistics.BlockOutputStreamStatistics;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsLogging;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
import org.apache.hadoop.util.Progressable;
import static org.apache.hadoop.fs.s3a.S3AUtils.*;
import static org.apache.hadoop.fs.s3a.Statistic.*;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.emptyStatistics;
import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
/**
@@ -67,7 +72,7 @@
@InterfaceAudience.Private
@InterfaceStability.Unstable
class S3ABlockOutputStream extends OutputStream implements
- StreamCapabilities {
+ StreamCapabilities, IOStatisticsSource {
private static final Logger LOG =
LoggerFactory.getLogger(S3ABlockOutputStream.class);
@@ -81,6 +86,9 @@ class S3ABlockOutputStream extends OutputStream implements
/** Size of all blocks. */
private final int blockSize;
+ /** IO Statistics. */
+ private final IOStatistics iostatistics;
+
/** Total bytes for uploads submitted so far. */
private long bytesSubmitted;
@@ -109,7 +117,7 @@ class S3ABlockOutputStream extends OutputStream implements
private long blockCount = 0;
/** Statistics to build up. */
- private final S3AInstrumentation.OutputStreamStatistics statistics;
+ private final BlockOutputStreamStatistics statistics;
/**
* Write operation helper; encapsulation of the filesystem operations.
@@ -146,7 +154,7 @@ class S3ABlockOutputStream extends OutputStream implements
Progressable progress,
long blockSize,
S3ADataBlocks.BlockFactory blockFactory,
- S3AInstrumentation.OutputStreamStatistics statistics,
+ BlockOutputStreamStatistics statistics,
WriteOperationHelper writeOperationHelper,
PutTracker putTracker)
throws IOException {
@@ -155,6 +163,10 @@ class S3ABlockOutputStream extends OutputStream implements
this.blockFactory = blockFactory;
this.blockSize = (int) blockSize;
this.statistics = statistics;
+ // test instantiations may not provide statistics;
+ iostatistics = statistics != null
+ ? statistics.getIOStatistics()
+ : emptyStatistics();
this.writeOperationHelper = writeOperationHelper;
this.putTracker = putTracker;
Preconditions.checkArgument(blockSize >= Constants.MULTIPART_MIN_SIZE,
@@ -282,6 +294,7 @@ public synchronized void write(byte[] source, int offset, int len)
if (len == 0) {
return;
}
+ statistics.writeBytes(len);
S3ADataBlocks.DataBlock block = createBlockIfNeeded();
int written = block.write(source, offset, len);
int remainingCapacity = block.remainingCapacity();
@@ -473,6 +486,7 @@ public String toString() {
if (block != null) {
sb.append(", activeBlock=").append(block);
}
+ sb.append(IOStatisticsLogging.ioStatisticsSourceToString(this));
sb.append('}');
return sb.toString();
}
@@ -493,7 +507,7 @@ private long now() {
* Get the statistics for this stream.
* @return stream statistics
*/
- S3AInstrumentation.OutputStreamStatistics getStatistics() {
+ BlockOutputStreamStatistics getStatistics() {
return statistics;
}
@@ -525,6 +539,11 @@ public boolean hasCapability(String capability) {
}
}
+ @Override
+ public IOStatistics getIOStatistics() {
+ return iostatistics;
+ }
+
/**
* Multiple partition upload.
*/
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java
index 156defb7ca031..9fbf0008cf397 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java
@@ -37,6 +37,7 @@
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.FSExceptionMessages;
+import org.apache.hadoop.fs.s3a.impl.statistics.BlockOutputStreamStatistics;
import org.apache.hadoop.util.DirectBufferPool;
import static org.apache.hadoop.fs.s3a.S3ADataBlocks.DataBlock.DestState.*;
@@ -180,7 +181,7 @@ protected BlockFactory(S3AFileSystem owner) {
* @return a new block.
*/
abstract DataBlock create(long index, int limit,
- S3AInstrumentation.OutputStreamStatistics statistics)
+ BlockOutputStreamStatistics statistics)
throws IOException;
/**
@@ -210,10 +211,10 @@ enum DestState {Writing, Upload, Closed}
private volatile DestState state = Writing;
protected final long index;
- protected final S3AInstrumentation.OutputStreamStatistics statistics;
+ private final BlockOutputStreamStatistics statistics;
protected DataBlock(long index,
- S3AInstrumentation.OutputStreamStatistics statistics) {
+ BlockOutputStreamStatistics statistics) {
this.index = index;
this.statistics = statistics;
}
@@ -372,6 +373,10 @@ protected void blockReleased() {
statistics.blockReleased();
}
}
+
+ protected BlockOutputStreamStatistics getStatistics() {
+ return statistics;
+ }
}
// ====================================================================
@@ -387,7 +392,7 @@ static class ArrayBlockFactory extends BlockFactory {
@Override
DataBlock create(long index, int limit,
- S3AInstrumentation.OutputStreamStatistics statistics)
+ BlockOutputStreamStatistics statistics)
throws IOException {
return new ByteArrayBlock(0, limit, statistics);
}
@@ -432,7 +437,7 @@ static class ByteArrayBlock extends DataBlock {
ByteArrayBlock(long index,
int limit,
- S3AInstrumentation.OutputStreamStatistics statistics) {
+ BlockOutputStreamStatistics statistics) {
super(index, statistics);
this.limit = limit;
buffer = new S3AByteArrayOutputStream(limit);
@@ -510,7 +515,7 @@ static class ByteBufferBlockFactory extends BlockFactory {
@Override
ByteBufferBlock create(long index, int limit,
- S3AInstrumentation.OutputStreamStatistics statistics)
+ BlockOutputStreamStatistics statistics)
throws IOException {
return new ByteBufferBlock(index, limit, statistics);
}
@@ -560,7 +565,7 @@ class ByteBufferBlock extends DataBlock {
*/
ByteBufferBlock(long index,
int bufferSize,
- S3AInstrumentation.OutputStreamStatistics statistics) {
+ BlockOutputStreamStatistics statistics) {
super(index, statistics);
this.bufferSize = bufferSize;
blockBuffer = requestBuffer(bufferSize);
@@ -805,7 +810,7 @@ static class DiskBlockFactory extends BlockFactory {
@Override
DataBlock create(long index,
int limit,
- S3AInstrumentation.OutputStreamStatistics statistics)
+ BlockOutputStreamStatistics statistics)
throws IOException {
File destFile = getOwner()
.createTmpFileForWrite(String.format("s3ablock-%04d-", index),
@@ -829,7 +834,7 @@ static class DiskBlock extends DataBlock {
DiskBlock(File bufferFile,
int limit,
long index,
- S3AInstrumentation.OutputStreamStatistics statistics)
+ BlockOutputStreamStatistics statistics)
throws FileNotFoundException {
super(index, statistics);
this.limit = limit;
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
index 6d2b3a84ca702..a1b2754a70e54 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
@@ -110,8 +110,15 @@
import org.apache.hadoop.fs.s3a.impl.RenameOperation;
import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum;
import org.apache.hadoop.fs.s3a.impl.StoreContext;
+import org.apache.hadoop.fs.s3a.impl.statistics.CommitterStatistics;
+import org.apache.hadoop.fs.s3a.impl.statistics.BondedS3AStatisticsContext;
+import org.apache.hadoop.fs.s3a.impl.statistics.S3AStatisticsContext;
+import org.apache.hadoop.fs.s3a.impl.statistics.StatisticsFromAwsSdk;
import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
import org.apache.hadoop.fs.s3a.select.InternalSelectConstants;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+import org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.token.TokenIdentifier;
@@ -174,6 +181,7 @@
import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit;
import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletionIgnoringExceptions;
import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isUnknownBucket;
+import static org.apache.hadoop.fs.s3a.impl.InternalConstants.AWS_SDK_METRICS_ENABLED;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_404;
import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.fixBucketRegion;
import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.logDnsLookup;
@@ -195,7 +203,7 @@
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class S3AFileSystem extends FileSystem implements StreamCapabilities,
- AWSPolicyProvider, DelegationTokenProvider {
+ AWSPolicyProvider, DelegationTokenProvider, IOStatisticsSource {
/**
* Default blocksize as used in blocksize and FS status queries.
*/
@@ -250,6 +258,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
private S3AInstrumentation instrumentation;
private final S3AStorageStatistics storageStatistics =
createStorageStatistics();
+
+ private S3AStatisticsContext statisticsContext;
+
private long readAhead;
private S3AInputPolicy inputPolicy;
private ChangeDetectionPolicy changeDetectionPolicy;
@@ -314,7 +325,8 @@ private static void addDeprecatedKeys() {
* bucket-specific options are patched over the base ones before any use is
* made of the config.
*/
- public void initialize(URI name, Configuration originalConf)
+ public void
+ initialize(URI name, Configuration originalConf)
throws IOException {
// get the host; this is guaranteed to be non-null, non-empty
bucket = name.getHost();
@@ -343,6 +355,7 @@ public void initialize(URI name, Configuration originalConf)
invoker = new Invoker(new S3ARetryPolicy(getConf()), onRetry);
instrumentation = new S3AInstrumentation(uri);
+ initializeStatisticsBinding();
// Username is the current user at the time the FS was instantiated.
owner = UserGroupInformation.getCurrentUser();
@@ -352,7 +365,8 @@ public void initialize(URI name, Configuration originalConf)
s3guardInvoker = new Invoker(new S3GuardExistsRetryPolicy(getConf()),
onRetry);
- writeHelper = new WriteOperationHelper(this, getConf());
+ writeHelper = new WriteOperationHelper(this, getConf(),
+ statisticsContext);
failOnMetadataWriteError = conf.getBoolean(FAIL_ON_METADATA_WRITE_ERROR,
FAIL_ON_METADATA_WRITE_ERROR_DEFAULT);
@@ -502,6 +516,31 @@ private void doBucketProbing() throws IOException {
}
}
+ /**
+ * Initialize the statistics binding.
+ * This is done by creating an {@code IntegratedS3AStatisticsContext}
+ * with callbacks to get the FS's instrumentation and FileSystem.statistics
+ * field; the latter may change after {@link #initialize(URI, Configuration)},
+ * so needs to be dynamically adapted.
+ * Protected so that (mock) subclasses can replace it with a
+ * different statistics binding, if desired.
+ */
+ protected void initializeStatisticsBinding() {
+ statisticsContext = new BondedS3AStatisticsContext(
+ new BondedS3AStatisticsContext.S3AFSStatisticsSource() {
+
+ @Override
+ public S3AInstrumentation getInstrumentation() {
+ return S3AFileSystem.this.getInstrumentation();
+ }
+
+ @Override
+ public Statistics getInstanceStatistics() {
+ return S3AFileSystem.this.statistics;
+ }
+ });
+ }
+
/**
* Initialize the thread pool.
* This must be re-invoked after replacing the S3Client during test
@@ -581,6 +620,7 @@ protected void verifyBucketExistsV2()
* Get S3A Instrumentation. For test purposes.
* @return this instance's instrumentation.
*/
+ @VisibleForTesting
public S3AInstrumentation getInstrumentation() {
return instrumentation;
}
@@ -640,8 +680,16 @@ private void bindAWSClient(URI name, boolean dtEnabled) throws IOException {
S3_CLIENT_FACTORY_IMPL, DEFAULT_S3_CLIENT_FACTORY_IMPL,
S3ClientFactory.class);
+ StatisticsFromAwsSdk awsStats = null;
+// TODO: when the S3 client building code works with different regions,
+// then non-null stats can be passed in here.
+ if (AWS_SDK_METRICS_ENABLED) {
+ awsStats = statisticsContext.newStatisticsFromAwsSdk();
+ }
+
s3 = ReflectionUtils.newInstance(s3ClientFactoryClass, conf)
- .createS3Client(getUri(), bucket, credentials, uaSuffix);
+ .createS3Client(getUri(), bucket, credentials, uaSuffix,
+ awsStats);
}
/**
@@ -1139,7 +1187,7 @@ private S3AReadOpContext createReadContext(
invoker,
s3guardInvoker,
statistics,
- instrumentation,
+ statisticsContext,
fileStatus,
seekPolicy,
changePolicy,
@@ -1244,7 +1292,7 @@ public FSDataOutputStream create(Path f, FsPermission permission,
progress,
partSize,
blockFactory,
- instrumentation.newOutputStreamStatistics(statistics),
+ statisticsContext.newOutputStreamStatistics(),
getWriteOperationHelper(),
putTracker),
null);
@@ -1706,7 +1754,7 @@ protected void incrementStatistic(Statistic statistic) {
* @param count the count to increment
*/
protected void incrementStatistic(Statistic statistic, long count) {
- instrumentation.incrementCounter(statistic, count);
+ statisticsContext.incrementCounter(statistic, count);
storageStatistics.incrementCounter(statistic, count);
}
@@ -1716,7 +1764,7 @@ protected void incrementStatistic(Statistic statistic, long count) {
* @param count the count to decrement
*/
protected void decrementGauge(Statistic statistic, long count) {
- instrumentation.decrementGauge(statistic, count);
+ statisticsContext.decrementGauge(statistic, count);
}
/**
@@ -1725,7 +1773,7 @@ protected void decrementGauge(Statistic statistic, long count) {
* @param count the count to increment
*/
protected void incrementGauge(Statistic statistic, long count) {
- instrumentation.incrementGauge(statistic, count);
+ statisticsContext.incrementGauge(statistic, count);
}
/**
@@ -1738,6 +1786,7 @@ public void operationRetried(Exception ex) {
if (isThrottleException(ex)) {
operationThrottled(false);
} else {
+ incrementStatistic(STORE_IO_RETRY);
incrementStatistic(IGNORED_ERRORS);
}
}
@@ -1789,11 +1838,11 @@ private void operationThrottled(boolean metastore) {
LOG.debug("Request throttled on {}", metastore ? "S3": "DynamoDB");
if (metastore) {
incrementStatistic(S3GUARD_METADATASTORE_THROTTLED);
- instrumentation.addValueToQuantiles(S3GUARD_METADATASTORE_THROTTLE_RATE,
+ statisticsContext.addValueToQuantiles(S3GUARD_METADATASTORE_THROTTLE_RATE,
1);
} else {
incrementStatistic(STORE_IO_THROTTLED);
- instrumentation.addValueToQuantiles(STORE_IO_THROTTLE_RATE, 1);
+ statisticsContext.addValueToQuantiles(STORE_IO_THROTTLE_RATE, 1);
}
}
@@ -1806,6 +1855,16 @@ public S3AStorageStatistics getStorageStatistics() {
return storageStatistics;
}
+ /**
+ * Get this filesystem's storage statistics as IO Statistics.
+ * @return statistics
+ */
+ @Override
+ public IOStatistics getIOStatistics() {
+ return IOStatisticsBinding.fromStorageStatistics(
+ storageStatistics);
+ }
+
/**
* Request object metadata; increments counters in the process.
* Retry policy: retry untranslated.
@@ -3441,8 +3500,9 @@ private CopyResult copyFile(String srcKey, String dstKey, long size,
ChangeTracker changeTracker = new ChangeTracker(
keyToQualifiedPath(srcKey).toString(),
changeDetectionPolicy,
- readContext.instrumentation.newInputStreamStatistics()
- .getVersionMismatchCounter(),
+ readContext.getS3AStatisticsContext()
+ .newInputStreamStatistics()
+ .getChangeTrackerStatistics(),
srcAttributes);
String action = "copyFile(" + srcKey + ", " + dstKey + ")";
@@ -4448,8 +4508,8 @@ void abortMultipartUpload(MultipartUpload upload) {
* Create a new instance of the committer statistics.
* @return a new committer statistics instance
*/
- public S3AInstrumentation.CommitterStatistics newCommitterStatistics() {
- return instrumentation.newCommitterStatistics();
+ public CommitterStatistics newCommitterStatistics() {
+ return statisticsContext.newCommitterStatistics();
}
@SuppressWarnings("deprecation")
@@ -4563,8 +4623,9 @@ private FSDataInputStream select(final Path source,
ChangeTracker changeTracker =
new ChangeTracker(uri.toString(),
changeDetectionPolicy,
- readContext.instrumentation.newInputStreamStatistics()
- .getVersionMismatchCounter(),
+ readContext.getS3AStatisticsContext()
+ .newInputStreamStatistics()
+ .getChangeTrackerStatistics(),
objectAttributes);
// will retry internally if wrong version detected
@@ -4720,7 +4781,7 @@ public StoreContext createStoreContext() {
boundedThreadPool,
executorCapacity,
invoker,
- getInstrumentation(),
+ statisticsContext,
getStorageStatistics(),
getInputPolicy(),
changeDetectionPolicy,
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
index 9c8b9ae7a156e..956eeb4608f74 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
@@ -33,10 +33,13 @@
import org.apache.hadoop.fs.CanSetReadahead;
import org.apache.hadoop.fs.CanUnbuffer;
import org.apache.hadoop.fs.FSExceptionMessages;
-import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.fs.s3a.impl.statistics.S3AInputStreamStatistics;
+import org.apache.hadoop.fs.s3a.impl.ChangeTracker;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
import org.apache.hadoop.fs.PathIOException;
import org.apache.hadoop.fs.StreamCapabilities;
-import org.apache.hadoop.fs.s3a.impl.ChangeTracker;
+import org.apache.hadoop.fs.FSInputStream;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -68,7 +71,7 @@
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class S3AInputStream extends FSInputStream implements CanSetReadahead,
- CanUnbuffer, StreamCapabilities {
+ CanUnbuffer, StreamCapabilities, IOStatisticsSource {
public static final String E_NEGATIVE_READAHEAD_VALUE
= "Negative readahead value";
@@ -97,7 +100,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
private final String uri;
private static final Logger LOG =
LoggerFactory.getLogger(S3AInputStream.class);
- private final S3AInstrumentation.InputStreamStatistics streamStatistics;
+ private final S3AInputStreamStatistics streamStatistics;
private S3AEncryptionMethods serverSideEncryptionAlgorithm;
private String serverSideEncryptionKey;
private S3AInputPolicy inputPolicy;
@@ -123,6 +126,11 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
/** change tracker. */
private final ChangeTracker changeTracker;
+ /**
+ * IOStatistics report.
+ */
+ private final IOStatistics ioStatistics;
+
/**
* Create the stream.
* This does not attempt to open it; that is only done on the first
@@ -146,13 +154,15 @@ public S3AInputStream(S3AReadOpContext ctx,
this.contentLength = l;
this.client = client;
this.uri = "s3a://" + this.bucket + "/" + this.key;
- this.streamStatistics = ctx.instrumentation.newInputStreamStatistics();
+ this.streamStatistics = ctx.getS3AStatisticsContext()
+ .newInputStreamStatistics();
+ this.ioStatistics = streamStatistics.getIOStatistics();
this.serverSideEncryptionAlgorithm =
s3Attributes.getServerSideEncryptionAlgorithm();
this.serverSideEncryptionKey = s3Attributes.getServerSideEncryptionKey();
this.changeTracker = new ChangeTracker(uri,
ctx.getChangeDetectionPolicy(),
- streamStatistics.getVersionMismatchCounter(),
+ streamStatistics.getChangeTrackerStatistics(),
s3Attributes);
setInputPolicy(ctx.getInputPolicy());
setReadahead(ctx.getReadahead());
@@ -290,8 +300,6 @@ private void seekInStream(long targetPos, long length) throws IOException {
long skipped = wrappedStream.skip(diff);
if (skipped > 0) {
pos += skipped;
- // as these bytes have been read, they are included in the counter
- incrementBytesRead(diff);
}
if (pos == targetPos) {
@@ -348,7 +356,7 @@ private void lazySeek(long targetPos, long len) throws IOException {
// open. After that, an exception generally means the file has changed
// and there is no point retrying anymore.
Invoker invoker = context.getReadInvoker();
- invoker.maybeRetry(streamStatistics.openOperations == 0,
+ invoker.maybeRetry(streamStatistics.getOpenOperations() == 0,
"lazySeek", pathStr, true,
() -> {
//For lazy seek
@@ -747,7 +755,7 @@ public void readFully(long position, byte[] buffer, int offset, int length)
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
- public S3AInstrumentation.InputStreamStatistics getS3AStreamStatistics() {
+ public S3AInputStreamStatistics getS3AStreamStatistics() {
return streamStatistics;
}
@@ -850,4 +858,9 @@ public boolean hasCapability(String capability) {
boolean isObjectStreamOpen() {
return wrappedStream != null;
}
+
+ @Override
+ public IOStatistics getIOStatistics() {
+ return ioStatistics;
+ }
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
index b9918b5098946..040895a40c9e5 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
@@ -18,14 +18,31 @@
package org.apache.hadoop.fs.s3a;
+import javax.annotation.Nullable;
+
import com.google.common.annotations.VisibleForTesting;
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.FileSystem.Statistics;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.s3a.impl.statistics.ChangeTrackerStatistics;
+import org.apache.hadoop.fs.s3a.impl.statistics.CommitterStatistics;
+import org.apache.hadoop.fs.s3a.impl.statistics.CountersAndGauges;
+import org.apache.hadoop.fs.s3a.impl.statistics.CountingChangeTracker;
+import org.apache.hadoop.fs.s3a.impl.statistics.DelegationTokenStatistics;
+import org.apache.hadoop.fs.s3a.impl.statistics.S3AInputStreamStatistics;
+import org.apache.hadoop.fs.s3a.impl.statistics.BlockOutputStreamStatistics;
+import org.apache.hadoop.fs.s3a.impl.statistics.StatisticsFromAwsSdk;
import org.apache.hadoop.fs.s3a.s3guard.MetastoreInstrumentation;
+import org.apache.hadoop.fs.statistics.IOStatisticEntry;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsLogging;
+import org.apache.hadoop.fs.statistics.IOStatisticsSupport;
+import org.apache.hadoop.fs.statistics.StreamStatisticNames;
+import org.apache.hadoop.fs.statistics.impl.CounterIOStatistics;
+import org.apache.hadoop.fs.statistics.impl.DynamicIOStatisticsBuilder;
import org.apache.hadoop.metrics2.AbstractMetric;
import org.apache.hadoop.metrics2.MetricStringBuilder;
import org.apache.hadoop.metrics2.MetricsCollector;
@@ -43,18 +60,37 @@
import java.io.Closeable;
import java.net.URI;
+import java.time.Duration;
import java.util.HashMap;
import java.util.Map;
import java.util.UUID;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
+import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_OPENED;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.counterIOStatistics;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.dynamicIOStatistics;
import static org.apache.hadoop.fs.s3a.Statistic.*;
/**
* Instrumentation of S3a.
- * Derived from the {@code AzureFileSystemInstrumentation}.
- *
+ *
+ * Some of these stats are propagated to any passed in
+ * {@link FileSystem.Statistics} instance; this is only done
+ * in close() for better cross-thread accounting.
*/
- @InterfaceAudience.Private
- @InterfaceStability.Unstable
- public final class OutputStreamStatistics implements Closeable {
+ private final class BlockOutputStreamStatisticsImpl implements
+ BlockOutputStreamStatistics {
private final AtomicLong blocksSubmitted = new AtomicLong(0);
private final AtomicLong blocksInQueue = new AtomicLong(0);
private final AtomicLong blocksActive = new AtomicLong(0);
@@ -950,6 +1121,7 @@ public final class OutputStreamStatistics implements Closeable {
private final AtomicLong blockUploadsFailed = new AtomicLong(0);
private final AtomicLong bytesPendingUpload = new AtomicLong(0);
+ private final AtomicLong bytesWritten = new AtomicLong(0);
private final AtomicLong bytesUploaded = new AtomicLong(0);
private final AtomicLong transferDuration = new AtomicLong(0);
private final AtomicLong queueDuration = new AtomicLong(0);
@@ -957,30 +1129,34 @@ public final class OutputStreamStatistics implements Closeable {
private final AtomicInteger blocksAllocated = new AtomicInteger(0);
private final AtomicInteger blocksReleased = new AtomicInteger(0);
- private Statistics statistics;
+ private final FileSystem.Statistics filesystemStatistics;
- public OutputStreamStatistics(Statistics statistics){
- this.statistics = statistics;
+ private BlockOutputStreamStatisticsImpl(
+ @Nullable FileSystem.Statistics filesystemStatistics) {
+ this.filesystemStatistics = filesystemStatistics;
}
/**
* A block has been allocated.
*/
- void blockAllocated() {
+ @Override
+ public void blockAllocated() {
blocksAllocated.incrementAndGet();
}
/**
* A block has been released.
*/
- void blockReleased() {
+ @Override
+ public void blockReleased() {
blocksReleased.incrementAndGet();
}
/**
* Block is queued for upload.
*/
- void blockUploadQueued(int blockSize) {
+ @Override
+ public void blockUploadQueued(int blockSize) {
blocksSubmitted.incrementAndGet();
blocksInQueue.incrementAndGet();
bytesPendingUpload.addAndGet(blockSize);
@@ -989,7 +1165,8 @@ void blockUploadQueued(int blockSize) {
}
/** Queued block has been scheduled for upload. */
- void blockUploadStarted(long duration, int blockSize) {
+ @Override
+ public void blockUploadStarted(long duration, int blockSize) {
queueDuration.addAndGet(duration);
blocksInQueue.decrementAndGet();
blocksActive.incrementAndGet();
@@ -998,8 +1175,9 @@ void blockUploadStarted(long duration, int blockSize) {
}
/** A block upload has completed. */
- void blockUploadCompleted(long duration, int blockSize) {
- this.transferDuration.addAndGet(duration);
+ @Override
+ public void blockUploadCompleted(long duration, int blockSize) {
+ transferDuration.addAndGet(duration);
incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_ACTIVE, -1);
blocksActive.decrementAndGet();
blockUploadsCompleted.incrementAndGet();
@@ -1010,14 +1188,15 @@ void blockUploadCompleted(long duration, int blockSize) {
* A final transfer completed event is still expected, so this
* does not decrement the active block counter.
*/
- void blockUploadFailed(long duration, int blockSize) {
+ @Override
+ public void blockUploadFailed(long duration, int blockSize) {
blockUploadsFailed.incrementAndGet();
}
/** Intermediate report of bytes uploaded. */
- void bytesTransferred(long byteCount) {
+ @Override
+ public void bytesTransferred(long byteCount) {
bytesUploaded.addAndGet(byteCount);
- statistics.incrementBytesWritten(byteCount);
bytesPendingUpload.addAndGet(-byteCount);
incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING, -byteCount);
}
@@ -1026,7 +1205,8 @@ void bytesTransferred(long byteCount) {
* Note exception in a multipart complete.
* @param count count of exceptions
*/
- void exceptionInMultipartComplete(int count) {
+ @Override
+ public void exceptionInMultipartComplete(int count) {
if (count > 0) {
exceptionsInMultipartFinalize.addAndGet(count);
}
@@ -1035,7 +1215,8 @@ void exceptionInMultipartComplete(int count) {
/**
* Note an exception in a multipart abort.
*/
- void exceptionInMultipartAbort() {
+ @Override
+ public void exceptionInMultipartAbort() {
exceptionsInMultipartFinalize.incrementAndGet();
}
@@ -1043,6 +1224,7 @@ void exceptionInMultipartAbort() {
* Get the number of bytes pending upload.
* @return the number of bytes in the pending upload state.
*/
+ @Override
public long getBytesPendingUpload() {
return bytesPendingUpload.get();
}
@@ -1052,6 +1234,7 @@ public long getBytesPendingUpload() {
* to be called at the end of the write.
* @param size size in bytes
*/
+ @Override
public void commitUploaded(long size) {
incrementCounter(COMMITTER_BYTES_UPLOADED, size);
}
@@ -1067,6 +1250,12 @@ public void close() {
" as pending upload in {}", this);
}
mergeOutputStreamStatistics(this);
+ // and patch the FS statistics.
+ // provided the stream is closed in the worker thread, this will
+ // ensure that the thread-specific worker stats are updated.
+ if (filesystemStatistics != null) {
+ filesystemStatistics.incrementBytesWritten(bytesUploaded.get());
+ }
}
long averageQueueTime() {
@@ -1084,23 +1273,43 @@ long totalUploadDuration() {
return queueDuration.get() + transferDuration.get();
}
- public int blocksAllocated() {
+ @Override
+ public int getBlocksAllocated() {
return blocksAllocated.get();
}
- public int blocksReleased() {
+ @Override
+ public int getBlocksReleased() {
return blocksReleased.get();
}
/**
- * Get counters of blocks actively allocated; my be inaccurate
+ * Get counters of blocks actively allocated; may be inaccurate
* if the numbers change during the (non-synchronized) calculation.
* @return the number of actively allocated blocks.
*/
- public int blocksActivelyAllocated() {
+ @Override
+ public int getBlocksActivelyAllocated() {
return blocksAllocated.get() - blocksReleased.get();
}
+ /**
+ * Record bytes written.
+ * @param count number of bytes
+ */
+ @Override
+ public void writeBytes(long count) {
+ bytesWritten.addAndGet(count);
+ }
+
+ /**
+ * Get the current count of bytes written.
+ * @return the counter value.
+ */
+ @Override
+ public long getBytesWritten() {
+ return bytesWritten.get();
+ }
@Override
public String toString() {
@@ -1113,11 +1322,13 @@ public String toString() {
sb.append(", blockUploadsFailed=").append(blockUploadsFailed);
sb.append(", bytesPendingUpload=").append(bytesPendingUpload);
sb.append(", bytesUploaded=").append(bytesUploaded);
+ sb.append(", bytesWritten=").append(bytesWritten);
sb.append(", blocksAllocated=").append(blocksAllocated);
sb.append(", blocksReleased=").append(blocksReleased);
- sb.append(", blocksActivelyAllocated=").append(blocksActivelyAllocated());
- sb.append(", exceptionsInMultipartFinalize=").append(
- exceptionsInMultipartFinalize);
+ sb.append(", blocksActivelyAllocated=")
+ .append(getBlocksActivelyAllocated());
+ sb.append(", exceptionsInMultipartFinalize=")
+ .append(exceptionsInMultipartFinalize);
sb.append(", transferDuration=").append(transferDuration).append(" ms");
sb.append(", queueDuration=").append(queueDuration).append(" ms");
sb.append(", averageQueueTime=").append(averageQueueTime()).append(" ms");
@@ -1128,6 +1339,24 @@ public String toString() {
sb.append('}');
return sb.toString();
}
+
+ /**
+ * Create an IOStatistics instance which is dynamically updated.
+ * @return statistics
+ */
+ @Override
+ public IOStatistics getIOStatistics() {
+ DynamicIOStatisticsBuilder builder = dynamicIOStatistics();
+
+ builder
+ .withAtomicLong(StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS,
+ blocksSubmitted)
+ .withAtomicLong(StreamStatisticNames.STREAM_WRITE_BYTES,
+ bytesWritten)
+ .withAtomicLong(StreamStatisticNames.STREAM_WRITE_EXCEPTIONS,
+ blockUploadsFailed);
+ return builder.build();
+ }
}
/**
@@ -1192,13 +1421,12 @@ public void entryAdded(final long durationNanos) {
}
/**
- * Instrumentation exported to S3Guard Committers.
+ * Instrumentation exported to S3A Committers.
*/
- @InterfaceAudience.Private
- @InterfaceStability.Unstable
- public final class CommitterStatistics {
+ private final class CommitterStatisticsImpl implements CommitterStatistics {
/** A commit has been created. */
+ @Override
public void commitCreated() {
incrementCounter(COMMITTER_COMMITS_CREATED, 1);
}
@@ -1207,6 +1435,7 @@ public void commitCreated() {
* Data has been uploaded to be committed in a subsequent operation.
* @param size size in bytes
*/
+ @Override
public void commitUploaded(long size) {
incrementCounter(COMMITTER_BYTES_UPLOADED, size);
}
@@ -1215,24 +1444,29 @@ public void commitUploaded(long size) {
* A commit has been completed.
* @param size size in bytes
*/
+ @Override
public void commitCompleted(long size) {
incrementCounter(COMMITTER_COMMITS_COMPLETED, 1);
incrementCounter(COMMITTER_BYTES_COMMITTED, size);
}
/** A commit has been aborted. */
+ @Override
public void commitAborted() {
incrementCounter(COMMITTER_COMMITS_ABORTED, 1);
}
+ @Override
public void commitReverted() {
incrementCounter(COMMITTER_COMMITS_REVERTED, 1);
}
+ @Override
public void commitFailed() {
incrementCounter(COMMITTER_COMMITS_FAILED, 1);
}
+ @Override
public void taskCompleted(boolean success) {
incrementCounter(
success ? COMMITTER_TASKS_SUCCEEDED
@@ -1240,6 +1474,7 @@ public void taskCompleted(boolean success) {
1);
}
+ @Override
public void jobCompleted(boolean success) {
incrementCounter(
success ? COMMITTER_JOBS_SUCCEEDED
@@ -1253,20 +1488,20 @@ public void jobCompleted(boolean success) {
* @return an instance of delegation token statistics
*/
public DelegationTokenStatistics newDelegationTokenStatistics() {
- return new DelegationTokenStatistics();
+ return new DelegationTokenStatisticsImpl();
}
/**
* Instrumentation exported to S3A Delegation Token support.
*/
- @InterfaceAudience.Private
- @InterfaceStability.Unstable
- public final class DelegationTokenStatistics {
+ private final class DelegationTokenStatisticsImpl implements
+ DelegationTokenStatistics {
- private DelegationTokenStatistics() {
+ private DelegationTokenStatisticsImpl() {
}
/** A token has been issued. */
+ @Override
public void tokenIssued() {
incrementCounter(DELEGATION_TOKENS_ISSUED, 1);
}
@@ -1366,4 +1601,67 @@ public Map
+ * See {@code com.facebook.presto.hive.s3.PrestoS3FileSystemMetricCollector}
+ * for the inspiration for this.
+ *
+ * See {@code com.amazonaws.util.AWSRequestMetrics} for metric names.
+ */
+public class AwsStatisticsCollector extends RequestMetricCollector {
+
+ /**
+ * final destination of updates.
+ */
+ private final StatisticsFromAwsSdk collector;
+
+ /**
+ * Instantiate.
+ * @param collector final destination of updates
+ */
+ public AwsStatisticsCollector(final StatisticsFromAwsSdk collector) {
+ this.collector = collector;
+ }
+
+ /**
+ * This is the callback from the AWS SDK where metrics
+ * can be collected.
+ * @param request AWS request
+ * @param response AWS response
+ */
+ @Override
+ public void collectMetrics(
+ final Request> request,
+ final Response> response) {
+
+ TimingInfo timingInfo = request.getAWSRequestMetrics().getTimingInfo();
+
+ counter(timingInfo, HttpClientRetryCount.name(),
+ collector::updateAwsRetryCount);
+ counter(timingInfo, RequestCount.name(),
+ collector::updateAwsRequestCount);
+ counter(timingInfo, ThrottleException.name(),
+ collector::updateAwsThrottleExceptionsCount);
+
+ timing(timingInfo, ClientExecuteTime.name(),
+ collector::noteAwsClientExecuteTime);
+ timing(timingInfo, HttpRequestTime.name(),
+ collector::noteAwsRequestTime);
+ timing(timingInfo, RequestMarshallTime.name(),
+ collector::noteRequestMarshallTime);
+ timing(timingInfo, RequestSigningTime.name(),
+ collector::noteRequestSigningTime);
+ timing(timingInfo, ResponseProcessingTime.name(),
+ collector::noteResponseProcessingTime);
+ }
+
+ /**
+ * Process a timing.
+ * @param timingInfo timing info
+ * @param subMeasurementName sub measurement
+ * @param durationConsumer consumer
+ */
+ private void timing(
+ TimingInfo timingInfo,
+ String subMeasurementName,
+ Consumer
+ * A final transfer completed event is still expected, so this
+ * does not decrement the active block counter.
+ */
+ void blockUploadFailed(long duration, int blockSize);
+
+ /** Intermediate report of bytes uploaded. */
+ void bytesTransferred(long byteCount);
+
+ /**
+ * Note exception in a multipart complete.
+ * @param count count of exceptions
+ */
+ void exceptionInMultipartComplete(int count);
+
+ /**
+ * Note an exception in a multipart abort.
+ */
+ void exceptionInMultipartAbort();
+
+ /**
+ * Get the number of bytes pending upload.
+ * @return the number of bytes in the pending upload state.
+ */
+ long getBytesPendingUpload();
+
+ /**
+ * Data has been uploaded to be committed in a subsequent operation;
+ * to be called at the end of the write.
+ * @param size size in bytes
+ */
+ void commitUploaded(long size);
+
+ int getBlocksAllocated();
+
+ int getBlocksReleased();
+
+ /**
+ * Get counters of blocks actively allocated; may be inaccurate
+ * if the numbers change during the (non-synchronized) calculation.
+ * @return the number of actively allocated blocks.
+ */
+ int getBlocksActivelyAllocated();
+
+ /**
+ * Record bytes written.
+ * @param count number of bytes
+ */
+ void writeBytes(long count);
+
+ /**
+ * Get the current count of bytes written.
+ * @return the counter value.
+ */
+ long getBytesWritten();
+
+ /**
+ * A block has been allocated.
+ */
+ void blockAllocated();
+
+ /**
+ * A block has been released.
+ */
+ void blockReleased();
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/BondedS3AStatisticsContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/BondedS3AStatisticsContext.java
new file mode 100644
index 0000000000000..2c9fd0cfecc3f
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/BondedS3AStatisticsContext.java
@@ -0,0 +1,209 @@
+/*
+ * 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.s3a.impl.statistics;
+
+import javax.annotation.Nullable;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.s3a.S3AInstrumentation;
+import org.apache.hadoop.fs.s3a.Statistic;
+import org.apache.hadoop.fs.s3a.s3guard.MetastoreInstrumentation;
+
+/**
+ * An S3A statistics context which is bonded to a
+ * S3AInstrumentation instance -inevitably that of an S3AFileSystem
+ * instance.
+ *
+ * An interface is used to bind to the relevant fields, rather
+ * than have them passed in the constructor because some
+ * production code, specifically, DelegateToFileSystem,
+ * patches the protected field after initialization.
+ *
+ * All operations are passed through directly to that class.
+ *
+ *
+ * If an instance of FileSystem.Statistics is passed in, it
+ * will be used whenever input stream statistics are created -
+ * However, Internally always increments the statistics in the
+ * current thread.
+ *
+ * As a result, cross-thread IO will under-report.
+ *
+ *
+ * This is addressed through the stream statistics classes
+ * only updating the stats in the close() call. Provided
+ * they are closed in the worker thread, all stats collected in
+ * helper threads will be included.
+ */
+public class BondedS3AStatisticsContext implements S3AStatisticsContext {
+
+ /** Source of statistics services. */
+ private final S3AFSStatisticsSource statisticsSource;
+
+ /**
+ * Instantiate.
+ * @param statisticsSource integration binding
+ */
+ public BondedS3AStatisticsContext(
+ final S3AFSStatisticsSource statisticsSource) {
+ this.statisticsSource = statisticsSource;
+ }
+
+
+ /**
+ * Get the instrumentation from the FS integration.
+ * @return instrumentation instance.
+ */
+ private S3AInstrumentation getInstrumentation() {
+ return statisticsSource.getInstrumentation();
+ }
+
+ /**
+ * The filesystem statistics: know this is thread-local.
+ * @return FS statistics.
+ */
+ private FileSystem.Statistics getInstanceStatistics() {
+ return statisticsSource.getInstanceStatistics();
+ }
+
+ /**
+ * Get a MetastoreInstrumentation getInstrumentation() instance for this
+ * context.
+ * @return the S3Guard getInstrumentation() point.
+ */
+ @Override
+ public MetastoreInstrumentation getS3GuardInstrumentation() {
+ return getInstrumentation().getS3GuardInstrumentation();
+ }
+
+ /**
+ * Create a stream input statistics instance.
+ * @return the new instance
+ */
+ @Override
+ public S3AInputStreamStatistics newInputStreamStatistics() {
+ return getInstrumentation().newInputStreamStatistics(
+ statisticsSource.getInstanceStatistics());
+ }
+
+ /**
+ * Create a new instance of the committer statistics.
+ * @return a new committer statistics instance
+ */
+ @Override
+ public CommitterStatistics newCommitterStatistics() {
+ return getInstrumentation().newCommitterStatistics();
+ }
+
+ /**
+ * Create a stream output statistics instance.
+ * @return the new instance
+ */
+ @Override
+ public BlockOutputStreamStatistics newOutputStreamStatistics() {
+ return getInstrumentation()
+ .newOutputStreamStatistics(getInstanceStatistics());
+ }
+
+ /**
+ * Increment a specific counter.
+ *
+ * No-op if not defined.
+ * @param op operation
+ * @param count increment value
+ */
+ @Override
+ public void incrementCounter(Statistic op, long count) {
+ getInstrumentation().incrementCounter(op, count);
+ }
+
+ /**
+ * Increment a specific gauge.
+ *
+ * No-op if not defined.
+ * @param op operation
+ * @param count increment value
+ * @throws ClassCastException if the metric is of the wrong type
+ */
+ @Override
+ public void incrementGauge(Statistic op, long count) {
+ getInstrumentation().incrementGauge(op, count);
+ }
+
+ /**
+ * Decrement a specific gauge.
+ *
+ * No-op if not defined.
+ * @param op operation
+ * @param count increment value
+ * @throws ClassCastException if the metric is of the wrong type
+ */
+ @Override
+ public void decrementGauge(Statistic op, long count) {
+ getInstrumentation().decrementGauge(op, count);
+ }
+
+ /**
+ * Add a value to a quantiles statistic. No-op if the quantile
+ * isn't found.
+ * @param op operation to look up.
+ * @param value value to add.
+ * @throws ClassCastException if the metric is not a Quantiles.
+ */
+ @Override
+ public void addValueToQuantiles(Statistic op, long value) {
+ getInstrumentation().addValueToQuantiles(op, value);
+ }
+
+ /**
+ * Create a delegation token statistics instance.
+ * @return an instance of delegation token statistics
+ */
+ @Override
+ public DelegationTokenStatistics newDelegationTokenStatistics() {
+ return getInstrumentation().newDelegationTokenStatistics();
+ }
+
+ @Override
+ public StatisticsFromAwsSdk newStatisticsFromAwsSdk() {
+ return getInstrumentation().newStatisticsFromAwsSdk();
+ }
+
+ /**
+ * This is the interface which an integration source must implement
+ * for the integration.
+ * Note that the FileSystem.statistics field may be null for a class;
+ */
+ public interface S3AFSStatisticsSource {
+
+ /**
+ * Get the S3A Instrumentation.
+ * @return a non-null instrumentation instance
+ */
+ S3AInstrumentation getInstrumentation();
+
+ /**
+ * Get the statistics of the FS instance, shared across all threads.
+ * @return filesystem statistics
+ */
+ @Nullable
+ FileSystem.Statistics getInstanceStatistics();
+
+ }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/ChangeTrackerStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/ChangeTrackerStatistics.java
new file mode 100644
index 0000000000000..c4aa1bab69c9e
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/ChangeTrackerStatistics.java
@@ -0,0 +1,35 @@
+/*
+ * 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.s3a.impl.statistics;
+
+/**
+ * Interface for change tracking statistics.
+ */
+public interface ChangeTrackerStatistics {
+
+ /**
+ * A version mismatch was detected.
+ */
+ void versionMismatchError();
+
+ /**
+ * How many version mismatches have occurred.
+ */
+ long getVersionMismatches();
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/CommitterStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/CommitterStatistics.java
new file mode 100644
index 0000000000000..b3fa552bef7e4
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/CommitterStatistics.java
@@ -0,0 +1,65 @@
+/*
+ * 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.s3a.impl.statistics;
+
+/**
+ * Statistics for committed work.
+ */
+public interface CommitterStatistics {
+
+ /** A commit has been created. */
+ void commitCreated();
+
+ /**
+ * Data has been uploaded to be committed in a subsequent operation.
+ * @param size size in bytes
+ */
+ void commitUploaded(long size);
+
+ /**
+ * A commit has been completed.
+ * @param size size in bytes
+ */
+ void commitCompleted(long size);
+
+ /** A commit has been aborted. */
+ void commitAborted();
+
+ /**
+ * A commit was reverted.
+ */
+ void commitReverted();
+
+ /**
+ * A commit failed.
+ */
+ void commitFailed();
+
+ /**
+ * Note that a task has completed.
+ * @param success success flag
+ */
+ void taskCompleted(boolean success);
+
+ /**
+ * Note that a job has completed.
+ * @param success success flag
+ */
+ void jobCompleted(boolean success);
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/CountersAndGauges.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/CountersAndGauges.java
new file mode 100644
index 0000000000000..5a0651f0878bf
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/CountersAndGauges.java
@@ -0,0 +1,62 @@
+/*
+ * 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.s3a.impl.statistics;
+
+import org.apache.hadoop.fs.s3a.Statistic;
+
+/**
+ * This is the foundational API for collecting S3A statistics.
+ */
+public interface CountersAndGauges {
+
+ /**
+ * Increment a specific counter.
+ * No-op if not defined.
+ * @param op operation
+ * @param count increment value
+ */
+ void incrementCounter(Statistic op, long count);
+
+ /**
+ * Increment a specific gauge.
+ * No-op if not defined.
+ * @param op operation
+ * @param count increment value
+ * @throws ClassCastException if the metric is of the wrong type
+ */
+ void incrementGauge(Statistic op, long count);
+
+ /**
+ * Decrement a specific gauge.
+ * No-op if not defined.
+ * @param op operation
+ * @param count increment value
+ * @throws ClassCastException if the metric is of the wrong type
+ */
+ void decrementGauge(Statistic op, long count);
+
+ /**
+ * Add a value to a quantiles statistic. No-op if the quantile
+ * isn't found.
+ * @param op operation to look up.
+ * @param value value to add.
+ * @throws ClassCastException if the metric is not a Quantiles.
+ */
+ void addValueToQuantiles(Statistic op, long value);
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/CountingChangeTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/CountingChangeTracker.java
new file mode 100644
index 0000000000000..3581267b354ef
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/CountingChangeTracker.java
@@ -0,0 +1,51 @@
+/*
+ * 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.s3a.impl.statistics;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * A change tracker which increments an atomic long.
+ */
+public class CountingChangeTracker implements
+ ChangeTrackerStatistics {
+
+ /**
+ * The counter which is updated on every mismatch.
+ */
+ private final AtomicLong counter;
+
+ public CountingChangeTracker(final AtomicLong counter) {
+ this.counter = counter;
+ }
+
+ public CountingChangeTracker() {
+ this(new AtomicLong());
+ }
+
+ @Override
+ public void versionMismatchError() {
+ counter.incrementAndGet();
+ }
+
+ @Override
+ public long getVersionMismatches() {
+ return counter.get();
+ }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/DelegationTokenStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/DelegationTokenStatistics.java
new file mode 100644
index 0000000000000..d918b8743985c
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/DelegationTokenStatistics.java
@@ -0,0 +1,28 @@
+/*
+ * 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.s3a.impl.statistics;
+
+/**
+ * Instrumentation exported to for S3A Delegation Token support.
+ */
+public interface DelegationTokenStatistics {
+
+ /** A token has been issued. */
+ void tokenIssued();
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/EmptyS3AStatisticsContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/EmptyS3AStatisticsContext.java
new file mode 100644
index 0000000000000..0e64dd1d37eac
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/EmptyS3AStatisticsContext.java
@@ -0,0 +1,463 @@
+/*
+ * 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.s3a.impl.statistics;
+
+import java.io.IOException;
+import java.time.Duration;
+
+import org.apache.hadoop.fs.s3a.Statistic;
+import org.apache.hadoop.fs.s3a.s3guard.MetastoreInstrumentation;
+import org.apache.hadoop.fs.s3a.s3guard.MetastoreInstrumentationImpl;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.emptyStatistics;
+
+/**
+ * Special statistics context, all of whose context operations are no-ops.
+ * All statistics instances it returns are also empty.
+ *
+ * This class is here primarily to aid in testing, but it also allows for
+ * classes to require a non-empty statistics context in their constructor -yet
+ * still be instantiated without one bound to any filesystem.
+ */
+public final class EmptyS3AStatisticsContext implements S3AStatisticsContext {
+
+ @Override
+ public MetastoreInstrumentation getS3GuardInstrumentation() {
+ return new MetastoreInstrumentationImpl();
+ }
+
+ @Override
+ public S3AInputStreamStatistics newInputStreamStatistics() {
+ return new EmptyInputStreamStatistics();
+ }
+
+ @Override
+ public CommitterStatistics newCommitterStatistics() {
+ return new EmptyCommitterStatistics();
+ }
+
+ @Override
+ public BlockOutputStreamStatistics newOutputStreamStatistics() {
+ return new EmptyBlockOutputStreamStatistics();
+ }
+
+ @Override
+ public DelegationTokenStatistics newDelegationTokenStatistics() {
+ return new EmptyDelegationTokenStatistics();
+ }
+
+ @Override
+ public StatisticsFromAwsSdk newStatisticsFromAwsSdk() {
+ return new EmptyStatisticsFromAwsSdk();
+ }
+
+ @Override
+ public void incrementCounter(final Statistic op, final long count) {
+
+ }
+
+ @Override
+ public void incrementGauge(final Statistic op, final long count) {
+
+ }
+
+ @Override
+ public void decrementGauge(final Statistic op, final long count) {
+
+ }
+
+ @Override
+ public void addValueToQuantiles(final Statistic op, final long value) {
+
+ }
+
+ /**
+ * Input Stream statistics callbacks.
+ */
+ private static final class EmptyInputStreamStatistics
+ implements S3AInputStreamStatistics {
+
+ @Override
+ public void seekBackwards(final long negativeOffset) {
+
+ }
+
+ @Override
+ public void seekForwards(final long skipped) {
+
+ }
+
+ @Override
+ public long streamOpened() {
+ return 0;
+ }
+
+ @Override
+ public void streamClose(final boolean abortedConnection,
+ final long remainingInCurrentRequest) {
+
+ }
+
+ @Override
+ public void readException() {
+
+ }
+
+ @Override
+ public void bytesRead(final long bytes) {
+
+ }
+
+ @Override
+ public void readOperationStarted(final long pos, final long len) {
+
+ }
+
+ @Override
+ public void readFullyOperationStarted(final long pos, final long len) {
+
+ }
+
+ @Override
+ public void readOperationCompleted(final int requested, final int actual) {
+
+ }
+
+ @Override
+ public void close() {
+
+ }
+
+ @Override
+ public void inputPolicySet(final int updatedPolicy) {
+
+ }
+
+ @Override
+ public void merge(final boolean isClosed) {
+
+ }
+
+ /**
+ * Return an IO statistics instance.
+ * @return an empty IO statistics instance.
+ */
+ @Override
+ public IOStatistics getIOStatistics() {
+ return emptyStatistics();
+ }
+
+ @Override
+ public long getCloseOperations() {
+ return 0;
+ }
+
+ @Override
+ public long getClosed() {
+ return 0;
+ }
+
+ @Override
+ public long getAborted() {
+ return 0;
+ }
+
+ @Override
+ public long getForwardSeekOperations() {
+ return 0;
+ }
+
+ @Override
+ public long getBackwardSeekOperations() {
+ return 0;
+ }
+
+ @Override
+ public long getBytesRead() {
+ return 0;
+ }
+
+ @Override
+ public long getBytesSkippedOnSeek() {
+ return 0;
+ }
+
+ @Override
+ public long getBytesBackwardsOnSeek() {
+ return 0;
+ }
+
+ @Override
+ public long getBytesReadInClose() {
+ return 0;
+ }
+
+ @Override
+ public long getBytesDiscardedInAbort() {
+ return 0;
+ }
+
+ @Override
+ public long getOpenOperations() {
+ return 0;
+ }
+
+ @Override
+ public long getSeekOperations() {
+ return 0;
+ }
+
+ @Override
+ public long getReadExceptions() {
+ return 0;
+ }
+
+ @Override
+ public long getReadOperations() {
+ return 0;
+ }
+
+ @Override
+ public long getReadFullyOperations() {
+ return 0;
+ }
+
+ @Override
+ public long getReadsIncomplete() {
+ return 0;
+ }
+
+ @Override
+ public long getPolicySetCount() {
+ return 0;
+ }
+
+ @Override
+ public long getVersionMismatches() {
+ return 0;
+ }
+
+ @Override
+ public long getInputPolicy() {
+ return 0;
+ }
+
+ @Override
+ public Long getStatistic(final String name) {
+ return 0L;
+ }
+
+ @Override
+ public ChangeTrackerStatistics getChangeTrackerStatistics() {
+ return new CountingChangeTracker();
+ }
+ }
+
+ private static final class EmptyCommitterStatistics
+ implements CommitterStatistics {
+
+ @Override
+ public void commitCreated() {
+
+ }
+
+ @Override
+ public void commitUploaded(final long size) {
+
+ }
+
+ @Override
+ public void commitCompleted(final long size) {
+
+ }
+
+ @Override
+ public void commitAborted() {
+
+ }
+
+ @Override
+ public void commitReverted() {
+
+ }
+
+ @Override
+ public void commitFailed() {
+
+ }
+
+ @Override
+ public void taskCompleted(final boolean success) {
+
+ }
+
+ @Override
+ public void jobCompleted(final boolean success) {
+
+ }
+ }
+
+ private static final class EmptyBlockOutputStreamStatistics
+ implements BlockOutputStreamStatistics {
+
+ @Override
+ public void blockUploadQueued(final int blockSize) {
+
+ }
+
+ @Override
+ public void blockUploadStarted(final long duration, final int blockSize) {
+
+ }
+
+ @Override
+ public void blockUploadCompleted(final long duration, final int blockSize) {
+
+ }
+
+ @Override
+ public void blockUploadFailed(final long duration, final int blockSize) {
+
+ }
+
+ @Override
+ public void bytesTransferred(final long byteCount) {
+
+ }
+
+ @Override
+ public void exceptionInMultipartComplete(final int count) {
+
+ }
+
+ @Override
+ public void exceptionInMultipartAbort() {
+
+ }
+
+ @Override
+ public long getBytesPendingUpload() {
+ return 0;
+ }
+
+ @Override
+ public void commitUploaded(final long size) {
+
+ }
+
+ @Override
+ public int getBlocksAllocated() {
+ return 0;
+ }
+
+ @Override
+ public int getBlocksReleased() {
+ return 0;
+ }
+
+ @Override
+ public int getBlocksActivelyAllocated() {
+ return 0;
+ }
+
+ @Override
+ public IOStatistics getIOStatistics() {
+ return emptyStatistics();
+ }
+
+ @Override
+ public void blockAllocated() {
+
+ }
+
+ @Override
+ public void blockReleased() {
+
+ }
+
+ @Override
+ public void writeBytes(final long count) {
+ }
+
+ @Override
+ public long getBytesWritten() {
+ return 0;
+ }
+
+ @Override
+ public void close() throws IOException {
+
+ }
+ }
+
+ private static final class EmptyDelegationTokenStatistics
+ implements DelegationTokenStatistics {
+
+ @Override
+ public void tokenIssued() {
+
+ }
+ }
+
+ private static final class EmptyStatisticsFromAwsSdk implements
+ StatisticsFromAwsSdk {
+
+ @Override
+ public void updateAwsRequestCount(final long longValue) {
+
+ }
+
+ @Override
+ public void updateAwsRetryCount(final long longValue) {
+
+ }
+
+ @Override
+ public void updateAwsThrottleExceptionsCount(final long longValue) {
+
+ }
+
+ @Override
+ public void noteAwsRequestTime(final Duration ofMillis) {
+
+ }
+
+ @Override
+ public void noteAwsClientExecuteTime(final Duration ofMillis) {
+
+ }
+
+ @Override
+ public void noteRequestMarshallTime(final Duration duration) {
+
+ }
+
+ @Override
+ public void noteRequestSigningTime(final Duration duration) {
+
+ }
+
+ @Override
+ public void noteResponseProcessingTime(final Duration duration) {
+
+ }
+ }
+
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AInputStreamStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AInputStreamStatistics.java
new file mode 100644
index 0000000000000..c47c8fb82d1e3
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AInputStreamStatistics.java
@@ -0,0 +1,162 @@
+/*
+ * 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.s3a.impl.statistics;
+
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+
+/**
+ * Statistics updated by an input stream during its actual operation.
+ * It also contains getters for tests.
+ */
+public interface S3AInputStreamStatistics extends AutoCloseable,
+ IOStatisticsSource {
+
+ /**
+ * Seek backwards, incrementing the seek and backward seek counters.
+ * @param negativeOffset how far was the seek?
+ * This is expected to be negative.
+ */
+ void seekBackwards(long negativeOffset);
+
+ /**
+ * Record a forward seek, adding a seek operation, a forward
+ * seek operation, and any bytes skipped.
+ * @param skipped number of bytes skipped by reading from the stream.
+ * If the seek was implemented by a close + reopen, set this to zero.
+ */
+ void seekForwards(long skipped);
+
+ /**
+ * The inner stream was opened.
+ * @return the previous count
+ */
+ long streamOpened();
+
+ /**
+ * The inner stream was closed.
+ * @param abortedConnection flag to indicate the stream was aborted,
+ * rather than closed cleanly
+ * @param remainingInCurrentRequest the number of bytes remaining in
+ * the current request.
+ */
+ void streamClose(boolean abortedConnection,
+ long remainingInCurrentRequest);
+
+ /**
+ * An ignored stream read exception was received.
+ */
+ void readException();
+
+ /**
+ * Increment the bytes read counter by the number of bytes;
+ * no-op if the argument is negative.
+ * @param bytes number of bytes read
+ */
+ void bytesRead(long bytes);
+
+ /**
+ * A {@code read(byte[] buf, int off, int len)} operation has started.
+ * @param pos starting position of the read
+ * @param len length of bytes to read
+ */
+ void readOperationStarted(long pos, long len);
+
+ /**
+ * A {@code PositionedRead.read(position, buffer, offset, length)}
+ * operation has just started.
+ * @param pos starting position of the read
+ * @param len length of bytes to read
+ */
+ void readFullyOperationStarted(long pos, long len);
+
+ /**
+ * A read operation has completed.
+ * @param requested number of requested bytes
+ * @param actual the actual number of bytes
+ */
+ void readOperationCompleted(int requested, int actual);
+
+ @Override
+ void close();
+
+ /**
+ * The input policy has been switched.
+ * @param updatedPolicy enum value of new policy.
+ */
+ void inputPolicySet(int updatedPolicy);
+
+ /**
+ * Get a reference to the change tracker statistics for this
+ * stream.
+ * @return a reference to the change tracker statistics
+ */
+ ChangeTrackerStatistics getChangeTrackerStatistics();
+
+ /**
+ * Merge the statistics into the filesystem's instrumentation instance.
+ *
+ * Takes a diff between the current version of the stats and the
+ * version of the stats when merge was last called, and merges the diff
+ * into the instrumentation instance. Used to periodically merge the
+ * stats into the fs-wide stats
+ *
+ * Behavior is undefined if called on a closed instance.
+ */
+ void merge(boolean isClosed);
+
+ long getCloseOperations();
+
+ long getClosed();
+
+ long getAborted();
+
+ long getForwardSeekOperations();
+
+ long getBackwardSeekOperations();
+
+ long getBytesRead();
+
+ long getBytesSkippedOnSeek();
+
+ long getBytesBackwardsOnSeek();
+
+ long getBytesReadInClose();
+
+ long getBytesDiscardedInAbort();
+
+ long getOpenOperations();
+
+ long getSeekOperations();
+
+ long getReadExceptions();
+
+ long getReadOperations();
+
+ long getReadFullyOperations();
+
+ long getReadsIncomplete();
+
+ long getPolicySetCount();
+
+ long getVersionMismatches();
+
+ long getInputPolicy();
+
+ Long getStatistic(String name);
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AStatisticsContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AStatisticsContext.java
new file mode 100644
index 0000000000000..96c081a5c585d
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AStatisticsContext.java
@@ -0,0 +1,64 @@
+/*
+ * 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.s3a.impl.statistics;
+
+import org.apache.hadoop.fs.s3a.s3guard.MetastoreInstrumentation;
+
+/**
+ * This is the statistics context for ongoing operations in S3A.
+ */
+public interface S3AStatisticsContext extends CountersAndGauges {
+
+ /**
+ * Get the metastore instrumentation.
+ * @return an instance of the metastore statistics tracking.
+ */
+ MetastoreInstrumentation getS3GuardInstrumentation();
+
+ /**
+ * Create a stream input statistics instance.
+ * @return the new instance
+ */
+ S3AInputStreamStatistics newInputStreamStatistics();
+
+ /**
+ * Create a new instance of the committer statistics.
+ * @return a new committer statistics instance
+ */
+ CommitterStatistics newCommitterStatistics();
+
+ /**
+ * Create a stream output statistics instance.
+ * @return the new instance
+ */
+ BlockOutputStreamStatistics newOutputStreamStatistics();
+
+ /**
+ * Create a delegation token statistics instance.
+ * @return an instance of delegation token statistics
+ */
+ DelegationTokenStatistics newDelegationTokenStatistics();
+
+ /**
+ * Create a StatisticsFromAwsSdk instance.
+ * @return an instance of StatisticsFromAwsSdk
+ */
+ StatisticsFromAwsSdk newStatisticsFromAwsSdk();
+
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/StatisticsFromAwsSdk.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/StatisticsFromAwsSdk.java
new file mode 100644
index 0000000000000..d04de7a0d5603
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/StatisticsFromAwsSdk.java
@@ -0,0 +1,79 @@
+/*
+ * 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.s3a.impl.statistics;
+
+import java.time.Duration;
+
+/**
+ * interface to receive statistics events from the AWS SDK
+ * by way of {@link AwsStatisticsCollector}.
+ */
+public interface StatisticsFromAwsSdk {
+
+ /**
+ * Record a number of AWS requests.
+ * @param count number of events.
+ */
+ void updateAwsRequestCount(long count);
+
+ /**
+ * Record a number of AWS request retries.
+ * @param count number of events.
+ */
+ void updateAwsRetryCount(long count);
+
+ /**
+ * Record a number of throttle exceptions received.
+ * @param count number of events.
+ */
+ void updateAwsThrottleExceptionsCount(long count);
+
+ /**
+ * Record how long a request took overall.
+ * @param duration duration of operation.
+ */
+ void noteAwsRequestTime(Duration duration);
+
+ /**
+ * Record how long a request took to execute on the
+ * client.
+ * @param duration duration of operation.
+ */
+ void noteAwsClientExecuteTime(Duration duration);
+
+ /**
+ * Record how long a request took to marshall into
+ * XML.
+ * @param duration duration of operation.
+ */
+ void noteRequestMarshallTime(Duration duration);
+
+ /**
+ * Record how long a request took to sign, including
+ * any calls to EC2 credential endpoints.
+ * @param duration duration of operation.
+ */
+ void noteRequestSigningTime(Duration duration);
+
+ /**
+ * Record how long it took to process the response.
+ * @param duration duration of operation.
+ */
+ void noteResponseProcessingTime(Duration duration);
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/package-info.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/package-info.java
new file mode 100644
index 0000000000000..5d9d3b4d8d81a
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/package-info.java
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+/**
+ * Statistics collection for the S3A connector.
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.fs.s3a.impl.statistics;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectInputStream.java
index f4bd8d11708ef..b99ddb830c785 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectInputStream.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectInputStream.java
@@ -39,9 +39,9 @@
import org.apache.hadoop.fs.FSInputStream;
import org.apache.hadoop.fs.PathIOException;
import org.apache.hadoop.fs.s3a.Retries;
-import org.apache.hadoop.fs.s3a.S3AInstrumentation;
import org.apache.hadoop.fs.s3a.S3AReadOpContext;
import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
+import org.apache.hadoop.fs.s3a.impl.statistics.S3AInputStreamStatistics;
import org.apache.hadoop.io.IOUtils;
import static com.google.common.base.Preconditions.checkNotNull;
@@ -103,7 +103,7 @@ public class SelectInputStream extends FSInputStream implements
private final S3AReadOpContext readContext;
- private final S3AInstrumentation.InputStreamStatistics streamStatistics;
+ private final S3AInputStreamStatistics streamStatistics;
private long readahead;
@@ -130,7 +130,7 @@ public SelectInputStream(
this.uri = "s3a://" + this.bucket + "/" + this.key;
this.readContext = readContext;
this.readahead = readContext.getReadahead();
- this.streamStatistics = readContext.getInstrumentation()
+ this.streamStatistics = readContext.getS3AStatisticsContext()
.newInputStreamStatistics();
SelectRecordsInputStream stream = once(
"S3 Select",
@@ -428,7 +428,7 @@ private void incrementBytesRead(long bytesRead) {
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
- public S3AInstrumentation.InputStreamStatistics getS3AStreamStatistics() {
+ public S3AInputStreamStatistics getS3AStreamStatistics() {
return streamStatistics;
}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java
index afd3ec2bd3473..3e2786dbe475f 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java
@@ -22,6 +22,7 @@
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.s3a.impl.statistics.BlockOutputStreamStatistics;
import org.apache.hadoop.io.IOUtils;
import org.junit.BeforeClass;
@@ -94,16 +95,16 @@ public void testBlocksClosed() throws Throwable {
Path dest = path("testBlocksClosed");
describe(" testBlocksClosed");
FSDataOutputStream stream = getFileSystem().create(dest, true);
- S3AInstrumentation.OutputStreamStatistics statistics
+ BlockOutputStreamStatistics statistics
= S3ATestUtils.getOutputStreamStatistics(stream);
byte[] data = ContractTestUtils.dataset(16, 'a', 26);
stream.write(data);
LOG.info("closing output stream");
stream.close();
assertEquals("total allocated blocks in " + statistics,
- 1, statistics.blocksAllocated());
+ 1, statistics.getBlocksAllocated());
assertEquals("actively allocated blocks in " + statistics,
- 0, statistics.blocksActivelyAllocated());
+ 0, statistics.getBlocksActivelyAllocated());
LOG.info("end of test case");
}
@@ -129,7 +130,7 @@ private void markAndResetDatablock(S3ADataBlocks.BlockFactory factory)
throws Exception {
S3AInstrumentation instrumentation =
new S3AInstrumentation(new URI("s3a://example"));
- S3AInstrumentation.OutputStreamStatistics outstats
+ BlockOutputStreamStatistics outstats
= instrumentation.newOutputStreamStatistics(null);
S3ADataBlocks.DataBlock block = factory.create(1, BLOCK_SIZE, outstats);
block.write(dataset, 0, dataset.length);
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AUnbuffer.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AUnbuffer.java
index 2ba3fd7a65cde..e95a6d1f4bd62 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AUnbuffer.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AUnbuffer.java
@@ -71,7 +71,7 @@ public void testUnbuffer() throws IOException {
/**
* Test that calling {@link S3AInputStream#unbuffer()} merges a stream's
- * {@link org.apache.hadoop.fs.s3a.S3AInstrumentation.InputStreamStatistics}
+ * {@code InputStreamStatistics}
* into the {@link S3AFileSystem}'s {@link S3AInstrumentation} instance.
*/
@Test
@@ -108,7 +108,7 @@ public void testUnbufferStreamStatistics() throws IOException {
// Validate that the input stream stats are correct when the file is closed
assertEquals("S3AInputStream statistics were not updated properly", 12,
((S3AInputStream) inputStream.getWrappedStream())
- .getS3AStreamStatistics().bytesRead);
+ .getS3AStreamStatistics().getBytesRead());
}
private boolean isObjectStreamOpen(FSDataInputStream inputStream) {
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java
index e30269298111d..0bddd264a6c15 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java
@@ -39,11 +39,12 @@
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets;
import org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase;
+import org.apache.hadoop.fs.s3a.impl.statistics.CommitterStatistics;
+import org.apache.hadoop.fs.s3a.impl.statistics.EmptyS3AStatisticsContext;
import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
import org.apache.hadoop.util.Progressable;
import static com.google.common.base.Preconditions.checkNotNull;
-import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
/**
* Relays FS calls to the mocked FS, allows for some extra logging with
@@ -83,8 +84,6 @@ public class MockS3AFileSystem extends S3AFileSystem {
* mock FS.
*/
private int logEvents = LOG_NAME;
- private final S3AInstrumentation instrumentation =
- new S3AInstrumentation(FS_URI);
private Configuration conf;
private WriteOperationHelper writeHelper;
@@ -146,12 +145,12 @@ public Path qualify(final Path path) {
public void initialize(URI name, Configuration originalConf)
throws IOException {
conf = originalConf;
- writeHelper = new WriteOperationHelper(this, conf);
+ writeHelper = new WriteOperationHelper(this, conf,
+ new EmptyS3AStatisticsContext());
}
@Override
public void close() {
- cleanupWithLogger(LOG, instrumentation);
}
@Override
@@ -359,12 +358,56 @@ public String toString() {
}
@Override
- public S3AInstrumentation.CommitterStatistics newCommitterStatistics() {
- return instrumentation.newCommitterStatistics();
+ public CommitterStatistics newCommitterStatistics() {
+ return new StubCommitterStatistics();
}
@Override
public void operationRetried(Exception ex) {
/** no-op */
}
+
+ private final class StubCommitterStatistics implements CommitterStatistics {
+
+ @Override
+ public void commitCreated() {
+
+ }
+
+ @Override
+ public void commitUploaded(final long size) {
+
+ }
+
+ @Override
+ public void commitCompleted(final long size) {
+
+ }
+
+ @Override
+ public void commitAborted() {
+
+ }
+
+ @Override
+ public void commitReverted() {
+
+ }
+
+ @Override
+ public void commitFailed() {
+
+ }
+
+ @Override
+ public void taskCompleted(final boolean success) {
+
+ }
+
+ @Override
+ public void jobCompleted(final boolean success) {
+
+ }
+ }
+
}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java
index 4644cf24764ae..86a1675fae2f5 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java
@@ -28,6 +28,8 @@
import com.amazonaws.services.s3.model.MultipartUploadListing;
import com.amazonaws.services.s3.model.Region;
+import org.apache.hadoop.fs.s3a.impl.statistics.StatisticsFromAwsSdk;
+
/**
* An {@link S3ClientFactory} that returns Mockito mocks of the {@link AmazonS3}
* interface suitable for unit testing.
@@ -38,7 +40,8 @@ public class MockS3ClientFactory implements S3ClientFactory {
public AmazonS3 createS3Client(URI name,
final String bucket,
final AWSCredentialsProvider credentialSet,
- final String userAgentSuffix) {
+ final String userAgentSuffix,
+ final StatisticsFromAwsSdk statisticsFromAwsSdks) {
AmazonS3 s3 = mock(AmazonS3.class);
when(s3.doesBucketExist(bucket)).thenReturn(true);
when(s3.doesBucketExistV2(bucket)).thenReturn(true);
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
index aa5979dbf751e..79e82de4537ed 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
@@ -34,8 +34,8 @@
import org.apache.hadoop.fs.s3a.auth.MarshalledCredentialBinding;
import org.apache.hadoop.fs.s3a.auth.MarshalledCredentials;
import org.apache.hadoop.fs.s3a.commit.CommitConstants;
-
import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum;
+import org.apache.hadoop.fs.s3a.impl.statistics.BlockOutputStreamStatistics;
import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
import org.apache.hadoop.fs.s3a.s3guard.MetadataStoreCapabilities;
import org.apache.hadoop.fs.s3native.S3xLoginHelper;
@@ -1162,7 +1162,7 @@ public static void assume(String message, boolean condition) {
* @param out output stream
* @return the (active) stats of the write
*/
- public static S3AInstrumentation.OutputStreamStatistics
+ public static BlockOutputStreamStatistics
getOutputStreamStatistics(FSDataOutputStream out) {
S3ABlockOutputStream blockOutputStream
= (S3ABlockOutputStream) out.getWrappedStream();
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java
index 60904d7ae83f1..c96a137ef1834 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java
@@ -21,6 +21,7 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.PathIOException;
import org.apache.hadoop.fs.s3a.commit.PutTracker;
+import org.apache.hadoop.fs.s3a.impl.statistics.EmptyS3AStatisticsContext;
import org.apache.hadoop.util.Progressable;
import org.junit.Before;
import org.junit.Test;
@@ -50,11 +51,10 @@ public void setUp() throws Exception {
S3ADataBlocks.BlockFactory blockFactory =
mock(S3ADataBlocks.BlockFactory.class);
long blockSize = Constants.DEFAULT_MULTIPART_SIZE;
- S3AInstrumentation.OutputStreamStatistics statistics = null;
WriteOperationHelper oHelper = mock(WriteOperationHelper.class);
PutTracker putTracker = mock(PutTracker.class);
stream = spy(new S3ABlockOutputStream(fs, "", executorService,
- progressable, blockSize, blockFactory, statistics, oHelper,
+ progressable, blockSize, blockFactory, null, oHelper,
putTracker));
}
@@ -70,7 +70,8 @@ public void testWriteOperationHelperPartLimits() throws Throwable {
S3AFileSystem s3a = mock(S3AFileSystem.class);
when(s3a.getBucket()).thenReturn("bucket");
WriteOperationHelper woh = new WriteOperationHelper(s3a,
- new Configuration());
+ new Configuration(),
+ new EmptyS3AStatisticsContext());
ByteArrayInputStream inputStream = new ByteArrayInputStream(
"a".getBytes());
// first one works
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java
index 3d7cdfc08dec4..41ff4c2fbf3fd 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java
@@ -18,8 +18,6 @@
package org.apache.hadoop.fs.s3a;
-import java.util.concurrent.atomic.AtomicLong;
-
import com.amazonaws.AmazonServiceException;
import com.amazonaws.SdkBaseException;
import com.amazonaws.services.s3.Headers;
@@ -36,6 +34,7 @@
import org.apache.hadoop.fs.PathIOException;
import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy;
import org.apache.hadoop.fs.s3a.impl.ChangeTracker;
+import org.apache.hadoop.fs.s3a.impl.statistics.CountingChangeTracker;
import org.apache.hadoop.test.HadoopTestBase;
import static org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy.CHANGE_DETECTED;
@@ -359,7 +358,7 @@ protected void assertTrackerMismatchCount(
final ChangeTracker tracker,
final int expectedCount) {
assertEquals("counter in tracker " + tracker,
- expectedCount, tracker.getVersionMismatches().get());
+ expectedCount, tracker.getVersionMismatches());
}
/**
@@ -386,7 +385,7 @@ protected ChangeTracker newTracker(final ChangeDetectionPolicy.Mode mode,
source,
requireVersion);
ChangeTracker tracker = new ChangeTracker(URI, policy,
- new AtomicLong(0), objectAttributes);
+ new CountingChangeTracker(), objectAttributes);
if (objectAttributes.getVersionId() == null
&& objectAttributes.getETag() == null) {
assertFalse("Tracker should not have applied constraints " + tracker,
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationInFileystem.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationInFileystem.java
index f6c908ea2f7e6..1910ed91fe98d 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationInFileystem.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationInFileystem.java
@@ -44,6 +44,8 @@
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.fs.s3a.S3ATestUtils;
import org.apache.hadoop.fs.s3a.Statistic;
+import org.apache.hadoop.fs.s3a.impl.statistics.EmptyS3AStatisticsContext;
+import org.apache.hadoop.fs.s3a.impl.statistics.StatisticsFromAwsSdk;
import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.security.TokenCache;
@@ -70,6 +72,7 @@
import static org.apache.hadoop.fs.s3a.auth.delegation.MiniKerberizedHadoopCluster.ALICE;
import static org.apache.hadoop.fs.s3a.auth.delegation.MiniKerberizedHadoopCluster.assertSecurityEnabled;
import static org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens.lookupS3ADelegationToken;
+import static org.apache.hadoop.fs.s3a.impl.InternalConstants.AWS_SDK_METRICS_ENABLED;
import static org.apache.hadoop.test.LambdaTestUtils.doAs;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.hamcrest.Matchers.containsString;
@@ -305,6 +308,9 @@ public void testDelegatedFileSystem() throws Throwable {
describe("Delegation tokens can be passed to a new filesystem;"
+ " if role restricted, permissions are tightened.");
S3AFileSystem fs = getFileSystem();
+ // force a probe of the remote FS to make sure its endpoint is valid
+ // (this always hits S3, even when S3Guard is enabled)
+ fs.getObjectMetadata(new Path("/"));
readLandsatMetadata(fs);
URI uri = fs.getUri();
@@ -561,8 +567,13 @@ protected ObjectMetadata readLandsatMetadata(final S3AFileSystem delegatedFS)
conf.set(ENDPOINT, "");
factory.setConf(conf);
String host = landsat.getHost();
+ StatisticsFromAwsSdk awsStats = null;
+ if (AWS_SDK_METRICS_ENABLED) {
+ awsStats = new EmptyS3AStatisticsContext()
+ .newStatisticsFromAwsSdk();
+ }
AmazonS3 s3 = factory.createS3Client(landsat, host, testing,
- "ITestSessionDelegationInFileystem");
+ "ITestSessionDelegationInFileystem", awsStats);
return Invoker.once("HEAD", host,
() -> s3.getObjectMetadata(host, landsat.getPath().substring(1)));
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNetworkBinding.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNetworkBinding.java
new file mode 100644
index 0000000000000..10fe339174fc5
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNetworkBinding.java
@@ -0,0 +1,105 @@
+/*
+ * 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.s3a.impl;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.client.builder.AwsClientBuilder;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import org.apache.hadoop.test.AbstractHadoopTestBase;
+
+import static org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.createEndpointConfiguration;
+import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.fixBucketRegion;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Unit tests related to the {@link NetworkBinding} class.
+ */
+public class TestNetworkBinding extends AbstractHadoopTestBase {
+
+ private static final String US_EAST_1 = "us-east-1";
+
+ private static final String US_WEST_2 = "us-west-2";
+
+ @Test
+ public void testUSEast() {
+ assertRegionFixup(US_EAST_1, US_EAST_1);
+ }
+
+ @Test
+ public void testUSWest() {
+ assertRegionFixup(US_WEST_2, US_WEST_2);
+ }
+
+ @Test
+ public void testRegionUStoUSEast() {
+ assertRegionFixup("US", US_EAST_1);
+ }
+
+ @Test
+ public void testRegionNullToUSEast() {
+ assertRegionFixup(null, US_EAST_1);
+ }
+
+ private static void assertRegionFixup(String region, String expected) {
+ assertThat(fixBucketRegion(region))
+ .describedAs("Fixup of %s", region)
+ .isEqualTo(expected);
+ }
+
+ @Test
+ public void testNull() throws Throwable {
+ expectEndpoint("", true, "unused");
+ }
+
+ @Test
+ @Ignore("disabled until endpoint logic works for S3 client builder API")
+ public void testUSEastEndpoint() throws Throwable {
+ expectEndpoint(US_EAST_1, false, US_EAST_1);
+ }
+
+ @Test
+ @Ignore("disabled until endpoint logic works for S3 client builder API")
+ public void testUSWestEndpoint() throws Throwable {
+ expectEndpoint(US_WEST_2, false, US_WEST_2);
+ }
+
+ public void expectEndpoint(final String src,
+ final boolean expectNull,
+ final String expectRegion) {
+ AwsClientBuilder.EndpointConfiguration epr =
+ createEndpointConfiguration(src, new ClientConfiguration());
+ String eprStr = epr == null
+ ? "(empty)"
+ : ("(" + epr.getServiceEndpoint() + " " + epr.getSigningRegion());
+ if (expectNull) {
+ assertThat(epr)
+ .describedAs("Endpoint configuration of %s =",
+ src, eprStr)
+ .isNull();
+ } else {
+ assertThat(epr)
+ .describedAs("Endpoint configuration of %s =",
+ src, eprStr)
+ .hasFieldOrPropertyWithValue("serviceEndpoint", src)
+ .hasFieldOrPropertyWithValue("signingRegion", expectRegion);
+ }
+ }
+}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNeworkBinding.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNeworkBinding.java
deleted file mode 100644
index eebc3bfdf2fe4..0000000000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNeworkBinding.java
+++ /dev/null
@@ -1,61 +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.s3a.impl;
-
-import org.assertj.core.api.Assertions;
-import org.junit.Test;
-
-import org.apache.hadoop.test.HadoopTestBase;
-
-import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.fixBucketRegion;
-
-/**
- * Unit tests related to the {@link NetworkBinding} class.
- */
-public class TestNeworkBinding extends HadoopTestBase {
-
- private static final String US_EAST_1 = "us-east-1";
- private static final String US_WEST_2 = "us-west-2";
-
- @Test
- public void testUSEast() {
- assertRegionFixup(US_EAST_1, US_EAST_1);
- }
-
- @Test
- public void testUSWest() {
- assertRegionFixup(US_WEST_2, US_WEST_2);
- }
-
- @Test
- public void testRegionUStoUSEast() {
- assertRegionFixup("US", US_EAST_1);
- }
-
- @Test
- public void testRegionNullToUSEast() {
- assertRegionFixup(null, US_EAST_1);
- }
-
- private static void assertRegionFixup(String region, String expected) {
- Assertions.assertThat(fixBucketRegion(region))
- .describedAs("Fixup of %s", region)
- .isEqualTo(expected);
- }
-}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java
index 244d2eed324c7..bfac41a48a48e 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java
@@ -48,8 +48,8 @@
import org.apache.hadoop.fs.s3a.Invoker;
import org.apache.hadoop.fs.s3a.S3AFileStatus;
import org.apache.hadoop.fs.s3a.S3AInputPolicy;
-import org.apache.hadoop.fs.s3a.S3AInstrumentation;
import org.apache.hadoop.fs.s3a.S3AStorageStatistics;
+import org.apache.hadoop.fs.s3a.impl.statistics.EmptyS3AStatisticsContext;
import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
import org.apache.hadoop.fs.s3a.s3guard.DirListingMetadata;
import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider;
@@ -216,7 +216,7 @@ private StoreContext createMockStoreContext(boolean multiDelete,
"s3a-transfer-shared"),
Constants.DEFAULT_EXECUTOR_CAPACITY,
new Invoker(RetryPolicies.TRY_ONCE_THEN_FAIL, Invoker.LOG_EVENT),
- new S3AInstrumentation(name),
+ new EmptyS3AStatisticsContext(),
new S3AStorageStatistics(),
S3AInputPolicy.Normal,
ChangeDetectionPolicy.createPolicy(ChangeDetectionPolicy.Mode.None,
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/statistics/ITestS3AContractStreamIOStatistics.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/statistics/ITestS3AContractStreamIOStatistics.java
new file mode 100644
index 0000000000000..697dd195e8b22
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/statistics/ITestS3AContractStreamIOStatistics.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.s3a.impl.statistics;
+
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractStreamIOStatisticsTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.contract.s3a.S3AContract;
+import org.apache.hadoop.fs.statistics.StreamStatisticNames;
+
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeEnableS3Guard;
+import static org.apache.hadoop.fs.statistics.StreamStatisticNames.*;
+
+/**
+ * Test the S3A Streams IOStatistics support.
+ */
+public class ITestS3AContractStreamIOStatistics extends
+ AbstractContractStreamIOStatisticsTest {
+
+ @Override
+ protected Configuration createConfiguration() {
+ Configuration conf = super.createConfiguration();
+ // patch in S3Guard options
+ maybeEnableS3Guard(conf);
+ return conf;
+ }
+
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new S3AContract(conf);
+ }
+
+ /**
+ * Keys which the input stream must support.
+ * @return a list of keys
+ */
+ public List
+ *
+ */
+@InterfaceStability.Unstable
+public final class StoreStatisticNames {
+
+
+ private StoreStatisticNames() {
+ }
+
+ /** {@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";
+
+ /** {@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";
+
+}
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..9c37426e6f702
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StreamStatisticNames.java
@@ -0,0 +1,165 @@
+/*
+ * 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;
+
+/**
+ * These are common statistic names.
+ *
+ *
+ */
+@InterfaceStability.Unstable
+public final class StreamStatisticNames {
+
+ private StreamStatisticNames() {
+ }
+
+ /** {@value}. */
+ public static final String STREAM_READ_ABORTED = "stream_aborted";
+
+ /** {@value}. */
+ public static final String STREAM_READ_BYTES_DISCARDED_ABORT
+ = "stream_read_bytes_discarded_in_abort";
+
+ /** {@value}. */
+ public static final String STREAM_READ_BYTES_DISCARDED_SEEK
+ = "stream_read_bytes_discarded_in_seek";
+
+ /** {@value}. */
+ public static final String STREAM_READ_CLOSED = "stream_read_closed";
+
+ /** {@value}. */
+ public static final String STREAM_READ_CLOSE_BYTES_READ
+ = "stream_read_bytes_read_in_close";
+
+ /** {@value}. */
+ public static final String STREAM_READ_CLOSE_OPERATIONS
+ = "stream_read_close_operations";
+
+ /** {@value}. */
+ public static final String STREAM_READ_OPENED = "stream_read_opened";
+
+ /** {@value}. */
+ public static final String STREAM_READ_BYTES
+ = "stream_read_bytes";
+
+ /** {@value}. */
+ public static final String STREAM_READ_EXCEPTIONS =
+ "stream_read_exceptions";
+
+ /** {@value}. */
+ public static final String STREAM_READ_FULLY_OPERATIONS
+ = "stream_read_fully_operations";
+
+ /** {@value}. */
+ public static final String STREAM_READ_OPERATIONS =
+ "stream_read_operations";
+
+ /** {@value}. */
+ public static final String STREAM_READ_OPERATIONS_INCOMPLETE
+ = "stream_read_operations_incomplete";
+
+ /** {@value}. */
+ public static final String STREAM_READ_VERSION_MISMATCHES
+ = "stream_read_version_mismatches";
+
+ /** {@value}. */
+ public static final String STREAM_READ_SEEK_BYTES_BACKWARDS
+ = "stream_read_bytes_backwards_on_seek";
+
+ /** {@value}. */
+ public static final String STREAM_READ_SEEK_FORWARD_OPERATIONS
+ = "stream_read_seek_forward_operations";
+
+ /** {@value}. */
+ public static final String STREAM_READ_SEEK_OPERATIONS =
+ "stream_read_seek_operations";
+
+ /** {@value}. */
+ public static final String STREAM_READ_SEEK_BACKWARD_OPERATIONS =
+ "stream_read_seek_backward_operations";
+
+ /** {@value}. */
+ public static final String STREAM_READ_SEEK_BYTES_READ =
+ "stream_read_seek_bytes_read";
+
+ /** {@value}. */
+ public static final String STREAM_READ_SEEK_BYTES_SKIPPED
+ = "stream_read_bytes_skipped_on_seek";
+
+ /** {@value}. */
+ public static final String STREAM_READ_SKIP_OPERATIONS =
+ "stream_read_skip_operations";
+
+ /** {@value}. */
+ public static final String STREAM_READ_SKIP_BYTES =
+ "stream_read_skip_bytes";
+
+ /** {@value}. */
+ public static final String STREAM_WRITE_EXCEPTIONS =
+ "stream_write_exceptions";
+
+ /** {@value}. */
+ public static final String STREAM_WRITE_BLOCK_UPLOADS
+ = "stream_write_block_uploads";
+
+ /** {@value}. */
+ public static final String STREAM_WRITE_BLOCK_UPLOADS_ABORTED
+ = "stream_write_block_uploads_aborted";
+
+ /** {@value}. */
+ public static final String STREAM_WRITE_BLOCK_UPLOADS_ACTIVE
+ = "stream_write_block_uploads_active";
+
+ /** {@value}. */
+ public static final String STREAM_WRITE_BLOCK_UPLOADS_COMMITTED
+ = "stream_write_block_uploads_committed";
+
+ /** {@value}. */
+ public static final String STREAM_WRITE_BLOCK_UPLOADS_PENDING
+ = "stream_write_block_uploads_pending";
+
+ /** {@value}. */
+ public static final String STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING =
+ "stream_write_block_uploads_data_pending";
+
+ /** {@value}. */
+ public static final String STREAM_WRITE_BYTES
+ = "stream_write_bytes";
+
+ /** {@value}. */
+ public static final String STREAM_WRITE_TOTAL_TIME
+ = "stream_write_total_time";
+
+ /** {@value}. */
+ public static final String STREAM_WRITE_QUEUE_DURATION
+ = "stream_write_queue_duration";
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/BufferedIOStatisticsInputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/BufferedIOStatisticsInputStream.java
new file mode 100644
index 0000000000000..3412fdd3be25e
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/BufferedIOStatisticsInputStream.java
@@ -0,0 +1,52 @@
+/*
+ * 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.BufferedInputStream;
+import java.io.InputStream;
+
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+
+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.
+ */
+public class BufferedIOStatisticsInputStream
+ extends BufferedInputStream
+ implements IOStatisticsSource {
+
+ public BufferedIOStatisticsInputStream(final InputStream in) {
+ super(in);
+ }
+
+ public BufferedIOStatisticsInputStream(final InputStream in, final int size) {
+ super(in, size);
+ }
+
+ @Override
+ public IOStatistics getIOStatistics() {
+ return retrieveIOStatistics(in);
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/BufferedIOStatisticsOutputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/BufferedIOStatisticsOutputStream.java
new file mode 100644
index 0000000000000..4b81365fb94ff
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/BufferedIOStatisticsOutputStream.java
@@ -0,0 +1,53 @@
+/*
+ * 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.BufferedOutputStream;
+import java.io.OutputStream;
+
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+
+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.
+ */
+public class BufferedIOStatisticsOutputStream
+ extends BufferedOutputStream
+ implements IOStatisticsSource {
+
+ public BufferedIOStatisticsOutputStream(final OutputStream out) {
+ super(out);
+ }
+
+ public BufferedIOStatisticsOutputStream(final OutputStream out,
+ final int size) {
+ super(out, size);
+ }
+
+ @Override
+ public IOStatistics getIOStatistics() {
+ return retrieveIOStatistics(out);
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/CounterIOStatistics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/CounterIOStatistics.java
new file mode 100644
index 0000000000000..42b438e7459bb
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/CounterIOStatistics.java
@@ -0,0 +1,80 @@
+/*
+ * 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;
+
+/**
+ * Interface an IOStatistics source where all the counters
+ * are implemented as a static set of counters.
+ *
+ * Thread safe.
+ */
+public interface CounterIOStatistics extends IOStatistics {
+
+ long increment(String key, long value);
+
+ /**
+ * Increment the counter by one.
+ * No-op if the counter is unknown.
+ * @param key statistics key
+ * @return old value or 0
+ */
+ default long increment(String key) {
+ return increment(key, 1);
+ }
+
+ void set(String key, long value);
+
+ /**
+ * Reset all counters.
+ * Unsynchronized.
+ */
+ void resetCounters();
+
+ /**
+ * Update the counter values from a statistics source.
+ * The source must have all keys in this instance;
+ * extra keys are ignored.
+ * @param source source of statistics.
+ */
+ void copy(IOStatistics source);
+
+ /**
+ * Add the counter values from a statistics source.
+ *
+ * All entries must be counters.
+ *
+ * The source must have all keys in this instance;
+ * extra keys are ignored.
+ * @param source source of statistics.
+ */
+ void add(IOStatistics source);
+
+ /**
+ * Subtract the counter values from a statistics source.
+ *
+ * All entries must be counters.
+ *
+ * The source must have all keys in this instance;
+ * extra keys are ignored.
+ * @param source source of statistics.
+ */
+ void subtract(IOStatistics source);
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/CounterIOStatisticsImpl.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/CounterIOStatisticsImpl.java
new file mode 100644
index 0000000000000..8c552ed2e7a51
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/CounterIOStatisticsImpl.java
@@ -0,0 +1,125 @@
+/*
+ * 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.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+
+
+import com.google.common.base.Preconditions;
+
+import org.apache.hadoop.fs.statistics.IOStatisticEntry;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.dynamicIOStatistics;
+
+/**
+ * Implement counter statistics as a map of AtomicLong counters
+ * created in the constructor.
+ */
+final class CounterIOStatisticsImpl extends WrappedIOStatistics
+ implements CounterIOStatistics {
+
+ private final Map
+ *
+ *
+ *
+ *
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+package org.apache.hadoop.fs.statistics;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md
index df538ee6cf96b..25cc9d13d1fca 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md
@@ -38,3 +38,4 @@ HDFS as these are commonly expected by Hadoop client applications.
2. [Testing with the Filesystem specification](testing.html)
2. [Extending the specification and its tests](extending.html)
1. [Uploading a file using Multiple Parts](multipartuploader.html)
+1. [IOStatistics](iostatistics.html)
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/iostatistics.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/iostatistics.md
new file mode 100644
index 0000000000000..7f917622300e9
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/iostatistics.md
@@ -0,0 +1,272 @@
+
+
+# Introduction to the IOStatistics API
+
+```java
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+```
+
+The `IOStatistics` API is intended to provide statistics on individual IO
+classes -such as input and output streams, *in a standard way which
+applications can query*
+
+Many filesystem-related classes have implemented statistics gathering
+and provided private/unstable ways to query this, but as they were
+not common across implementations it was unsafe for applications
+to reference these values. Example: `S3AInputStream` and its statistics
+API. This is used in internal tests, but cannot be used downstream in
+applications such as
+
+The new IOStatistics API is intended to
+
+1. Be instance specific:, rather than shared across multiple instances
+ of a class, or thread local.
+1. Be public and stable enough to be used by applications.
+1. Be easy to use in applications written in Java, Scala, and, via libhdfs, C/C++
+1. Have foundational interfaces and classes in the `hadoop-common` JAR.
+
+## Core model
+
+Any Hadoop I/O class *may* implement `IOStatisticsSource` in order to
+provide statistics.
+
+Wrapper I/O Classes (e.g `FSDataInputStream`, `FSDataOutputStream` *should*
+implement the interface and forward it to the wrapped class, if they also
+implement it -or return `null` if they do not.
+
+`IOStatisticsSource` implementations `getIOStatistics()` return an
+instance of `IOStatistics` enumerating the statistics of that specific
+instance.
+
+The `IOStatistics` implementations provide
+
+* A way to enumerate all keys/statistics monitored.
+* An iterator over all such keys and their latest values.
+* A way to explitly request the value of specific statistic.
+
+A single statistic is reprsented by an `IOStatisticEntry` instance.
+This supports multiple types of statistics, each as a type (integer)
+and an array of values -which are viewed as a _tuple_ whose _arity_
+is the length of the array.
+
+
+
+## package `org.apache.hadoop.fs.statistics`
+
+This package contains the public statistics APIs intended
+for use by applications.
+
+
+
+
+
+
+### class `org.apache.hadoop.fs.statistics.IOStatisticEntry`
+
+```java
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public final class IOStatisticEntry implements Serializable {
+...
+}
+```
+
+This represents the evaluated value of a statistic.
+It is more than just a simple scala value
+
+1. Serializable.
+1. Extensible: new statistic types MAY be added.
+1. non-scalar: a statistic may require more than one long value to represent it.
+1. Aggregatable: two `IOStatisticEntry` instances of the same type
+ can be aggregated. For some types (e.g the mean), this requires more
+ than one element of data to
+1. Forwards compatible to the extent that adding a new statistics type
+ MUST NOT break any existing applications, even if any new and unknown
+ types cannot be aggregated.
+
+| Type ID | Name | Data | Function |
+|---------|------|------|----------|
+| 0 | counter | `(count)` | a counter which can be aggregated through addition |
+| 1 | min | `(value)` | a minimum recorded value |
+| 2 | max | `(value)` | a maximum recorded value |
+| 3 | mean | `(mean, sample_count)` | an arithmentic mean and the number of samples used |
+
+Please consult the javadocs for its full set of methods.
+
+
+
+
+
+
+### class `org.apache.hadoop.fs.statistics.IOStatisticsSource`
+
+```java
+
+/**
+ * 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
+ */
+ IOStatistics getIOStatistics();
+}
+```
+
+This is the interface which an object instance MUST implement if they are a source of
+IOStatistics information.
+
+
+
+
+
+### class `org.apache.hadoop.fs.statistics.IOStatistics`
+
+These are per-instance statistics provided by an object which
+implements `IOStatisticsSource`.
+
+```java
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public interface IOStatistics
+ extends Iterable
+ *
+ *
* Counters and metrics are generally addressed in code by their name or
* {@link Statistic} key. There may be some Statistics which do
* not have an entry here. To avoid attempts to access such counters failing,
@@ -63,32 +99,33 @@
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
-public class S3AInstrumentation implements Closeable, MetricsSource {
+public class S3AInstrumentation implements Closeable, MetricsSource,
+ CountersAndGauges {
private static final Logger LOG = LoggerFactory.getLogger(
S3AInstrumentation.class);
private static final String METRICS_SOURCE_BASENAME = "S3AMetrics";
/**
- * {@value #METRICS_SYSTEM_NAME} The name of the s3a-specific metrics
+ * {@value} The name of the s3a-specific metrics
* system instance used for s3a metrics.
*/
public static final String METRICS_SYSTEM_NAME = "s3a-file-system";
/**
- * {@value #CONTEXT} Currently all s3a metrics are placed in a single
+ * {@value} Currently all s3a metrics are placed in a single
* "context". Distinct contexts may be used in the future.
*/
public static final String CONTEXT = "s3aFileSystem";
/**
- * {@value #METRIC_TAG_FILESYSTEM_ID} The name of a field added to metrics
+ * {@value} The name of a field added to metrics
* records that uniquely identifies a specific FileSystem instance.
*/
public static final String METRIC_TAG_FILESYSTEM_ID = "s3aFileSystemId";
/**
- * {@value #METRIC_TAG_BUCKET} The name of a field added to metrics records
+ * {@value} The name of a field added to metrics records
* that indicates the hostname portion of the FS URL.
*/
public static final String METRIC_TAG_BUCKET = "bucket";
@@ -192,6 +229,8 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
S3GUARD_METADATASTORE_THROTTLED,
S3GUARD_METADATASTORE_AUTHORITATIVE_DIRECTORIES_UPDATED,
STORE_IO_THROTTLED,
+ STORE_IO_REQUEST,
+ STORE_IO_RETRY,
DELEGATION_TOKENS_ISSUED,
FILES_DELETE_REJECTED
};
@@ -561,9 +600,11 @@ public void decrementGauge(Statistic op, long count) {
/**
* Create a stream input statistics instance.
* @return the new instance
+ * @param filesystemStatistics FS Stats.
*/
- public InputStreamStatistics newInputStreamStatistics() {
- return new InputStreamStatistics();
+ public S3AInputStreamStatistics newInputStreamStatistics(
+ final FileSystem.Statistics filesystemStatistics) {
+ return new InputStreamStatisticsImpl(filesystemStatistics);
}
/**
@@ -579,8 +620,8 @@ public MetastoreInstrumentation getS3GuardInstrumentation() {
* Create a new instance of the committer statistics.
* @return a new committer statistics instance
*/
- CommitterStatistics newCommitterStatistics() {
- return new CommitterStatistics();
+ public CommitterStatistics newCommitterStatistics() {
+ return new CommitterStatisticsImpl();
}
/**
@@ -588,25 +629,27 @@ CommitterStatistics newCommitterStatistics() {
* the filesystem-wide statistics.
* @param statistics stream statistics
*/
- private void mergeInputStreamStatistics(InputStreamStatistics statistics) {
- streamOpenOperations.incr(statistics.openOperations);
- streamCloseOperations.incr(statistics.closeOperations);
- streamClosed.incr(statistics.closed);
- streamAborted.incr(statistics.aborted);
- streamSeekOperations.incr(statistics.seekOperations);
- streamReadExceptions.incr(statistics.readExceptions);
- streamForwardSeekOperations.incr(statistics.forwardSeekOperations);
- streamBytesSkippedOnSeek.incr(statistics.bytesSkippedOnSeek);
- streamBackwardSeekOperations.incr(statistics.backwardSeekOperations);
- streamBytesBackwardsOnSeek.incr(statistics.bytesBackwardsOnSeek);
- streamBytesRead.incr(statistics.bytesRead);
- streamReadOperations.incr(statistics.readOperations);
- streamReadFullyOperations.incr(statistics.readFullyOperations);
- streamReadsIncomplete.incr(statistics.readsIncomplete);
- streamBytesReadInClose.incr(statistics.bytesReadInClose);
- streamBytesDiscardedInAbort.incr(statistics.bytesDiscardedInAbort);
+ private void mergeInputStreamStatistics(
+ InputStreamStatisticsImpl statistics) {
+
+ streamOpenOperations.incr(statistics.getStatistic(STREAM_READ_OPENED));
+ streamCloseOperations.incr(statistics.getCloseOperations());
+ streamClosed.incr(statistics.getClosed());
+ streamAborted.incr(statistics.getAborted());
+ streamSeekOperations.incr(statistics.getSeekOperations());
+ streamReadExceptions.incr(statistics.getReadExceptions());
+ streamForwardSeekOperations.incr(statistics.getForwardSeekOperations());
+ streamBytesSkippedOnSeek.incr(statistics.getBytesSkippedOnSeek());
+ streamBackwardSeekOperations.incr(statistics.getBackwardSeekOperations());
+ streamBytesBackwardsOnSeek.incr(statistics.getBytesBackwardsOnSeek());
+ streamBytesRead.incr(statistics.getBytesRead());
+ streamReadOperations.incr(statistics.getReadOperations());
+ streamReadFullyOperations.incr(statistics.getReadFullyOperations());
+ streamReadsIncomplete.incr(statistics.getReadsIncomplete());
+ streamBytesReadInClose.incr(statistics.getBytesReadInClose());
+ streamBytesDiscardedInAbort.incr(statistics.getBytesDiscardedInAbort());
incrementCounter(STREAM_READ_VERSION_MISMATCHES,
- statistics.versionMismatches.get());
+ statistics.getVersionMismatches());
}
@Override
@@ -622,7 +665,8 @@ public void close() {
throttleRateQuantile.stop();
s3GuardThrottleRateQuantile.stop();
metricsSystem.unregisterSource(metricsSourceName);
- int activeSources = --metricsSourceActiveCounter;
+ metricsSourceActiveCounter--;
+ int activeSources = metricsSourceActiveCounter;
if (activeSources == 0) {
LOG.debug("Shutting down metrics publisher");
metricsSystem.publishMetricsNow();
@@ -634,35 +678,62 @@ public void close() {
/**
* Statistics updated by an input stream during its actual operation.
- * These counters not thread-safe and are for use in a single instance
- * of a stream.
*/
- @InterfaceAudience.Private
- @InterfaceStability.Unstable
- public final class InputStreamStatistics implements AutoCloseable {
- public long openOperations;
- public long closeOperations;
- public long closed;
- public long aborted;
- public long seekOperations;
- public long readExceptions;
- public long forwardSeekOperations;
- public long backwardSeekOperations;
- public long bytesRead;
- public long bytesSkippedOnSeek;
- public long bytesBackwardsOnSeek;
- public long readOperations;
- public long readFullyOperations;
- public long readsIncomplete;
- public long bytesReadInClose;
- public long bytesDiscardedInAbort;
- public long policySetCount;
- public long inputPolicy;
- /** This is atomic so that it can be passed as a reference. */
+ private final class InputStreamStatisticsImpl implements
+ S3AInputStreamStatistics {
+
+ /**
+ * Distance used when incrementing FS stats.
+ */
+ private static final int DISTANCE = 5;
+
+ private final FileSystem.Statistics filesystemStatistics;
+
+ /**
+ * Counters implemented directly in the IOStatistics.
+ */
+ private final CounterIOStatistics statsCounters = counterIOStatistics(
+ StreamStatisticNames.STREAM_READ_ABORTED,
+ StreamStatisticNames.STREAM_READ_BYTES_DISCARDED_ABORT,
+ StreamStatisticNames.STREAM_READ_CLOSED,
+ StreamStatisticNames.STREAM_READ_CLOSE_BYTES_READ,
+ StreamStatisticNames.STREAM_READ_CLOSE_OPERATIONS,
+ StreamStatisticNames.STREAM_READ_OPENED,
+ StreamStatisticNames.STREAM_READ_BYTES,
+ StreamStatisticNames.STREAM_READ_EXCEPTIONS,
+ StreamStatisticNames.STREAM_READ_FULLY_OPERATIONS,
+ StreamStatisticNames.STREAM_READ_OPERATIONS,
+ StreamStatisticNames.STREAM_READ_OPERATIONS_INCOMPLETE,
+ StreamStatisticNames.STREAM_READ_VERSION_MISMATCHES,
+ StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS,
+ StreamStatisticNames.STREAM_READ_SEEK_BACKWARD_OPERATIONS,
+ StreamStatisticNames.STREAM_READ_SEEK_FORWARD_OPERATIONS,
+ StreamStatisticNames.STREAM_READ_SEEK_BYTES_BACKWARDS,
+ StreamStatisticNames.STREAM_READ_SEEK_BYTES_READ,
+ StreamStatisticNames.STREAM_READ_SEEK_BYTES_SKIPPED);
+
+ private final AtomicLong policySetCount= new AtomicLong(0);
+ private volatile long inputPolicy;
private final AtomicLong versionMismatches = new AtomicLong(0);
- private InputStreamStatistics mergedStats;
+ private InputStreamStatisticsImpl mergedStats;
+
+ private InputStreamStatisticsImpl(
+ FileSystem.Statistics filesystemStatistics) {
+ this.filesystemStatistics = filesystemStatistics;
+ }
- private InputStreamStatistics() {
+ private long increment(String name) {
+ return statsCounters.increment(name);
+ }
+
+ private long increment(String name, long value) {
+ return statsCounters.increment(name, value);
+ }
+
+ @Override
+ public Long getStatistic(final String name) {
+ return statsCounters.getStatistic(name)
+ .scalar(IOStatisticEntry.IOSTATISTIC_COUNTER);
}
/**
@@ -670,10 +741,12 @@ private InputStreamStatistics() {
* @param negativeOffset how far was the seek?
* This is expected to be negative.
*/
+ @Override
public void seekBackwards(long negativeOffset) {
- seekOperations++;
- backwardSeekOperations++;
- bytesBackwardsOnSeek -= negativeOffset;
+ increment(StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS);
+ increment(StreamStatisticNames.STREAM_READ_SEEK_BACKWARD_OPERATIONS);
+ increment(StreamStatisticNames.STREAM_READ_SEEK_BYTES_BACKWARDS,
+ -negativeOffset);
}
/**
@@ -682,11 +755,13 @@ public void seekBackwards(long negativeOffset) {
* @param skipped number of bytes skipped by reading from the stream.
* If the seek was implemented by a close + reopen, set this to zero.
*/
+ @Override
public void seekForwards(long skipped) {
- seekOperations++;
- forwardSeekOperations++;
+ increment(StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS);
+ increment(StreamStatisticNames.STREAM_READ_SEEK_FORWARD_OPERATIONS);
if (skipped > 0) {
- bytesSkippedOnSeek += skipped;
+ increment(StreamStatisticNames.STREAM_READ_SEEK_BYTES_SKIPPED,
+ skipped);
}
}
@@ -694,10 +769,9 @@ public void seekForwards(long skipped) {
* The inner stream was opened.
* @return the previous count
*/
+ @Override
public long streamOpened() {
- long count = openOperations;
- openOperations++;
- return count;
+ return increment(STREAM_READ_OPENED);
}
/**
@@ -707,23 +781,27 @@ public long streamOpened() {
* @param remainingInCurrentRequest the number of bytes remaining in
* the current request.
*/
+ @Override
public void streamClose(boolean abortedConnection,
long remainingInCurrentRequest) {
- closeOperations++;
+ increment(StreamStatisticNames.STREAM_READ_CLOSED);
if (abortedConnection) {
- this.aborted++;
- bytesDiscardedInAbort += remainingInCurrentRequest;
+ increment(StreamStatisticNames.STREAM_READ_ABORTED);
+ increment(StreamStatisticNames.STREAM_READ_BYTES_DISCARDED_ABORT,
+ remainingInCurrentRequest);
} else {
- closed++;
- bytesReadInClose += remainingInCurrentRequest;
+ increment(StreamStatisticNames.STREAM_READ_CLOSED);
+ increment(StreamStatisticNames.STREAM_READ_CLOSE_BYTES_READ,
+ remainingInCurrentRequest);
}
}
/**
* An ignored stream read exception was received.
*/
+ @Override
public void readException() {
- readExceptions++;
+ increment(StreamStatisticNames.STREAM_READ_EXCEPTIONS);
}
/**
@@ -731,9 +809,10 @@ public void readException() {
* no-op if the argument is negative.
* @param bytes number of bytes read
*/
+ @Override
public void bytesRead(long bytes) {
if (bytes > 0) {
- bytesRead += bytes;
+ increment(StreamStatisticNames.STREAM_READ_BYTES, bytes);
}
}
@@ -742,8 +821,9 @@ public void bytesRead(long bytes) {
* @param pos starting position of the read
* @param len length of bytes to read
*/
+ @Override
public void readOperationStarted(long pos, long len) {
- readOperations++;
+ increment(StreamStatisticNames.STREAM_READ_OPERATIONS);
}
/**
@@ -752,8 +832,9 @@ public void readOperationStarted(long pos, long len) {
* @param pos starting position of the read
* @param len length of bytes to read
*/
+ @Override
public void readFullyOperationStarted(long pos, long len) {
- readFullyOperations++;
+ increment(StreamStatisticNames.STREAM_READ_FULLY_OPERATIONS);
}
/**
@@ -761,9 +842,10 @@ public void readFullyOperationStarted(long pos, long len) {
* @param requested number of requested bytes
* @param actual the actual number of bytes
*/
+ @Override
public void readOperationCompleted(int requested, int actual) {
if (requested > actual) {
- readsIncomplete++;
+ increment(StreamStatisticNames.STREAM_READ_OPERATIONS_INCOMPLETE);
}
}
@@ -780,17 +862,20 @@ public void close() {
* The input policy has been switched.
* @param updatedPolicy enum value of new policy.
*/
+ @Override
public void inputPolicySet(int updatedPolicy) {
- policySetCount++;
+ policySetCount.incrementAndGet();
inputPolicy = updatedPolicy;
}
/**
- * Get a reference to the version mismatch counter.
- * @return a counter which can be incremented.
+ * The change tracker increments {@code versionMismatches} on any
+ * mismatch.
+ * @return change tracking.
*/
- public AtomicLong getVersionMismatchCounter() {
- return versionMismatches;
+ @Override
+ public ChangeTrackerStatistics getChangeTrackerStatistics() {
+ return new CountingChangeTracker(versionMismatches);
}
/**
@@ -804,26 +889,7 @@ public AtomicLong getVersionMismatchCounter() {
public String toString() {
final StringBuilder sb = new StringBuilder(
"StreamStatistics{");
- sb.append("OpenOperations=").append(openOperations);
- sb.append(", CloseOperations=").append(closeOperations);
- sb.append(", Closed=").append(closed);
- sb.append(", Aborted=").append(aborted);
- sb.append(", SeekOperations=").append(seekOperations);
- sb.append(", ReadExceptions=").append(readExceptions);
- sb.append(", ForwardSeekOperations=")
- .append(forwardSeekOperations);
- sb.append(", BackwardSeekOperations=")
- .append(backwardSeekOperations);
- sb.append(", BytesSkippedOnSeek=").append(bytesSkippedOnSeek);
- sb.append(", BytesBackwardsOnSeek=").append(bytesBackwardsOnSeek);
- sb.append(", BytesRead=").append(bytesRead);
- sb.append(", BytesRead excluding skipped=")
- .append(bytesRead - bytesSkippedOnSeek);
- sb.append(", ReadOperations=").append(readOperations);
- sb.append(", ReadFullyOperations=").append(readFullyOperations);
- sb.append(", ReadsIncomplete=").append(readsIncomplete);
- sb.append(", BytesReadInClose=").append(bytesReadInClose);
- sb.append(", BytesDiscardedInAbort=").append(bytesDiscardedInAbort);
+ sb.append(IOStatisticsLogging.iostatisticsToString(statsCounters));
sb.append(", InputPolicy=").append(inputPolicy);
sb.append(", InputPolicySetCount=").append(policySetCount);
sb.append(", versionMismatches=").append(versionMismatches.get());
@@ -836,50 +902,56 @@ public String toString() {
* Takes a diff between the current version of the stats and the
* version of the stats when merge was last called, and merges the diff
* into the instrumentation instance. Used to periodically merge the
- * stats into the fs-wide stats. Behavior is undefined if called on a
- * closed instance.
+ * stats into the fs-wide stats.
+ *
+ * Behavior is undefined if called on a closed instance.
*/
- void merge(boolean isClosed) {
+ @Override
+ public void merge(boolean isClosed) {
if (mergedStats != null) {
- mergeInputStreamStatistics(diff(mergedStats));
+ mergeInputStreamStatistics(setd(mergedStats));
} else {
mergeInputStreamStatistics(this);
}
// If stats are closed, no need to create another copy
if (!isClosed) {
mergedStats = copy();
+ } else {
+ // stream is being closed.
+ // increment the filesystem statistics for this thread.
+ if (filesystemStatistics != null) {
+ filesystemStatistics.incrementBytesReadByDistance(DISTANCE,
+ getBytesRead() - getBytesReadInClose());
+ }
}
}
/**
- * Returns a diff between this {@link InputStreamStatistics} instance and
- * the given {@link InputStreamStatistics} instance.
+ * Set the dest variable to the difference of the two
+ * other values.
+ * @param dest destination
+ * @param l left side
+ * @param r right side
+ */
+ private void setd(AtomicLong dest, AtomicLong l, AtomicLong r) {
+ dest.set(l.get() - r.get());
+ }
+
+ /**
+ * Returns a diff between this {@link InputStreamStatisticsImpl} instance
+ * and the given {@link InputStreamStatisticsImpl} instance.
*/
- private InputStreamStatistics diff(InputStreamStatistics inputStats) {
- InputStreamStatistics diff = new InputStreamStatistics();
- diff.openOperations = openOperations - inputStats.openOperations;
- diff.closeOperations = closeOperations - inputStats.closeOperations;
- diff.closed = closed - inputStats.closed;
- diff.aborted = aborted - inputStats.aborted;
- diff.seekOperations = seekOperations - inputStats.seekOperations;
- diff.readExceptions = readExceptions - inputStats.readExceptions;
- diff.forwardSeekOperations =
- forwardSeekOperations - inputStats.forwardSeekOperations;
- diff.backwardSeekOperations =
- backwardSeekOperations - inputStats.backwardSeekOperations;
- diff.bytesRead = bytesRead - inputStats.bytesRead;
- diff.bytesSkippedOnSeek =
- bytesSkippedOnSeek - inputStats.bytesSkippedOnSeek;
- diff.bytesBackwardsOnSeek =
- bytesBackwardsOnSeek - inputStats.bytesBackwardsOnSeek;
- diff.readOperations = readOperations - inputStats.readOperations;
- diff.readFullyOperations =
- readFullyOperations - inputStats.readFullyOperations;
- diff.readsIncomplete = readsIncomplete - inputStats.readsIncomplete;
- diff.bytesReadInClose = bytesReadInClose - inputStats.bytesReadInClose;
- diff.bytesDiscardedInAbort =
- bytesDiscardedInAbort - inputStats.bytesDiscardedInAbort;
- diff.policySetCount = policySetCount - inputStats.policySetCount;
+ private InputStreamStatisticsImpl setd(
+ InputStreamStatisticsImpl inputStats) {
+ InputStreamStatisticsImpl diff =
+ new InputStreamStatisticsImpl(filesystemStatistics);
+
+ // build the io stats diff
+ diff.statsCounters.copy(statsCounters);
+ diff.statsCounters.subtract(inputStats.statsCounters);
+
+ setd(diff.policySetCount, policySetCount,
+ inputStats.policySetCount);
diff.inputPolicy = inputPolicy - inputStats.inputPolicy;
diff.versionMismatches.set(versionMismatches.longValue() -
inputStats.versionMismatches.longValue());
@@ -887,62 +959,161 @@ private InputStreamStatistics diff(InputStreamStatistics inputStats) {
}
/**
- * Returns a new {@link InputStreamStatistics} instance with all the same
- * values as this {@link InputStreamStatistics}.
+ * Returns a new {@link InputStreamStatisticsImpl} instance with
+ * all the same values as this {@link InputStreamStatisticsImpl}.
*/
- private InputStreamStatistics copy() {
- InputStreamStatistics copy = new InputStreamStatistics();
- copy.openOperations = openOperations;
- copy.closeOperations = closeOperations;
- copy.closed = closed;
- copy.aborted = aborted;
- copy.seekOperations = seekOperations;
- copy.readExceptions = readExceptions;
- copy.forwardSeekOperations = forwardSeekOperations;
- copy.backwardSeekOperations = backwardSeekOperations;
- copy.bytesRead = bytesRead;
- copy.bytesSkippedOnSeek = bytesSkippedOnSeek;
- copy.bytesBackwardsOnSeek = bytesBackwardsOnSeek;
- copy.readOperations = readOperations;
- copy.readFullyOperations = readFullyOperations;
- copy.readsIncomplete = readsIncomplete;
- copy.bytesReadInClose = bytesReadInClose;
- copy.bytesDiscardedInAbort = bytesDiscardedInAbort;
- copy.policySetCount = policySetCount;
+ private InputStreamStatisticsImpl copy() {
+ InputStreamStatisticsImpl copy =
+ new InputStreamStatisticsImpl(filesystemStatistics);
+
+ copy.policySetCount.set(policySetCount.get());
+ copy.versionMismatches.set(versionMismatches.get());
copy.inputPolicy = inputPolicy;
return copy;
}
+
+ /**
+ * Create an IOStatistics instance which is dynamically updated.
+ * @return statistics
+ */
+ @Override
+ public IOStatistics getIOStatistics() {
+ return IOStatisticsSupport.snapshot(statsCounters);
+ }
+
+ @Override
+ public long getCloseOperations() {
+ return getStatistic(StreamStatisticNames.STREAM_READ_CLOSE_OPERATIONS);
+ }
+
+ @Override
+ public long getClosed() {
+ return getStatistic(StreamStatisticNames.STREAM_READ_CLOSED);
+ }
+
+ @Override
+ public long getAborted() {
+ return getStatistic(StreamStatisticNames.STREAM_READ_ABORTED);
+ }
+
+ @Override
+ public long getForwardSeekOperations() {
+ return getStatistic(
+ StreamStatisticNames.STREAM_READ_SEEK_FORWARD_OPERATIONS);
+ }
+
+ @Override
+ public long getBackwardSeekOperations() {
+ return getStatistic(
+ StreamStatisticNames.STREAM_READ_SEEK_BACKWARD_OPERATIONS);
+ }
+
+ @Override
+ public long getBytesRead() {
+ return getStatistic(StreamStatisticNames.STREAM_READ_BYTES);
+ }
+
+ @Override
+ public long getBytesSkippedOnSeek() {
+ return getStatistic(StreamStatisticNames.STREAM_READ_SEEK_BYTES_SKIPPED);
+ }
+
+ @Override
+ public long getBytesBackwardsOnSeek() {
+ return getStatistic(
+ StreamStatisticNames.STREAM_READ_SEEK_BYTES_BACKWARDS);
+ }
+
+ @Override
+ public long getBytesReadInClose() {
+ return getStatistic(StreamStatisticNames.STREAM_READ_CLOSE_BYTES_READ);
+ }
+
+ @Override
+ public long getBytesDiscardedInAbort() {
+ return getStatistic(
+ StreamStatisticNames.STREAM_READ_BYTES_DISCARDED_ABORT);
+ }
+
+ @Override
+ public long getOpenOperations() {
+ return getStatistic(STREAM_READ_OPENED);
+ }
+
+ @Override
+ public long getSeekOperations() {
+ return getStatistic(StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS);
+ }
+
+ @Override
+ public long getReadExceptions() {
+ return getStatistic(StreamStatisticNames.STREAM_READ_EXCEPTIONS);
+ }
+
+ @Override
+ public long getReadOperations() {
+ return getStatistic(StreamStatisticNames.STREAM_READ_CLOSED);
+ }
+
+ @Override
+ public long getReadFullyOperations() {
+ return getStatistic(StreamStatisticNames.STREAM_READ_CLOSED);
+ }
+
+ @Override
+ public long getReadsIncomplete() {
+ return getStatistic(StreamStatisticNames.STREAM_READ_CLOSED);
+ }
+
+ @Override
+ public long getPolicySetCount() {
+ return getStatistic(StreamStatisticNames.STREAM_READ_CLOSED);
+ }
+
+ @Override
+ public long getVersionMismatches() {
+ return versionMismatches.get();
+ }
+
+ @Override
+ public long getInputPolicy() {
+ return inputPolicy;
+ }
}
/**
* Create a stream output statistics instance.
* @return the new instance
*/
- OutputStreamStatistics newOutputStreamStatistics(Statistics statistics) {
- return new OutputStreamStatistics(statistics);
+ public BlockOutputStreamStatistics newOutputStreamStatistics(
+ FileSystem.Statistics filesystemStatistics) {
+ return new BlockOutputStreamStatisticsImpl(filesystemStatistics);
}
/**
* Merge in the statistics of a single output stream into
* the filesystem-wide statistics.
- * @param statistics stream statistics
+ * @param source stream statistics
*/
- private void mergeOutputStreamStatistics(OutputStreamStatistics statistics) {
- incrementCounter(STREAM_WRITE_TOTAL_TIME, statistics.totalUploadDuration());
- incrementCounter(STREAM_WRITE_QUEUE_DURATION, statistics.queueDuration);
- incrementCounter(STREAM_WRITE_TOTAL_DATA, statistics.bytesUploaded);
+ private void mergeOutputStreamStatistics(
+ BlockOutputStreamStatisticsImpl source) {
+ incrementCounter(STREAM_WRITE_TOTAL_TIME, source.totalUploadDuration());
+ incrementCounter(STREAM_WRITE_QUEUE_DURATION, source.queueDuration);
+ incrementCounter(STREAM_WRITE_TOTAL_DATA, source.bytesUploaded);
incrementCounter(STREAM_WRITE_BLOCK_UPLOADS,
- statistics.blockUploadsCompleted);
+ source.blockUploadsCompleted);
+ incrementCounter(STREAM_WRITE_FAILURES, source.blockUploadsFailed);
}
/**
* Statistics updated by an output stream during its actual operation.
- * Some of these stats may be relayed. However, as block upload is
- * spans multiple
+ *