From 8278a5f8675118814cea1ffd6d24ffe0b2c0d09a Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 31 Dec 2020 11:52:42 +0000 Subject: [PATCH 1/4] HADOOP-17450. Add Public IOStatistics API. (#2577) This is the API and implementation classes of HADOOP-16830, which allows callers to query IO object instances (filesystems, streams, remote iterators, ...) and other classes for statistics on their I/O Usage: operation count and min/max/mean durations. New Packages org.apache.hadoop.fs.statistics. Public API, including: IOStatisticsSource IOStatistics IOStatisticsSnapshot (seralizable to java objects and json) +helper classes for logging and integration BufferedIOStatisticsInputStream implements IOStatisticsSource and StreamCapabilities BufferedIOStatisticsOutputStream implements IOStatisticsSource, Syncable and StreamCapabilities org.apache.hadoop.fs.statistics.impl Implementation classes for internal use. org.apache.hadoop.util.functional functional programming support for RemoteIterators and other operations which raise IOEs; all wrapper classes implement and propagate IOStatisticsSource Contributed by Steve Loughran. Change-Id: If56e8db2981613ff689c39239135e44feb25f78e --- .../hadoop/crypto/CryptoInputStream.java | 14 +- .../hadoop/crypto/CryptoOutputStream.java | 11 +- .../hadoop/fs/BufferedFSInputStream.java | 29 +- .../apache/hadoop/fs/ChecksumFileSystem.java | 42 +- .../apache/hadoop/fs/FSDataInputStream.java | 16 +- .../apache/hadoop/fs/FSDataOutputStream.java | 17 +- .../org/apache/hadoop/fs/FSInputStream.java | 22 + .../apache/hadoop/fs/MultipartUploader.java | 8 +- .../apache/hadoop/fs/RawLocalFileSystem.java | 96 ++- .../apache/hadoop/fs/StorageStatistics.java | 71 +- .../apache/hadoop/fs/StreamCapabilities.java | 5 + .../hadoop/fs/impl/FunctionsRaisingIOE.java | 6 +- .../hadoop/fs/impl/FutureIOSupport.java | 67 +- .../hadoop/fs/impl/WrappedIOException.java | 5 +- .../BufferedIOStatisticsInputStream.java | 85 +++ .../BufferedIOStatisticsOutputStream.java | 157 ++++ .../statistics/DurationStatisticSummary.java | 154 ++++ .../hadoop/fs/statistics/DurationTracker.java | 54 ++ .../fs/statistics/DurationTrackerFactory.java | 57 ++ .../hadoop/fs/statistics/IOStatistics.java | 78 ++ .../fs/statistics/IOStatisticsAggregator.java | 48 ++ .../fs/statistics/IOStatisticsLogging.java | 301 ++++++++ .../fs/statistics/IOStatisticsSnapshot.java | 285 +++++++ .../fs/statistics/IOStatisticsSource.java | 47 ++ .../fs/statistics/IOStatisticsSupport.java | 107 +++ .../hadoop/fs/statistics/MeanStatistic.java | 290 ++++++++ .../fs/statistics/StoreStatisticNames.java | 337 +++++++++ .../fs/statistics/StreamStatisticNames.java | 292 ++++++++ .../impl/AbstractIOStatisticsImpl.java | 30 + .../statistics/impl/DynamicIOStatistics.java | 132 ++++ .../impl/DynamicIOStatisticsBuilder.java | 248 +++++++ .../fs/statistics/impl/EmptyIOStatistics.java | 74 ++ .../impl/EvaluatingStatisticsMap.java | 202 +++++ .../statistics/impl/IOStatisticsBinding.java | 616 ++++++++++++++++ .../fs/statistics/impl/IOStatisticsStore.java | 258 +++++++ .../impl/IOStatisticsStoreBuilder.java | 75 ++ .../impl/IOStatisticsStoreBuilderImpl.java | 100 +++ .../impl/IOStatisticsStoreImpl.java | 469 ++++++++++++ .../impl/PairedDurationTrackerFactory.java | 93 +++ .../impl/SourceWrappedStatistics.java | 44 ++ .../impl/StatisticDurationTracker.java | 106 +++ .../StorageStatisticsFromIOStatistics.java | 98 +++ .../statistics/impl/StubDurationTracker.java | 51 ++ .../impl/StubDurationTrackerFactory.java | 44 ++ .../statistics/impl/WrappedIOStatistics.java | 108 +++ .../fs/statistics/impl/package-info.java | 31 + .../hadoop/fs/statistics/package-info.java | 134 ++++ .../io/compress/CompressionInputStream.java | 18 +- .../io/compress/CompressionOutputStream.java | 14 +- .../org/apache/hadoop/util/LineReader.java | 16 +- .../apache/hadoop/util/OperationDuration.java | 52 +- .../util/functional/BiFunctionRaisingIOE.java | 40 + .../util/functional/CallableRaisingIOE.java | 36 + .../util/functional/ConsumerRaisingIOE.java | 51 ++ .../util/functional/FunctionRaisingIOE.java | 38 + .../hadoop/util/functional/FutureIO.java | 188 +++++ .../util/functional/InvocationRaisingIOE.java | 42 ++ .../util/functional/RemoteIterators.java | 698 ++++++++++++++++++ .../hadoop/util/functional/package-info.java | 41 + .../src/site/markdown/filesystem/index.md | 1 + .../site/markdown/filesystem/iostatistics.md | 432 +++++++++++ ...AbstractContractMultipartUploaderTest.java | 3 + ...bstractContractStreamIOStatisticsTest.java | 313 ++++++++ ...TestLocalFSContractStreamIOStatistics.java | 80 ++ .../fs/statistics/IOStatisticAssertions.java | 528 +++++++++++++ .../fs/statistics/TestDurationTracking.java | 361 +++++++++ .../statistics/TestDynamicIOStatistics.java | 311 ++++++++ .../fs/statistics/TestEmptyIOStatistics.java | 110 +++ .../statistics/TestIOStatisticsSnapshot.java | 147 ++++ .../fs/statistics/TestIOStatisticsStore.java | 177 +++++ .../fs/statistics/TestMeanStatistic.java | 219 ++++++ .../util/functional/TestRemoteIterators.java | 469 ++++++++++++ 72 files changed, 9890 insertions(+), 109 deletions(-) create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/BufferedIOStatisticsInputStream.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/BufferedIOStatisticsOutputStream.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/DurationStatisticSummary.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/DurationTracker.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/DurationTrackerFactory.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatistics.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsAggregator.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsLogging.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSnapshot.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSource.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSupport.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/MeanStatistic.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StreamStatisticNames.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/AbstractIOStatisticsImpl.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/DynamicIOStatistics.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/DynamicIOStatisticsBuilder.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/EmptyIOStatistics.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/EvaluatingStatisticsMap.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStore.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStoreBuilder.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStoreBuilderImpl.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStoreImpl.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/PairedDurationTrackerFactory.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/SourceWrappedStatistics.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StatisticDurationTracker.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StorageStatisticsFromIOStatistics.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StubDurationTracker.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StubDurationTrackerFactory.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/WrappedIOStatistics.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/package-info.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/package-info.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/BiFunctionRaisingIOE.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/CallableRaisingIOE.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/ConsumerRaisingIOE.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/FunctionRaisingIOE.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/FutureIO.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/InvocationRaisingIOE.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RemoteIterators.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/package-info.java create mode 100644 hadoop-common-project/hadoop-common/src/site/markdown/filesystem/iostatistics.md create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractStreamIOStatisticsTest.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractStreamIOStatistics.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/IOStatisticAssertions.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestDurationTracking.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestDynamicIOStatistics.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestEmptyIOStatistics.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestIOStatisticsSnapshot.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestIOStatisticsStore.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestMeanStatistic.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/functional/TestRemoteIterators.java diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoInputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoInputStream.java index 30817a2a62529..0c156e3548d21 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoInputStream.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoInputStream.java @@ -46,9 +46,13 @@ import org.apache.hadoop.fs.Seekable; import org.apache.hadoop.fs.StreamCapabilities; import org.apache.hadoop.fs.StreamCapabilitiesPolicy; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; import org.apache.hadoop.io.ByteBufferPool; import org.apache.hadoop.util.StringUtils; +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics; + /** * CryptoInputStream decrypts data. It is not thread-safe. AES CTR mode is * required in order to ensure that the plain text and cipher text have a 1:1 @@ -66,7 +70,7 @@ public class CryptoInputStream extends FilterInputStream implements Seekable, PositionedReadable, ByteBufferReadable, HasFileDescriptor, CanSetDropBehind, CanSetReadahead, HasEnhancedByteBufferAccess, ReadableByteChannel, CanUnbuffer, StreamCapabilities, - ByteBufferPositionedReadable { + ByteBufferPositionedReadable, IOStatisticsSource { private final byte[] oneByteBuf = new byte[1]; private final CryptoCodec codec; private final Decryptor decryptor; @@ -867,8 +871,16 @@ public boolean hasCapability(String capability) { + " does not expose its stream capabilities."); } return ((StreamCapabilities) in).hasCapability(capability); + case StreamCapabilities.IOSTATISTICS: + return (in instanceof StreamCapabilities) + && ((StreamCapabilities) in).hasCapability(capability); default: return false; } } + + @Override + public IOStatistics getIOStatistics() { + return retrieveIOStatistics(in); + } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java index aeb6e4d0ed2ef..553915d755f6b 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java @@ -28,9 +28,13 @@ import org.apache.hadoop.fs.CanSetDropBehind; import org.apache.hadoop.fs.StreamCapabilities; import org.apache.hadoop.fs.Syncable; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics; + /** * CryptoOutputStream encrypts data. It is not thread-safe. AES CTR mode is * required in order to ensure that the plain text and cipher text have a 1:1 @@ -48,7 +52,7 @@ @InterfaceAudience.Private @InterfaceStability.Evolving public class CryptoOutputStream extends FilterOutputStream implements - Syncable, CanSetDropBehind, StreamCapabilities { + Syncable, CanSetDropBehind, StreamCapabilities, IOStatisticsSource { private final byte[] oneByteBuf = new byte[1]; private final CryptoCodec codec; private final Encryptor encryptor; @@ -313,4 +317,9 @@ public boolean hasCapability(String capability) { } return false; } + + @Override + public IOStatistics getIOStatistics() { + return retrieveIOStatistics(out); + } } 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..0c5b4f0d3745a 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, StreamCapabilities { /** * Creates a BufferedFSInputStream * with the specified buffer size, @@ -126,4 +131,26 @@ public FileDescriptor getFileDescriptor() throws IOException { return null; } } + + /** + * If the inner stream supports {@link StreamCapabilities}, + * forward the probe to it. + * Otherwise: return false. + * + * @param capability string to query the stream support for. + * @return true if a capability is known to be supported. + */ + @Override + public boolean hasCapability(final String capability) { + if (in instanceof StreamCapabilities) { + return ((StreamCapabilities) in).hasCapability(capability); + } else { + return false; + } + } + + @Override + public IOStatistics getIOStatistics() { + return retrieveIOStatistics(in); + } } 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 f081742ce59db..b24136bf9ec44 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, StreamCapabilities { private FSDataOutputStream datas; private FSDataOutputStream sums; private static final float CHKSUM_AS_FRACTION = 0.01f; @@ -449,6 +465,28 @@ protected void checkClosed() throws IOException { throw new ClosedChannelException(); } } + + /** + * Get the IO Statistics of the nested stream, falling back to + * null if the stream does not implement the interface + * {@link IOStatisticsSource}. + * @return an IOStatistics instance or null + */ + @Override + public IOStatistics getIOStatistics() { + return IOStatisticsSupport.retrieveIOStatistics(datas); + } + + /** + * Probe the inner stream for a capability. + * + * @param capability string to query the stream support for. + * @return true if a capability is known to be supported. + */ + @Override + public boolean hasCapability(final String capability) { + return datas.hasCapability(capability); + } } @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 b3b3fac0c09e1..ad2642f7db963 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSInputStream.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSInputStream.java @@ -24,6 +24,9 @@ import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.statistics.IOStatisticsLogging; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -134,4 +137,23 @@ public void readFully(long position, byte[] buffer) throws IOException { readFully(position, buffer, 0, buffer.length); } + + /** + * toString method returns the superclass toString, but if the subclass + * implements {@link IOStatisticsSource} then those statistics are + * extracted and included in the output. + * That is: statistics of subclasses are automatically reported. + * @return a string value. + */ + @Override + public String toString() { + final StringBuilder sb = new StringBuilder(super.toString()); + sb.append('{'); + if (this instanceof IOStatisticsSource) { + sb.append(IOStatisticsLogging.ioStatisticsSourceToString( + (IOStatisticsSource) this)); + } + sb.append('}'); + return sb.toString(); + } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MultipartUploader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MultipartUploader.java index 89848dc29ded0..dcb76b50b3429 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MultipartUploader.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MultipartUploader.java @@ -26,14 +26,20 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; /** * MultipartUploader is an interface for copying files multipart and across * multiple nodes. + *

+ * The interface extends {@link IOStatisticsSource} so that there is no + * need to cast an instance to see if is a source of statistics. + * However, implementations MAY return null for their actual statistics. */ @InterfaceAudience.Public @InterfaceStability.Unstable -public interface MultipartUploader extends Closeable { +public interface MultipartUploader extends Closeable, + IOStatisticsSource { /** 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 ba29f74cc5ca4..2aeb17b90c0e6 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 @@ -40,6 +40,7 @@ import java.nio.file.attribute.FileTime; import java.util.Arrays; import java.util.EnumSet; +import java.util.Locale; import java.util.Optional; import java.util.StringTokenizer; @@ -47,6 +48,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.BufferedIOStatisticsOutputStream; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.nativeio.NativeIO; import org.apache.hadoop.util.Progressable; @@ -54,6 +59,14 @@ import org.apache.hadoop.util.StringUtils; import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BYTES; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_EXCEPTIONS; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_SKIP_BYTES; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_SKIP_OPERATIONS; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_WRITE_BYTES; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_WRITE_EXCEPTIONS; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore; /**************************************************************** * Implement the FileSystem API for the raw local filesystem. @@ -107,10 +120,23 @@ 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, StreamCapabilities { private FileInputStream fis; private long position; + /** + * Minimal set of counters. + */ + private final IOStatisticsStore ioStatistics = iostatisticsStore() + .withCounters( + STREAM_READ_BYTES, + STREAM_READ_EXCEPTIONS, + STREAM_READ_SEEK_OPERATIONS, + STREAM_READ_SKIP_OPERATIONS, + STREAM_READ_SKIP_BYTES) + .build(); + public LocalFSFileInputStream(Path f) throws IOException { fis = new FileInputStream(pathToFile(f)); } @@ -152,9 +178,11 @@ public int read() throws IOException { if (value >= 0) { this.position++; statistics.incrementBytesRead(1); + ioStatistics.incrementCounter(STREAM_READ_BYTES); } return value; } catch (IOException e) { // unexpected exception + ioStatistics.incrementCounter(STREAM_READ_EXCEPTIONS); throw new FSError(e); // assume native fs error } } @@ -168,9 +196,11 @@ public int read(byte[] b, int off, int len) throws IOException { if (value > 0) { this.position += value; statistics.incrementBytesRead(value); + ioStatistics.incrementCounter(STREAM_READ_BYTES, value); } return value; } catch (IOException e) { // unexpected exception + ioStatistics.incrementCounter(STREAM_READ_EXCEPTIONS); throw new FSError(e); // assume native fs error } } @@ -189,18 +219,22 @@ public int read(long position, byte[] b, int off, int len) int value = fis.getChannel().read(bb, position); if (value > 0) { statistics.incrementBytesRead(value); + ioStatistics.incrementCounter(STREAM_READ_BYTES, value); } return value; } catch (IOException e) { + ioStatistics.incrementCounter(STREAM_READ_EXCEPTIONS); throw new FSError(e); } } @Override public long skip(long n) throws IOException { + ioStatistics.incrementCounter(STREAM_READ_SKIP_OPERATIONS); long value = fis.skip(n); if (value > 0) { this.position += value; + ioStatistics.incrementCounter(STREAM_READ_SKIP_BYTES, value); } return value; } @@ -209,6 +243,23 @@ public long skip(long n) throws IOException { public FileDescriptor getFileDescriptor() throws IOException { return fis.getFD(); } + + @Override + public boolean hasCapability(String capability) { + // a bit inefficient, but intended to make it easier to add + // new capabilities. + switch (capability.toLowerCase(Locale.ENGLISH)) { + case StreamCapabilities.IOSTATISTICS: + return true; + default: + return false; + } + } + + @Override + public IOStatistics getIOStatistics() { + return ioStatistics; + } } @Override @@ -233,9 +284,19 @@ public FSDataInputStream open(PathHandle fd, int bufferSize) /********************************************************* * For create()'s FSOutputStream. *********************************************************/ - class LocalFSFileOutputStream extends OutputStream { + final class LocalFSFileOutputStream extends OutputStream implements + IOStatisticsSource, StreamCapabilities { private FileOutputStream fos; - + + /** + * Minimal set of counters. + */ + private final IOStatisticsStore ioStatistics = iostatisticsStore() + .withCounters( + STREAM_WRITE_BYTES, + STREAM_WRITE_EXCEPTIONS) + .build(); + private LocalFSFileOutputStream(Path f, boolean append, FsPermission permission) throws IOException { File file = pathToFile(f); @@ -275,7 +336,9 @@ private LocalFSFileOutputStream(Path f, boolean append, public void write(byte[] b, int off, int len) throws IOException { try { fos.write(b, off, len); + ioStatistics.incrementCounter(STREAM_WRITE_BYTES, len); } catch (IOException e) { // unexpected exception + ioStatistics.incrementCounter(STREAM_WRITE_EXCEPTIONS); throw new FSError(e); // assume native fs error } } @@ -284,10 +347,29 @@ public void write(byte[] b, int off, int len) throws IOException { public void write(int b) throws IOException { try { fos.write(b); + ioStatistics.incrementCounter(STREAM_WRITE_BYTES); } catch (IOException e) { // unexpected exception + ioStatistics.incrementCounter(STREAM_WRITE_EXCEPTIONS); throw new FSError(e); // assume native fs error } } + + @Override + public boolean hasCapability(String capability) { + // a bit inefficient, but intended to make it easier to add + // new capabilities. + switch (capability.toLowerCase(Locale.ENGLISH)) { + case StreamCapabilities.IOSTATISTICS: + return true; + default: + return false; + } + } + + @Override + public IOStatistics getIOStatistics() { + return ioStatistics; + } } @Override @@ -320,8 +402,8 @@ private FSDataOutputStream create(Path f, boolean overwrite, if (parent != null && !mkdirs(parent)) { throw new IOException("Mkdirs failed to create " + parent.toString()); } - return new FSDataOutputStream(new BufferedOutputStream( - createOutputStreamWithMode(f, false, permission), bufferSize), + return new FSDataOutputStream(new BufferedIOStatisticsOutputStream( + createOutputStreamWithMode(f, false, permission), bufferSize, true), statistics); } @@ -342,8 +424,8 @@ public FSDataOutputStream createNonRecursive(Path f, FsPermission permission, if (exists(f) && !flags.contains(CreateFlag.OVERWRITE)) { throw new FileAlreadyExistsException("File already exists: " + f); } - return new FSDataOutputStream(new BufferedOutputStream( - createOutputStreamWithMode(f, false, permission), bufferSize), + return new FSDataOutputStream(new BufferedIOStatisticsOutputStream( + createOutputStreamWithMode(f, false, permission), bufferSize, true), statistics); } 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: *

+ * See {@link StoreStatisticNames} for the field names used here + * and elsewhere. */ @InterfaceStability.Evolving public interface CommonStatisticNames { // The following names are for file system operation invocations - String OP_APPEND = "op_append"; - String OP_COPY_FROM_LOCAL_FILE = "op_copy_from_local_file"; - String OP_CREATE = "op_create"; - String OP_CREATE_NON_RECURSIVE = "op_create_non_recursive"; - String OP_DELETE = "op_delete"; - String OP_EXISTS = "op_exists"; - String OP_GET_CONTENT_SUMMARY = "op_get_content_summary"; - String OP_GET_DELEGATION_TOKEN = "op_get_delegation_token"; - String OP_GET_FILE_CHECKSUM = "op_get_file_checksum"; - String OP_GET_FILE_STATUS = "op_get_file_status"; - String OP_GET_STATUS = "op_get_status"; - String OP_GLOB_STATUS = "op_glob_status"; - String OP_IS_FILE = "op_is_file"; - String OP_IS_DIRECTORY = "op_is_directory"; - String OP_LIST_FILES = "op_list_files"; - String OP_LIST_LOCATED_STATUS = "op_list_located_status"; - String OP_LIST_STATUS = "op_list_status"; - String OP_MKDIRS = "op_mkdirs"; - String OP_MODIFY_ACL_ENTRIES = "op_modify_acl_entries"; - String OP_OPEN = "op_open"; - String OP_REMOVE_ACL = "op_remove_acl"; - String OP_REMOVE_ACL_ENTRIES = "op_remove_acl_entries"; - String OP_REMOVE_DEFAULT_ACL = "op_remove_default_acl"; - String OP_RENAME = "op_rename"; - String OP_SET_ACL = "op_set_acl"; - String OP_SET_OWNER = "op_set_owner"; - String OP_SET_PERMISSION = "op_set_permission"; - String OP_SET_TIMES = "op_set_times"; - String OP_TRUNCATE = "op_truncate"; + String OP_APPEND = StoreStatisticNames.OP_APPEND; + String OP_COPY_FROM_LOCAL_FILE = + StoreStatisticNames.OP_COPY_FROM_LOCAL_FILE; + String OP_CREATE = StoreStatisticNames.OP_CREATE; + String OP_CREATE_NON_RECURSIVE = + StoreStatisticNames.OP_CREATE_NON_RECURSIVE; + String OP_DELETE = StoreStatisticNames.OP_DELETE; + String OP_EXISTS = StoreStatisticNames.OP_EXISTS; + String OP_GET_CONTENT_SUMMARY = + StoreStatisticNames.OP_GET_CONTENT_SUMMARY; + String OP_GET_DELEGATION_TOKEN = + StoreStatisticNames.OP_GET_DELEGATION_TOKEN; + String OP_GET_FILE_CHECKSUM = StoreStatisticNames.OP_GET_FILE_CHECKSUM; + String OP_GET_FILE_STATUS = StoreStatisticNames.OP_GET_FILE_STATUS; + String OP_GET_STATUS = StoreStatisticNames.OP_GET_STATUS; + String OP_GLOB_STATUS = StoreStatisticNames.OP_GLOB_STATUS; + String OP_IS_FILE = StoreStatisticNames.OP_IS_FILE; + String OP_IS_DIRECTORY = StoreStatisticNames.OP_IS_DIRECTORY; + String OP_LIST_FILES = StoreStatisticNames.OP_LIST_FILES; + String OP_LIST_LOCATED_STATUS = + StoreStatisticNames.OP_LIST_LOCATED_STATUS; + String OP_LIST_STATUS = StoreStatisticNames.OP_LIST_STATUS; + String OP_MKDIRS = StoreStatisticNames.OP_MKDIRS; + String OP_MODIFY_ACL_ENTRIES = StoreStatisticNames.OP_MODIFY_ACL_ENTRIES; + String OP_OPEN = StoreStatisticNames.OP_OPEN; + String OP_REMOVE_ACL = StoreStatisticNames.OP_REMOVE_ACL; + String OP_REMOVE_ACL_ENTRIES = StoreStatisticNames.OP_REMOVE_ACL_ENTRIES; + String OP_REMOVE_DEFAULT_ACL = StoreStatisticNames.OP_REMOVE_DEFAULT_ACL; + String OP_RENAME = StoreStatisticNames.OP_RENAME; + String OP_SET_ACL = StoreStatisticNames.OP_SET_ACL; + String OP_SET_OWNER = StoreStatisticNames.OP_SET_OWNER; + String OP_SET_PERMISSION = StoreStatisticNames.OP_SET_PERMISSION; + String OP_SET_TIMES = StoreStatisticNames.OP_SET_TIMES; + String OP_TRUNCATE = StoreStatisticNames.OP_TRUNCATE; } /** diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StreamCapabilities.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StreamCapabilities.java index e68e7b351ed78..15ea2ab325c33 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StreamCapabilities.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StreamCapabilities.java @@ -71,6 +71,11 @@ public interface StreamCapabilities { */ String PREADBYTEBUFFER = "in:preadbytebuffer"; + /** + * IOStatisticsSource API. + */ + String IOSTATISTICS = "iostatistics"; + /** * Capabilities that a stream can support and be queried for. */ diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FunctionsRaisingIOE.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FunctionsRaisingIOE.java index 7bbb34622647d..551cf9cff3d6f 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FunctionsRaisingIOE.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FunctionsRaisingIOE.java @@ -24,7 +24,8 @@ import org.apache.hadoop.classification.InterfaceStability; /** - * Evolving support for functional programming/lambda-expressions. + * Support for functional programming/lambda-expressions. + * @deprecated use {@code org.apache.hadoop.util.functional} */ @InterfaceAudience.Private @InterfaceStability.Unstable @@ -37,6 +38,7 @@ private FunctionsRaisingIOE() { * Function of arity 1 which may raise an IOException. * @param type of arg1 * @param type of return value. + * @deprecated use {@link org.apache.hadoop.util.functional.FunctionRaisingIOE} */ @FunctionalInterface public interface FunctionRaisingIOE { @@ -49,6 +51,7 @@ public interface FunctionRaisingIOE { * @param type of arg1 * @param type of arg2 * @param type of return value. + * @deprecated use {@link org.apache.hadoop.util.functional.BiFunctionRaisingIOE} */ @FunctionalInterface public interface BiFunctionRaisingIOE { @@ -59,6 +62,7 @@ public interface BiFunctionRaisingIOE { /** * This is a callable which only raises an IOException. * @param return type + * @deprecated use {@link org.apache.hadoop.util.functional.CallableRaisingIOE} */ @FunctionalInterface public interface CallableRaisingIOE { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureIOSupport.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureIOSupport.java index f13d701803d7e..fe112d59352f5 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureIOSupport.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureIOSupport.java @@ -32,9 +32,16 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSBuilder; +import org.apache.hadoop.util.functional.CallableRaisingIOE; +import org.apache.hadoop.util.functional.FutureIO; /** * Support for future IO and the FS Builder subclasses. + * If methods in here are needed for applications, promote + * to {@link FutureIO} for public use -with the original + * method relaying to it. This is to ensure that external + * filesystem implementations can safely use these methods + * without linkage problems surfacing. */ @InterfaceAudience.Private @InterfaceStability.Unstable @@ -55,14 +62,7 @@ private FutureIOSupport() { */ public static T awaitFuture(final Future future) throws InterruptedIOException, IOException, RuntimeException { - try { - return future.get(); - } catch (InterruptedException e) { - throw (InterruptedIOException)new InterruptedIOException(e.toString()) - .initCause(e); - } catch (ExecutionException e) { - return raiseInnerCause(e); - } + return FutureIO.awaitFuture(future); } @@ -82,18 +82,9 @@ public static T awaitFuture(final Future future, final TimeUnit unit) throws InterruptedIOException, IOException, RuntimeException, TimeoutException { - - try { - return future.get(timeout, unit); - } catch (InterruptedException e) { - throw (InterruptedIOException)new InterruptedIOException(e.toString()) - .initCause(e); - } catch (ExecutionException e) { - return raiseInnerCause(e); - } + return FutureIO.awaitFuture(future, timeout, unit); } - /** * From the inner cause of an execution exception, extract the inner cause * if it is an IOE or RTE. @@ -110,7 +101,7 @@ public static T awaitFuture(final Future future, */ public static T raiseInnerCause(final ExecutionException e) throws IOException { - throw unwrapInnerException(e); + return FutureIO.raiseInnerCause(e); } /** @@ -125,41 +116,7 @@ public static T raiseInnerCause(final ExecutionException e) */ public static T raiseInnerCause(final CompletionException e) throws IOException { - throw unwrapInnerException(e); - } - - /** - * From the inner cause of an execution exception, extract the inner cause. - * If it is an RTE: throw immediately. - * If it is an IOE: Return. - * If it is a WrappedIOException: Unwrap and return - * Else: create a new IOException. - * - * Recursively handles wrapped Execution and Completion Exceptions in - * case something very complicated has happened. - * @param e exception. - * @return an IOException extracted or built from the cause. - * @throws RuntimeException if that is the inner cause. - */ - private static IOException unwrapInnerException(final Throwable e) { - Throwable cause = e.getCause(); - if (cause instanceof IOException) { - return (IOException) cause; - } else if (cause instanceof WrappedIOException) { - return ((WrappedIOException) cause).getCause(); - } else if (cause instanceof CompletionException) { - return unwrapInnerException(cause); - } else if (cause instanceof ExecutionException) { - return unwrapInnerException(cause); - } else if (cause instanceof RuntimeException) { - throw (RuntimeException) cause; - } else if (cause != null) { - // other type: wrap with a new IOE - return new IOException(cause); - } else { - // this only happens if there was no cause. - return new IOException(e); - } + return FutureIO.raiseInnerCause(e); } /** @@ -236,7 +193,7 @@ public static void propagateOptions( * @throws IllegalArgumentException invalid argument */ public static CompletableFuture eval( - FunctionsRaisingIOE.CallableRaisingIOE callable) { + CallableRaisingIOE callable) { CompletableFuture result = new CompletableFuture<>(); try { result.complete(callable.apply()); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/WrappedIOException.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/WrappedIOException.java index ae1d548d34dfc..2fcdee915ede9 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/WrappedIOException.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/WrappedIOException.java @@ -19,6 +19,7 @@ package org.apache.hadoop.fs.impl; import java.io.IOException; +import java.io.UncheckedIOException; import java.util.concurrent.ExecutionException; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; @@ -33,10 +34,12 @@ * * The constructor signature guarantees the cause will be an IOException, * and as it checks for a null-argument, non-null. + * @deprecated use the {@code UncheckedIOException}. */ +@Deprecated @InterfaceAudience.Private @InterfaceStability.Unstable -public class WrappedIOException extends RuntimeException { +public class WrappedIOException extends UncheckedIOException { private static final long serialVersionUID = 2510210974235779294L; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/BufferedIOStatisticsInputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/BufferedIOStatisticsInputStream.java new file mode 100644 index 0000000000000..bdc432570542b --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/BufferedIOStatisticsInputStream.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics; + +import java.io.BufferedInputStream; +import java.io.InputStream; + +import org.apache.hadoop.fs.StreamCapabilities; + +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics; + +/** + * An extension of {@code BufferedInputStream} which implements + * {@link IOStatisticsSource} and forwards requests for the + * {@link IOStatistics} to the wrapped stream. + * + * This should be used when any input stream needs buffering while + * allowing the inner stream to be a source of statistics. + * + * It also implements {@link StreamCapabilities} and forwards the probe + * to the inner stream, if possible. + */ +public class BufferedIOStatisticsInputStream + extends BufferedInputStream + implements IOStatisticsSource, StreamCapabilities { + + /** + * Buffer an input stream with the default buffer size of 8k. + * @param in input stream + */ + public BufferedIOStatisticsInputStream(final InputStream in) { + super(in); + } + + /** + * Buffer an input stream with the chosen buffer size. + * @param in input stream + * @param size buffer size + */ + public BufferedIOStatisticsInputStream(final InputStream in, final int size) { + super(in, size); + } + + /** + * Return any IOStatistics offered by the inner stream. + * @return inner IOStatistics or null + */ + @Override + public IOStatistics getIOStatistics() { + return retrieveIOStatistics(in); + } + + /** + * If the inner stream supports {@link StreamCapabilities}, + * forward the probe to it. + * Otherwise: return false. + * + * @param capability string to query the stream support for. + * @return true if a capability is known to be supported. + */ + @Override + public boolean hasCapability(final String capability) { + if (in instanceof StreamCapabilities) { + return ((StreamCapabilities) in).hasCapability(capability); + } else { + return false; + } + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/BufferedIOStatisticsOutputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/BufferedIOStatisticsOutputStream.java new file mode 100644 index 0000000000000..88e73a0629b1d --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/BufferedIOStatisticsOutputStream.java @@ -0,0 +1,157 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics; + +import java.io.BufferedOutputStream; +import java.io.IOException; +import java.io.OutputStream; + +import org.apache.hadoop.fs.StreamCapabilities; +import org.apache.hadoop.fs.Syncable; + +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics; + +/** + * An extension of {@code BufferedOutputStream} which implements + * {@link IOStatisticsSource} and forwards requests for the + * {@link IOStatistics} to the wrapped stream. + * + * This should be used when any output stream needs buffering while + * allowing the inner stream to be a source of statistics. + * + * It also implements {@link StreamCapabilities} + * and {@link Syncable} and forwards to to the inner stream, + * if possible. + */ +public class BufferedIOStatisticsOutputStream + extends BufferedOutputStream + implements IOStatisticsSource, Syncable, StreamCapabilities { + + /** + * Should calls to Syncable downgrade to flush if the underlying + * stream does not support it? + * While that breaks a core contract requirement of Syncable: + * "Sync.sync() guarantees durability", downgrading is + * the default behavior of FsDataOutputStream. + */ + private final boolean downgradeSyncable; + + /** + * Construct with default buffer size. + * @param out output stream to buffer + * @param downgradeSyncable should Syncable calls downgrade? + */ + public BufferedIOStatisticsOutputStream( + final OutputStream out, + final boolean downgradeSyncable) { + super(out); + this.downgradeSyncable = downgradeSyncable; + } + + /** + * Construct with custom buffer size. + * + * @param out output stream to buffer + * @param size buffer. + * @param downgradeSyncable should Syncable calls downgrade? + */ + public BufferedIOStatisticsOutputStream( + final OutputStream out, + final int size, + final boolean downgradeSyncable) { + super(out, size); + this.downgradeSyncable = downgradeSyncable; + } + + /** + * Ask the inner stream for their IOStatistics. + * @return any IOStatistics offered by the inner stream. + */ + @Override + public IOStatistics getIOStatistics() { + return retrieveIOStatistics(out); + } + + /** + * If the inner stream supports {@link StreamCapabilities}, + * forward the probe to it. + * Otherwise: return false. + * + * @param capability string to query the stream support for. + * @return true if a capability is known to be supported. + */ + @Override + public boolean hasCapability(final String capability) { + if (out instanceof StreamCapabilities) { + return ((StreamCapabilities) out).hasCapability(capability); + } else { + return false; + } + } + + /** + * If the inner stream is Syncable, flush the buffer and then + * invoke the inner stream's hflush() operation. + * + * Otherwise: throw an exception, unless the stream was constructed with + * {@link #downgradeSyncable} set to true, in which case the stream + * is just flushed. + * @throws IOException IO Problem + * @throws UnsupportedOperationException if the inner class is not syncable + */ + @Override + public void hflush() throws IOException { + if (out instanceof Syncable) { + flush(); + ((Syncable) out).hflush(); + } else { + if (!downgradeSyncable) { + throw new UnsupportedOperationException("hflush not supported by " + + out); + } else { + flush(); + } + } + } + + /** + * If the inner stream is Syncable, flush the buffer and then + * invoke the inner stream's hsync() operation. + * + * Otherwise: throw an exception, unless the stream was constructed with + * {@link #downgradeSyncable} set to true, in which case the stream + * is just flushed. + * @throws IOException IO Problem + * @throws UnsupportedOperationException if the inner class is not syncable + */ + @Override + public void hsync() throws IOException { + if (out instanceof Syncable) { + flush(); + ((Syncable) out).hsync(); + } else { + if (!downgradeSyncable) { + throw new UnsupportedOperationException("hsync not supported by " + + out); + } else { + flush(); + } + } + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/DurationStatisticSummary.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/DurationStatisticSummary.java new file mode 100644 index 0000000000000..e1335d77d792a --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/DurationStatisticSummary.java @@ -0,0 +1,154 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics; + +import javax.annotation.Nullable; +import java.io.Serializable; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_FAILURES; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MAX; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MEAN; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MIN; + +/** + * Summary of duration tracking statistics + * as extracted from an IOStatistics instance. + *

+ * This is for reporting and testing. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public final class DurationStatisticSummary implements Serializable { + + private static final long serialVersionUID = 6776381340896518486L; + + /** Statistic key. */ + private final String key; + + /** Are these success or failure statistics. */ + private final boolean success; + + /** Count of operation invocations. */ + private final long count; + + /** Max duration; -1 if unknown. */ + private final long max; + + /** Min duration; -1 if unknown. */ + private final long min; + + /** Mean duration -may be null. */ + private final MeanStatistic mean; + + /** + * Constructor. + * @param key Statistic key. + * @param success Are these success or failure statistics. + * @param count Count of operation invocations. + * @param max Max duration; -1 if unknown. + * @param min Min duration; -1 if unknown. + * @param mean Mean duration -may be null. (will be cloned) + */ + public DurationStatisticSummary(final String key, + final boolean success, + final long count, + final long max, + final long min, + @Nullable final MeanStatistic mean) { + this.key = key; + this.success = success; + this.count = count; + this.max = max; + this.min = min; + this.mean = mean == null ? null : mean.clone(); + } + + public String getKey() { + return key; + } + + public boolean isSuccess() { + return success; + } + + public long getCount() { + return count; + } + + public long getMax() { + return max; + } + + public long getMin() { + return min; + } + + public MeanStatistic getMean() { + return mean; + } + + @Override + public String toString() { + return "DurationStatisticSummary{" + + "key='" + key + '\'' + + ", success=" + success + + ", counter=" + count + + ", max=" + max + + ", mean=" + mean + + '}'; + } + + /** + * Fetch the duration timing summary of success or failure operations + * from an IO Statistics source. + * If the duration key is unknown, the summary will be incomplete. + * @param source source of data + * @param key duration statistic key + * @param success fetch success statistics, or if false, failure stats. + * @return a summary of the statistics. + */ + public static DurationStatisticSummary fetchDurationSummary( + IOStatistics source, + String key, + boolean success) { + String fullkey = success ? key : key + SUFFIX_FAILURES; + return new DurationStatisticSummary(key, success, + source.counters().getOrDefault(fullkey, 0L), + source.maximums().getOrDefault(fullkey + SUFFIX_MAX, -1L), + source.minimums().getOrDefault(fullkey + SUFFIX_MIN, -1L), + source.meanStatistics() + .get(fullkey + SUFFIX_MEAN)); + } + + /** + * Fetch the duration timing summary from an IOStatistics source. + * If the duration key is unknown, the summary will be incomplete. + * @param source source of data + * @param key duration statistic key + * @return a summary of the statistics. + */ + public static DurationStatisticSummary fetchSuccessSummary( + IOStatistics source, + String key) { + return fetchDurationSummary(source, key, true); + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/DurationTracker.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/DurationTracker.java new file mode 100644 index 0000000000000..5a15c7ad66c4f --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/DurationTracker.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics; + +import java.time.Duration; + +/** + * Interface to be implemented by objects which can track duration. + * It extends AutoCloseable to fit into a try-with-resources statement, + * but then strips out the {@code throws Exception} aspect of the signature + * so it doesn't force code to add extra handling for any failures. + * + * If a duration is declared as "failed()" then the failure counters + * will be updated. + */ +public interface DurationTracker extends AutoCloseable { + + /** + * The operation failed. Failure statistics will be updated. + */ + void failed(); + + /** + * Finish tracking: update the statistics with the timings. + */ + void close(); + + /** + * Get the duration of an operation as a java Duration + * instance. If the duration tracker hasn't completed, + * or its duration tracking doesn't actually measure duration, + * returns Duration.ZERO. + * @return a duration, value of ZERO until close(). + */ + default Duration asDuration() { + return Duration.ZERO; + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/DurationTrackerFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/DurationTrackerFactory.java new file mode 100644 index 0000000000000..b1d87c9100f95 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/DurationTrackerFactory.java @@ -0,0 +1,57 @@ +/* + * 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; + +/** + * Interface for a source of duration tracking. + * + * This is intended for uses where it can be passed into classes + * which update operation durations, without tying those + * classes to internal implementation details. + */ +public interface DurationTrackerFactory { + + /** + * Initiate a duration tracking operation by creating/returning + * an object whose {@code close()} call will + * update the statistics. + * + * The statistics counter with the key name will be incremented + * by the given count. + * + * The expected use is within a try-with-resources clause. + * @param key statistic key prefix + * @param count #of times to increment the matching counter in this + * operation. + * @return an object to close after an operation completes. + */ + DurationTracker trackDuration(String key, long count); + + /** + * Initiate a duration tracking operation by creating/returning + * an object whose {@code close()} call will + * update the statistics. + * The expected use is within a try-with-resources clause. + * @param key statistic key + * @return an object to close after an operation completes. + */ + default DurationTracker trackDuration(String key) { + return trackDuration(key, 1); + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatistics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatistics.java new file mode 100644 index 0000000000000..75d9965128101 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatistics.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics; + +import java.util.Map; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * IO Statistics. + *

+ * These are low-cost per-instance statistics provided by any Hadoop + * I/O class instance. + *

+ * Consult the filesystem specification document for the requirements + * of an implementation of this interface. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public interface IOStatistics { + + /** + * Map of counters. + * @return the current map of counters. + */ + Map counters(); + + /** + * Map of gauges. + * @return the current map of gauges. + */ + Map gauges(); + + /** + * Map of minimums. + * @return the current map of minimums. + */ + Map minimums(); + + /** + * Map of maximums. + * @return the current map of maximums. + */ + Map maximums(); + + /** + * Map of meanStatistics. + * @return the current map of MeanStatistic statistics. + */ + Map meanStatistics(); + + /** + * Value when a minimum value has never been set. + */ + long MIN_UNSET_VALUE = -1; + + /** + * Value when a max value has never been set. + */ + long MAX_UNSET_VALUE = -1; +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsAggregator.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsAggregator.java new file mode 100644 index 0000000000000..1c5451c6f0e83 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsAggregator.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics; + +import javax.annotation.Nullable; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Interface exported by classes which support + * aggregation of {@link IOStatistics}. + * Implementations MAY aggregate all statistics + * exported by the IOStatistics reference passed in to + * {@link #aggregate(IOStatistics)}, or they + * may selectively aggregate specific values/classes + * of statistics. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public interface IOStatisticsAggregator { + + /** + * Aggregate the supplied statistics into the current + * set. + * + * @param statistics statistics; may be null + * @return true if the statistics reference was not null and + * so aggregated. + */ + boolean aggregate(@Nullable IOStatistics statistics); +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsLogging.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsLogging.java new file mode 100644 index 0000000000000..c7230e25c3434 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsLogging.java @@ -0,0 +1,301 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics; + +import javax.annotation.Nullable; +import java.util.Map; +import java.util.TreeMap; +import java.util.function.Predicate; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding; + +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics; + +/** + * Utility operations convert IO Statistics sources/instances + * to strings, especially for robustly logging. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public final class IOStatisticsLogging { + + private static final Logger LOG = + LoggerFactory.getLogger(IOStatisticsLogging.class); + + private IOStatisticsLogging() { + } + + /** + * Extract the statistics from a source object -or "" + * if it is not an instance of {@link IOStatistics}, + * {@link IOStatisticsSource} or the retrieved + * statistics are null. + *

+ * Exceptions are caught and downgraded to debug logging. + * @param source source of statistics. + * @return a string for logging. + */ + public static String ioStatisticsSourceToString(@Nullable Object source) { + try { + return ioStatisticsToString(retrieveIOStatistics(source)); + } catch (RuntimeException e) { + LOG.debug("Ignoring", e); + return ""; + } + } + + /** + * Convert IOStatistics to a string form. + * @param statistics A statistics instance. + * @return string value or the empty string if null + */ + public static String ioStatisticsToString( + @Nullable final IOStatistics statistics) { + if (statistics != null) { + StringBuilder sb = new StringBuilder(); + mapToString(sb, "counters", statistics.counters(), " "); + mapToString(sb, "gauges", statistics.gauges(), " "); + mapToString(sb, "minimums", statistics.minimums(), " "); + mapToString(sb, "maximums", statistics.maximums(), " "); + mapToString(sb, "means", statistics.meanStatistics(), " "); + + return sb.toString(); + } else { + return ""; + } + } + + /** + * Convert IOStatistics to a string form, with all the metrics sorted + * and empty value stripped. + * This is more expensive than the simple conversion, so should only + * be used for logging/output where it's known/highly likely that the + * caller wants to see the values. Not for debug logging. + * @param statistics A statistics instance. + * @return string value or the empty string if null + */ + public static String ioStatisticsToPrettyString( + @Nullable final IOStatistics statistics) { + if (statistics != null) { + StringBuilder sb = new StringBuilder(); + mapToSortedString(sb, "counters", statistics.counters(), + p -> p == 0); + mapToSortedString(sb, "\ngauges", statistics.gauges(), + p -> p == 0); + mapToSortedString(sb, "\nminimums", statistics.minimums(), + p -> p < 0); + mapToSortedString(sb, "\nmaximums", statistics.maximums(), + p -> p < 0); + mapToSortedString(sb, "\nmeans", statistics.meanStatistics(), + MeanStatistic::isEmpty); + + return sb.toString(); + } else { + return ""; + } + } + + /** + * Given a map, add its entryset to the string. + * The entries are only sorted if the source entryset + * iterator is sorted, such as from a TreeMap. + * @param sb string buffer to append to + * @param type type (for output) + * @param map map to evaluate + * @param separator separator + * @param type of values of the map + */ + private static void mapToString(StringBuilder sb, + final String type, + final Map map, + final String separator) { + int count = 0; + sb.append(type); + sb.append("=("); + for (Map.Entry entry : map.entrySet()) { + if (count > 0) { + sb.append(separator); + } + count++; + sb.append(IOStatisticsBinding.entryToString( + entry.getKey(), entry.getValue())); + } + sb.append(");\n"); + } + + /** + * Given a map, produce a string with all the values, sorted. + * Needs to create a treemap and insert all the entries. + * @param sb string buffer to append to + * @param type type (for output) + * @param map map to evaluate + * @param type of values of the map + */ + private static void mapToSortedString(StringBuilder sb, + final String type, + final Map map, + final Predicate isEmpty) { + mapToString(sb, type, sortedMap(map, isEmpty), "\n"); + } + + /** + * Create a sorted (tree) map from an unsorted map. + * This incurs the cost of creating a map and that + * of inserting every object into the tree. + * @param source source map + * @param value type + * @return a treemap with all the entries. + */ + private static Map sortedMap( + final Map source, + final Predicate isEmpty) { + Map tm = new TreeMap<>(); + for (Map.Entry entry : source.entrySet()) { + if (!isEmpty.test(entry.getValue())) { + tm.put(entry.getKey(), entry.getValue()); + } + } + return tm; + } + + /** + * On demand stringifier of an IOStatisticsSource instance. + *

+ * Whenever this object's toString() method is called, it evaluates the + * statistics. + *

+ * This is designed to affordable to use in log statements. + * @param source source of statistics -may be null. + * @return an object whose toString() operation returns the current values. + */ + public static Object demandStringifyIOStatisticsSource( + @Nullable IOStatisticsSource source) { + return new SourceToString(source); + } + + /** + * On demand stringifier of an IOStatistics instance. + *

+ * Whenever this object's toString() method is called, it evaluates the + * statistics. + *

+ * This is for use in log statements where for the cost of creation + * of this entry is low; it is affordable to use in log statements. + * @param statistics statistics to stringify -may be null. + * @return an object whose toString() operation returns the current values. + */ + public static Object demandStringifyIOStatistics( + @Nullable IOStatistics statistics) { + return new StatisticsToString(statistics); + } + + /** + * Extract any statistics from the source and log at debug, if + * the log is set to log at debug. + * No-op if logging is not at debug or the source is null/of + * the wrong type/doesn't provide statistics. + * @param log log to log to + * @param message message for log -this must contain "{}" for the + * statistics report to actually get logged. + * @param source source object + */ + public static void logIOStatisticsAtDebug( + Logger log, + String message, + Object source) { + if (log.isDebugEnabled()) { + // robust extract and convert to string + String stats = ioStatisticsSourceToString(source); + if (!stats.isEmpty()) { + log.debug(message, stats); + } + } + } + + /** + * Extract any statistics from the source and log to + * this class's log at debug, if + * the log is set to log at debug. + * No-op if logging is not at debug or the source is null/of + * the wrong type/doesn't provide statistics. + * @param message message for log -this must contain "{}" for the + * statistics report to actually get logged. + * @param source source object + */ + public static void logIOStatisticsAtDebug( + String message, + Object source) { + logIOStatisticsAtDebug(LOG, message, source); + } + + /** + * On demand stringifier. + *

+ * Whenever this object's toString() method is called, it + * retrieves the latest statistics instance and re-evaluates it. + */ + private static final class SourceToString { + + private final IOStatisticsSource source; + + private SourceToString(@Nullable IOStatisticsSource source) { + this.source = source; + } + + @Override + public String toString() { + return source != null + ? ioStatisticsSourceToString(source) + : IOStatisticsBinding.NULL_SOURCE; + } + } + + /** + * Stringifier of statistics: low cost to instantiate and every + * toString/logging will re-evaluate the statistics. + */ + private static final class StatisticsToString { + + private final IOStatistics statistics; + + /** + * Constructor. + * @param statistics statistics + */ + private StatisticsToString(@Nullable IOStatistics statistics) { + this.statistics = statistics; + } + + /** + * Evaluate and stringify the statistics. + * @return a string value. + */ + @Override + public String toString() { + return statistics != null + ? ioStatisticsToString(statistics) + : IOStatisticsBinding.NULL_SOURCE; + } + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSnapshot.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSnapshot.java new file mode 100644 index 0000000000000..5b8b2e284cc11 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSnapshot.java @@ -0,0 +1,285 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding; +import org.apache.hadoop.util.JsonSerialization; + +import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToString; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.aggregateMaps; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.snapshotMap; + +/** + * Snapshot of statistics from a different source. + *

+ * It is serializable so that frameworks which can use java serialization + * to propagate data (Spark, Flink...) can send the statistics + * back. For this reason, TreeMaps are explicitly used as field types, + * even though IDEs can recommend use of Map instead. + * For security reasons, untrusted java object streams should never be + * deserialized. If for some reason this is required, use + * {@link #requiredSerializationClasses()} to get the list of classes + * used when deserializing instances of this object. + *

+ *

+ * It is annotated for correct serializations with jackson2. + *

+ */ +@SuppressWarnings("CollectionDeclaredAsConcreteClass") +@InterfaceAudience.Public +@InterfaceStability.Evolving +public final class IOStatisticsSnapshot + implements IOStatistics, Serializable, IOStatisticsAggregator { + + private static final long serialVersionUID = -1762522703841538084L; + + /** + * List of chasses needed to deserialize. + */ + private static final Class[] DESERIALIZATION_CLASSES = { + IOStatisticsSnapshot.class, + TreeMap.class, + Long.class, + MeanStatistic.class, + }; + + /** + * Counters. + */ + @JsonProperty + private transient Map counters; + + /** + * Gauges. + */ + @JsonProperty + private transient Map gauges; + + /** + * Minimum values. + */ + @JsonProperty + private transient Map minimums; + + /** + * Maximum values. + */ + @JsonProperty + private transient Map maximums; + + /** + * mean statistics. The JSON key is all lower case.. + */ + @JsonProperty("meanstatistics") + private transient Map meanStatistics; + + /** + * Construct. + */ + public IOStatisticsSnapshot() { + createMaps(); + } + + /** + * Construct, taking a snapshot of the source statistics data + * if the source is non-null. + * If the source is null, the empty maps are created + * @param source statistics source. Nullable. + */ + public IOStatisticsSnapshot(IOStatistics source) { + if (source != null) { + snapshot(source); + } else { + createMaps(); + } + } + + /** + * Create the maps. + */ + private synchronized void createMaps() { + counters = new ConcurrentHashMap<>(); + gauges = new ConcurrentHashMap<>(); + minimums = new ConcurrentHashMap<>(); + maximums = new ConcurrentHashMap<>(); + meanStatistics = new ConcurrentHashMap<>(); + } + + /** + * Clear all the maps. + */ + public synchronized void clear() { + counters.clear(); + gauges.clear(); + minimums.clear(); + maximums.clear(); + meanStatistics.clear(); + } + + /** + * Take a snapshot. + * + * This completely overwrites the map data with the statistics + * from the source. + * @param source statistics source. + */ + public synchronized void snapshot(IOStatistics source) { + checkNotNull(source); + counters = snapshotMap(source.counters()); + gauges = snapshotMap(source.gauges()); + minimums = snapshotMap(source.minimums()); + maximums = snapshotMap(source.maximums()); + meanStatistics = snapshotMap(source.meanStatistics(), + MeanStatistic::copy); + } + + /** + * Aggregate the current statistics with the + * source reference passed in. + * + * The operation is synchronized. + * @param source source; may be null + * @return true if a merge took place. + */ + @Override + public synchronized boolean aggregate( + @Nullable IOStatistics source) { + if (source == null) { + return false; + } + aggregateMaps(counters, source.counters(), + IOStatisticsBinding::aggregateCounters, + IOStatisticsBinding::passthroughFn); + aggregateMaps(gauges, source.gauges(), + IOStatisticsBinding::aggregateGauges, + IOStatisticsBinding::passthroughFn); + aggregateMaps(minimums, source.minimums(), + IOStatisticsBinding::aggregateMinimums, + IOStatisticsBinding::passthroughFn); + aggregateMaps(maximums, source.maximums(), + IOStatisticsBinding::aggregateMaximums, + IOStatisticsBinding::passthroughFn); + aggregateMaps(meanStatistics, source.meanStatistics(), + IOStatisticsBinding::aggregateMeanStatistics, MeanStatistic::copy); + return true; + } + + @Override + public synchronized Map counters() { + return counters; + } + + @Override + public synchronized Map gauges() { + return gauges; + } + + @Override + public synchronized Map minimums() { + return minimums; + } + + @Override + public synchronized Map maximums() { + return maximums; + } + + @Override + public synchronized Map meanStatistics() { + return meanStatistics; + } + + @Override + public String toString() { + return ioStatisticsToString(this); + } + + /** + * Get a JSON serializer for this class. + * @return a serializer. + */ + public static JsonSerialization serializer() { + return new JsonSerialization<>(IOStatisticsSnapshot.class, false, true); + } + + /** + * Serialize by converting each map to a TreeMap, and saving that + * to the stream. + */ + private synchronized void writeObject(ObjectOutputStream s) + throws IOException { + // Write out the core + s.defaultWriteObject(); + s.writeObject(new TreeMap(counters)); + s.writeObject(new TreeMap(gauges)); + s.writeObject(new TreeMap(minimums)); + s.writeObject(new TreeMap(maximums)); + s.writeObject(new TreeMap(meanStatistics)); + } + + /** + * Deserialize by loading each TreeMap, and building concurrent + * hash maps from them. + */ + private void readObject(final ObjectInputStream s) + throws IOException, ClassNotFoundException { + // read in core + s.defaultReadObject(); + // and rebuild a concurrent hashmap from every serialized tree map + // read back from the stream. + counters = new ConcurrentHashMap<>( + (TreeMap) s.readObject()); + gauges = new ConcurrentHashMap<>( + (TreeMap) s.readObject()); + minimums = new ConcurrentHashMap<>( + (TreeMap) s.readObject()); + maximums = new ConcurrentHashMap<>( + (TreeMap) s.readObject()); + meanStatistics = new ConcurrentHashMap<>( + (TreeMap) s.readObject()); + } + + /** + * What classes are needed to deserialize this class? + * Needed to securely unmarshall this from untrusted sources. + * @return a list of required classes to deserialize the data. + */ + public static List requiredSerializationClasses() { + return Arrays.stream(DESERIALIZATION_CLASSES) + .collect(Collectors.toList()); + } + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSource.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSource.java new file mode 100644 index 0000000000000..67bf51fc0c3ae --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSource.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics; + +import org.apache.hadoop.classification.InterfaceStability; + +/** + * A source of IO statistics. + *

+ * These statistics MUST be instance specific, not thread local. + *

+ */ + +@InterfaceStability.Unstable +public interface IOStatisticsSource { + + /** + * Return a statistics instance. + *

+ * It is not a requirement that the same instance is returned every time. + * {@link IOStatisticsSource}. + *

+ * If the object implementing this is Closeable, this method + * may return null if invoked on a closed object, even if + * it returns a valid instance when called earlier. + * @return an IOStatistics instance or null + */ + default IOStatistics getIOStatistics() { + return null; + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSupport.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSupport.java new file mode 100644 index 0000000000000..75977047c0f2a --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSupport.java @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.statistics.impl.StubDurationTracker; +import org.apache.hadoop.fs.statistics.impl.StubDurationTrackerFactory; + +/** + * Support for working with IOStatistics. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public final class IOStatisticsSupport { + + private IOStatisticsSupport() { + } + + /** + * Take a snapshot of the current statistics state. + *

+ * This is not an atomic option. + *

+ * The instance can be serialized, and its + * {@code toString()} method lists all the values. + * @param statistics statistics + * @return a snapshot of the current values. + */ + public static IOStatisticsSnapshot + snapshotIOStatistics(IOStatistics statistics) { + + return new IOStatisticsSnapshot(statistics); + } + + /** + * Create a snapshot statistics instance ready to aggregate data. + * + * The instance can be serialized, and its + * {@code toString()} method lists all the values. + * @return an empty snapshot + */ + public static IOStatisticsSnapshot + snapshotIOStatistics() { + + return new IOStatisticsSnapshot(); + } + + /** + * Get the IOStatistics of the source, casting it + * if it is of the relevant type, otherwise, + * if it implements {@link IOStatisticsSource} + * extracting the value. + * + * Returns null if the source isn't of the write type + * or the return value of + * {@link IOStatisticsSource#getIOStatistics()} was null. + * @return an IOStatistics instance or null + */ + + public static IOStatistics retrieveIOStatistics( + final Object source) { + if (source instanceof IOStatistics) { + return (IOStatistics) source; + } else if (source instanceof IOStatisticsSource) { + return ((IOStatisticsSource) source).getIOStatistics(); + } else { + // null source or interface not implemented + return null; + } + } + + /** + * Return a stub duration tracker factory whose returned trackers + * are always no-ops. + * + * As singletons are returned, this is very low-cost to use. + * @return a duration tracker factory. + */ + public static DurationTrackerFactory stubDurationTrackerFactory() { + return StubDurationTrackerFactory.STUB_DURATION_TRACKER_FACTORY; + } + + /** + * Get a stub duration tracker. + * @return a stub tracker. + */ + public static DurationTracker stubDurationTracker() { + return StubDurationTracker.STUB_DURATION_TRACKER; + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/MeanStatistic.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/MeanStatistic.java new file mode 100644 index 0000000000000..d9ff0c25c6a21 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/MeanStatistic.java @@ -0,0 +1,290 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics; + +import java.io.Serializable; +import java.util.Objects; + +import com.fasterxml.jackson.annotation.JsonIgnore; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * A mean statistic represented as the sum and the sample count; + * the mean is calculated on demand. + *

+ * It can be used to accrue values so as to dynamically update + * the mean. If so, know that there is no synchronization + * on the methods. + *

+ *

+ * If a statistic has 0 samples then it is considered to be empty. + *

+ *

+ * All 'empty' statistics are equivalent, independent of the sum value. + *

+ *

+ * For non-empty statistics, sum and sample values must match + * for equality. + *

+ *

+ * It is serializable and annotated for correct serializations with jackson2. + *

+ *

+ * Thread safety. The operations to add/copy sample data, are thread safe. + *

+ *
    + *
  1. {@link #add(MeanStatistic)}
  2. + *
  3. {@link #addSample(long)}
  4. + *
  5. {@link #clear()}
  6. + *
  7. {@link #setSamplesAndSum(long, long)}
  8. + *
  9. {@link #set(MeanStatistic)}
  10. + *
  11. {@link #setSamples(long)} and {@link #setSum(long)}
  12. + *
+ *

+ * So is the {@link #mean()} method. This ensures that when + * used to aggregated statistics, the aggregate value and sample + * count are set and evaluated consistently. + *

+ *

+ * Other methods marked as synchronized because Findbugs overreacts + * to the idea that some operations to update sum and sample count + * are synchronized, but that things like equals are not. + *

+ */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public final class MeanStatistic implements Serializable, Cloneable { + + private static final long serialVersionUID = 567888327998615425L; + + /** + * Number of samples used to calculate + * the mean. + */ + private long samples; + + /** + * sum of the values. + */ + private long sum; + + /** + * Constructor, with some resilience against invalid sample counts. + * If the sample count is 0 or less, the sum is set to 0 and + * the sample count to 0. + * @param samples sample count. + * @param sum sum value + */ + public MeanStatistic(final long samples, final long sum) { + if (samples > 0) { + this.sum = sum; + this.samples = samples; + } + } + + /** + * Create from another statistic. + * @param that source + */ + public MeanStatistic(MeanStatistic that) { + synchronized (that) { + set(that); + } + } + + /** + * Create an empty statistic. + */ + public MeanStatistic() { + } + + /** + * Get the sum of samples. + * @return the sum + */ + public synchronized long getSum() { + return sum; + } + + /** + * Get the sample count. + * @return the sample count; 0 means empty + */ + public synchronized long getSamples() { + return samples; + } + + /** + * Is a statistic empty? + * @return true if the sample count is 0 + */ + @JsonIgnore + public synchronized boolean isEmpty() { + return samples == 0; + } + + /** + * Set the values to 0. + */ + public void clear() { + setSamplesAndSum(0, 0); + } + + /** + * Set the sum and samples. + * Synchronized. + * @param sampleCount new sample count. + * @param newSum new sum + */ + public synchronized void setSamplesAndSum(long sampleCount, + long newSum) { + setSamples(sampleCount); + setSum(newSum); + } + + /** + * Set the statistic to the values of another. + * Synchronized. + * @param other the source. + */ + public void set(final MeanStatistic other) { + setSamplesAndSum(other.getSamples(), other.getSum()); + } + + /** + * Set the sum. + * @param sum new sum + */ + public synchronized void setSum(final long sum) { + this.sum = sum; + } + + /** + * Set the sample count. + * + * If this is less than zero, it is set to zero. + * This stops an ill-formed JSON entry from + * breaking deserialization, or get an invalid sample count + * into an entry. + * @param samples sample count. + */ + public synchronized void setSamples(final long samples) { + if (samples < 0) { + this.samples = 0; + } else { + this.samples = samples; + } + } + + /** + * Get the arithmetic mean value. + * @return the mean + */ + public synchronized double mean() { + return samples > 0 + ? ((double) sum) / samples + : 0.0d; + } + + /** + * Add another MeanStatistic. + * @param other other value + */ + public synchronized MeanStatistic add(final MeanStatistic other) { + if (other.isEmpty()) { + return this; + } + long otherSamples; + long otherSum; + synchronized (other) { + otherSamples = other.samples; + otherSum = other.sum; + } + if (isEmpty()) { + samples = otherSamples; + sum = otherSum; + return this; + } + samples += otherSamples; + sum += otherSum; + return this; + } + + /** + * Add a sample. + * Thread safe. + * @param value value to add to the sum + */ + public synchronized void addSample(long value) { + samples++; + sum += value; + } + + /** + * The hash code is derived from the mean + * and sample count: if either is changed + * the statistic cannot be used as a key + * for hash tables/maps. + * @return a hash value + */ + @Override + public synchronized int hashCode() { + return Objects.hash(sum, samples); + } + + @Override + public synchronized boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MeanStatistic that = (MeanStatistic) o; + if (isEmpty()) { + // if we are empty, then so must the other. + return that.isEmpty(); + } + return getSum() == that.getSum() && + getSamples() == that.getSamples(); + } + + @Override + public MeanStatistic clone() { + return copy(); + } + + /** + * Create a copy of this instance. + * @return copy. + * + */ + public MeanStatistic copy() { + return new MeanStatistic(this); + } + + @Override + public String toString() { + return String.format("(samples=%d, sum=%d, mean=%.4f)", + samples, sum, mean()); + } + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java new file mode 100644 index 0000000000000..4baf37d10fd77 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java @@ -0,0 +1,337 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Common statistic names for object store operations.. + *

+ * When adding new common statistic name constants, please make them unique. + * By convention: + *

+ *
    + *
  • the name of the constants are uppercase, words separated by + * underscores.
  • + *
  • the value of the constants are lowercase of the constant names.
  • + *
+ */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public final class StoreStatisticNames { + + /** {@value}. */ + public static final String OP_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"; + + /** + * A store's equivalent of a paged LIST request was initiated: {@value}. + */ + public static final String OBJECT_LIST_REQUEST + = "object_list_request"; + + /** + * Number of continued object listings made. + * Value :{@value}. + */ + public static final String OBJECT_CONTINUE_LIST_REQUEST = + "object_continue_list_request"; + + /** + * A bulk DELETE request was made: {@value}. + * A separate statistic from {@link #OBJECT_DELETE_REQUEST} + * so that metrics on duration of the operations can + * be distinguished. + */ + public static final String OBJECT_BULK_DELETE_REQUEST + = "object_bulk_delete_request"; + + /** + * A store's equivalent of a DELETE request was made: {@value}. + * This may be an HTTP DELETE verb, or it may be some custom + * operation which takes a list of objects to delete. + */ + public static final String OBJECT_DELETE_REQUEST + = "object_delete_request"; + + /** + * The count of objects deleted in delete requests. + */ + public static final String OBJECT_DELETE_OBJECTS + = "object_delete_objects"; + + /** + * Object multipart upload initiated. + * Value :{@value}. + */ + public static final String OBJECT_MULTIPART_UPLOAD_INITIATED = + "object_multipart_initiated"; + + /** + * Object multipart upload aborted. + * Value :{@value}. + */ + public static final String OBJECT_MULTIPART_UPLOAD_ABORTED = + "object_multipart_aborted"; + + /** + * Object put/multipart upload count. + * Value :{@value}. + */ + public static final String OBJECT_PUT_REQUEST = + "object_put_request"; + + /** + * Object put/multipart upload completed count. + * Value :{@value}. + */ + public static final String OBJECT_PUT_REQUEST_COMPLETED = + "object_put_request_completed"; + + /** + * Current number of active put requests. + * Value :{@value}. + */ + public static final String OBJECT_PUT_REQUEST_ACTIVE = + "object_put_request_active"; + + /** + * number of bytes uploaded. + * Value :{@value}. + */ + public static final String OBJECT_PUT_BYTES = + "object_put_bytes"; + + /** + * number of bytes queued for upload/being actively uploaded. + * Value :{@value}. + */ + public static final String OBJECT_PUT_BYTES_PENDING = + "object_put_bytes_pending"; + + /** + * Count of S3 Select (or similar) requests issued. + * Value :{@value}. + */ + public static final String OBJECT_SELECT_REQUESTS = + "object_select_requests"; + + /** + * Suffix to use for a minimum value when + * the same key is shared across min/mean/max + * statistics. + * + * Value {@value}. + */ + public static final String SUFFIX_MIN = ".min"; + + /** + * Suffix to use for a maximum value when + * the same key is shared across max/mean/max + * statistics. + * + * Value {@value}. + */ + public static final String SUFFIX_MAX = ".max"; + + /** + * Suffix to use for a mean statistic value when + * the same key is shared across mean/mean/max + * statistics. + * + * Value {@value}. + */ + public static final String SUFFIX_MEAN = ".mean"; + + /** + * String to add to counters and other stats to track failures. + * This comes before the .min/.mean//max suffixes. + * + * Value {@value}. + */ + public static final String SUFFIX_FAILURES = ".failures"; + + /** + * The name of the statistic collected for executor acquisition if + * a duration tracker factory is passed in to the constructor. + * {@value}. + */ + public static final String ACTION_EXECUTOR_ACQUIRED = + "action_executor_acquired"; + + /** + * An HTTP HEAD request was made: {@value}. + */ + public static final String ACTION_HTTP_HEAD_REQUEST + = "action_http_head_request"; + + /** + * An HTTP GET request was made: {@value}. + */ + public static final String ACTION_HTTP_GET_REQUEST + = "action_http_get_request"; + + /** + * An HTTP HEAD request was made: {@value}. + */ + public static final String OBJECT_METADATA_REQUESTS + = "object_metadata_request"; + + public static final String OBJECT_COPY_REQUESTS + = "object_copy_requests"; + + public static final String STORE_IO_THROTTLE_RATE + = "store_io_throttle_rate"; + + public static final String DELEGATION_TOKEN_ISSUED + = "delegation_token_issued"; + + public static final String MULTIPART_UPLOAD_INSTANTIATED + = "multipart_instantiated"; + + public static final String MULTIPART_UPLOAD_PART_PUT + = "multipart_upload_part_put"; + + public static final String MULTIPART_UPLOAD_PART_PUT_BYTES + = "multipart_upload_part_put_bytes"; + + public static final String MULTIPART_UPLOAD_ABORTED + = "multipart_upload_aborted"; + + public static final String MULTIPART_UPLOAD_ABORT_UNDER_PATH_INVOKED + = "multipart_upload_abort_under_path_invoked"; + + public static final String MULTIPART_UPLOAD_COMPLETED + = "multipart_upload_completed"; + + public static final String MULTIPART_UPLOAD_STARTED + = "multipart_upload_started"; + + private StoreStatisticNames() { + } + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StreamStatisticNames.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StreamStatisticNames.java new file mode 100644 index 0000000000000..02072d464debf --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StreamStatisticNames.java @@ -0,0 +1,292 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * These are common statistic names. + *

+ * When adding new common statistic name constants, please make them unique. + * By convention, they are implicitly unique: + *

    + *
  • + * The name of the constants are uppercase, words separated by + * underscores. + *
  • + *
  • + * The value of the constants are lowercase of the constant names. + *
  • + *
+ */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public final class StreamStatisticNames { + + /** + * Count of times the TCP stream was aborted. + * Value: {@value}. + */ + public static final String STREAM_READ_ABORTED = "stream_aborted"; + + /** + * Bytes read from an input stream in read() calls. + * Does not include bytes read and then discarded in seek/close etc. + * These are the bytes returned to the caller. + * Value: {@value}. + */ + public static final String STREAM_READ_BYTES + = "stream_read_bytes"; + + /** + * Count of bytes discarded by aborting an input stream . + * Value: {@value}. + */ + public static final String STREAM_READ_BYTES_DISCARDED_ABORT + = "stream_read_bytes_discarded_in_abort"; + + /** + * Count of bytes read and discarded when closing an input stream. + * Value: {@value}. + */ + public static final String STREAM_READ_BYTES_DISCARDED_CLOSE + = "stream_read_bytes_discarded_in_close"; + + /** + * Count of times the TCP stream was closed. + * Value: {@value}. + */ + public static final String STREAM_READ_CLOSED = "stream_read_closed"; + + /** + * Total count of times an attempt to close an input stream was made + * Value: {@value}. + */ + public static final String STREAM_READ_CLOSE_OPERATIONS + = "stream_read_close_operations"; + + /** + * Total count of times an input stream to was opened. + * For object stores, that means the count a GET request was initiated. + * Value: {@value}. + */ + public static final String STREAM_READ_OPENED = "stream_read_opened"; + + /** + * Count of exceptions raised during input stream reads. + * Value: {@value}. + */ + public static final String STREAM_READ_EXCEPTIONS = + "stream_read_exceptions"; + + /** + * Count of readFully() operations in an input stream. + * Value: {@value}. + */ + public static final String STREAM_READ_FULLY_OPERATIONS + = "stream_read_fully_operations"; + + /** + * Count of read() operations in an input stream. + * Value: {@value}. + */ + public static final String STREAM_READ_OPERATIONS = + "stream_read_operations"; + + /** + * Count of incomplete read() operations in an input stream, + * that is, when the bytes returned were less than that requested. + * Value: {@value}. + */ + public static final String STREAM_READ_OPERATIONS_INCOMPLETE + = "stream_read_operations_incomplete"; + + /** + * Count of version mismatches encountered while reading an input stream. + * Value: {@value}. + */ + public static final String STREAM_READ_VERSION_MISMATCHES + = "stream_read_version_mismatches"; + + /** + * Count of executed seek operations which went backwards in a stream. + * Value: {@value}. + */ + public static final String STREAM_READ_SEEK_BACKWARD_OPERATIONS = + "stream_read_seek_backward_operations"; + + /** + * Count of bytes moved backwards during seek operations + * in an input stream. + * Value: {@value}. + */ + public static final String STREAM_READ_SEEK_BYTES_BACKWARDS + = "stream_read_bytes_backwards_on_seek"; + + /** + * Count of bytes read and discarded during seek() in an input stream. + * Value: {@value}. + */ + public static final String STREAM_READ_SEEK_BYTES_DISCARDED = + "stream_read_seek_bytes_discarded"; + + /** + * Count of bytes skipped during forward seek operations. + * Value: {@value}. + */ + public static final String STREAM_READ_SEEK_BYTES_SKIPPED + = "stream_read_seek_bytes_skipped"; + + /** + * Count of executed seek operations which went forward in + * an input stream. + * Value: {@value}. + */ + public static final String STREAM_READ_SEEK_FORWARD_OPERATIONS + = "stream_read_seek_forward_operations"; + + /** + * Count of times the seek policy was dynamically changed + * in an input stream. + * Value: {@value}. + */ + public static final String STREAM_READ_SEEK_POLICY_CHANGED = + "stream_read_seek_policy_changed"; + + /** + * Count of seek operations in an input stream. + * Value: {@value}. + */ + public static final String STREAM_READ_SEEK_OPERATIONS = + "stream_read_seek_operations"; + + /** + * Count of {@code InputStream.skip()} calls. + * Value: {@value}. + */ + public static final String STREAM_READ_SKIP_OPERATIONS = + "stream_read_skip_operations"; + + /** + * Count bytes skipped in {@code InputStream.skip()} calls. + * Value: {@value}. + */ + public static final String STREAM_READ_SKIP_BYTES = + "stream_read_skip_bytes"; + + /** + * Total count of bytes read from an input stream. + * Value: {@value}. + */ + public static final String STREAM_READ_TOTAL_BYTES + = "stream_read_total_bytes"; + + /** + * Count of calls of {@code CanUnbuffer.unbuffer()}. + * Value: {@value}. + */ + public static final String STREAM_READ_UNBUFFERED + = "stream_read_unbuffered"; + + /** + * "Count of stream write failures reported. + * Value: {@value}. + */ + public static final String STREAM_WRITE_EXCEPTIONS = + "stream_write_exceptions"; + + /** + * Count of failures when finalizing a multipart upload: + * {@value}. + */ + public static final String STREAM_WRITE_EXCEPTIONS_COMPLETING_UPLOADS = + "stream_write_exceptions_completing_upload"; + + /** + * Count of block/partition uploads complete. + * Value: {@value}. + */ + public static final String STREAM_WRITE_BLOCK_UPLOADS + = "stream_write_block_uploads"; + + /** + * Count of number of block uploads aborted. + * Value: {@value}. + */ + public static final String STREAM_WRITE_BLOCK_UPLOADS_ABORTED + = "stream_write_block_uploads_aborted"; + + /** + * Count of block/partition uploads active. + * Value: {@value}. + */ + public static final String STREAM_WRITE_BLOCK_UPLOADS_ACTIVE + = "stream_write_block_uploads_active"; + + /** + * Gauge of data queued to be written. + * Value: {@value}. + */ + public static final String STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING = + "stream_write_block_uploads_data_pending"; + + /** + * Count of number of block uploads committed. + * Value: {@value}. + */ + public static final String STREAM_WRITE_BLOCK_UPLOADS_COMMITTED + = "stream_write_block_uploads_committed"; + + /** + * Gauge of block/partitions uploads queued to be written. + * Value: {@value}. + */ + public static final String STREAM_WRITE_BLOCK_UPLOADS_PENDING + = "stream_write_block_uploads_pending"; + + + /** + * "Count of bytes written to output stream including all not yet uploaded. + * {@value}. + */ + public static final String STREAM_WRITE_BYTES + = "stream_write_bytes"; + + /** + * Count of total time taken for uploads to complete. + * {@value}. + */ + public static final String STREAM_WRITE_TOTAL_TIME + = "stream_write_total_time"; + + /** + * Total queue duration of all block uploads. + * {@value}. + */ + public static final String STREAM_WRITE_QUEUE_DURATION + = "stream_write_queue_duration"; + + public static final String STREAM_WRITE_TOTAL_DATA + = "stream_write_total_data"; + + private StreamStatisticNames() { + } + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/AbstractIOStatisticsImpl.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/AbstractIOStatisticsImpl.java new file mode 100644 index 0000000000000..c701a509d8951 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/AbstractIOStatisticsImpl.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics.impl; + +import org.apache.hadoop.fs.statistics.IOStatistics; + +/** + * Base implementation in case common methods/fields need to be added + * in future. + */ +public abstract class AbstractIOStatisticsImpl implements IOStatistics { + + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/DynamicIOStatistics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/DynamicIOStatistics.java new file mode 100644 index 0000000000000..50c2625c3513d --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/DynamicIOStatistics.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics.impl; + +import java.util.Collections; +import java.util.Map; +import java.util.function.Function; + +import org.apache.hadoop.fs.statistics.MeanStatistic; + +/** + * These statistics are dynamically evaluated by the supplied + * String -> type functions. + * + * This allows statistic sources to supply a list of callbacks used to + * generate the statistics on demand; similar to some of the Coda Hale metrics. + * + * The evaluation actually takes place during the iteration's {@code next()} + * call. + */ +final class DynamicIOStatistics + extends AbstractIOStatisticsImpl { + + /** + * Counter evaluators. + */ + private final EvaluatingStatisticsMap counters + = new EvaluatingStatisticsMap<>(); + + private final EvaluatingStatisticsMap gauges + = new EvaluatingStatisticsMap<>(); + + private final EvaluatingStatisticsMap minimums + = new EvaluatingStatisticsMap<>(); + + private final EvaluatingStatisticsMap maximums + = new EvaluatingStatisticsMap<>(); + + private final EvaluatingStatisticsMap meanStatistics + = new EvaluatingStatisticsMap<>(MeanStatistic::copy); + + DynamicIOStatistics() { + } + + @Override + public Map counters() { + return Collections.unmodifiableMap(counters); + } + + @Override + public Map gauges() { + return Collections.unmodifiableMap(gauges); + } + + @Override + public Map minimums() { + return Collections.unmodifiableMap(minimums); + } + + @Override + public Map maximums() { + return Collections.unmodifiableMap(maximums); + } + + @Override + public Map meanStatistics() { + return Collections.unmodifiableMap(meanStatistics); + } + + /** + * add a mapping of a key to a counter function. + * @param key the key + * @param eval the evaluator + */ + void addCounterFunction(String key, Function eval) { + counters.addFunction(key, eval); + } + + /** + * add a mapping of a key to a gauge function. + * @param key the key + * @param eval the evaluator + */ + void addGaugeFunction(String key, Function eval) { + gauges.addFunction(key, eval); + } + + /** + * add a mapping of a key to a minimum function. + * @param key the key + * @param eval the evaluator + */ + void addMinimumFunction(String key, Function eval) { + minimums.addFunction(key, eval); + } + + /** + * add a mapping of a key to a maximum function. + * @param key the key + * @param eval the evaluator + */ + void addMaximumFunction(String key, Function eval) { + maximums.addFunction(key, eval); + } + + /** + * add a mapping of a key to a meanStatistic function. + * @param key the key + * @param eval the evaluator + */ + void addMeanStatisticFunction(String key, + Function eval) { + meanStatistics.addFunction(key, eval); + } + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/DynamicIOStatisticsBuilder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/DynamicIOStatisticsBuilder.java new file mode 100644 index 0000000000000..47a317076dcf2 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/DynamicIOStatisticsBuilder.java @@ -0,0 +1,248 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics.impl; + +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Function; +import java.util.function.ToLongFunction; + +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.MeanStatistic; +import org.apache.hadoop.metrics2.lib.MutableCounterLong; + +import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkState; + +/** + * Builder of {@link DynamicIOStatistics}. + * + * Instantiate through + * {@link IOStatisticsBinding#dynamicIOStatistics()}. + */ +public class DynamicIOStatisticsBuilder { + + /** + * the instance being built up. Will be null after the (single) + * call to {@link #build()}. + */ + private DynamicIOStatistics instance = new DynamicIOStatistics(); + + /** + * Build the IOStatistics instance. + * @return an instance. + * @throws IllegalStateException if the builder has already been built. + */ + public IOStatistics build() { + final DynamicIOStatistics stats = activeInstance(); + // stop the builder from working any more. + instance = null; + return stats; + } + + /** + * Get the statistics instance. + * @return the instance to build/return + * @throws IllegalStateException if the builder has already been built. + */ + private DynamicIOStatistics activeInstance() { + checkState(instance != null, "Already built"); + return instance; + } + + /** + * Add a new evaluator to the counter statistics. + * @param key key of this statistic + * @param eval evaluator for the statistic + * @return the builder. + */ + public DynamicIOStatisticsBuilder withLongFunctionCounter(String key, + ToLongFunction eval) { + activeInstance().addCounterFunction(key, eval::applyAsLong); + return this; + } + + /** + * Add a counter statistic to dynamically return the + * latest value of the source. + * @param key key of this statistic + * @param source atomic long counter + * @return the builder. + */ + public DynamicIOStatisticsBuilder withAtomicLongCounter(String key, + AtomicLong source) { + withLongFunctionCounter(key, s -> source.get()); + return this; + } + + /** + * Add a counter statistic to dynamically return the + * latest value of the source. + * @param key key of this statistic + * @param source atomic int counter + * @return the builder. + */ + public DynamicIOStatisticsBuilder withAtomicIntegerCounter(String key, + AtomicInteger source) { + withLongFunctionCounter(key, s -> source.get()); + return this; + } + + /** + * Build a dynamic counter statistic from a + * {@link MutableCounterLong}. + * @param key key of this statistic + * @param source mutable long counter + * @return the builder. + */ + public DynamicIOStatisticsBuilder withMutableCounter(String key, + MutableCounterLong source) { + withLongFunctionCounter(key, s -> source.value()); + return this; + } + + /** + * Add a new evaluator to the gauge statistics. + * @param key key of this statistic + * @param eval evaluator for the statistic + * @return the builder. + */ + public DynamicIOStatisticsBuilder withLongFunctionGauge(String key, + ToLongFunction eval) { + activeInstance().addGaugeFunction(key, eval::applyAsLong); + return this; + } + + /** + * Add a gauge statistic to dynamically return the + * latest value of the source. + * @param key key of this statistic + * @param source atomic long gauge + * @return the builder. + */ + public DynamicIOStatisticsBuilder withAtomicLongGauge(String key, + AtomicLong source) { + withLongFunctionGauge(key, s -> source.get()); + return this; + } + + /** + * Add a gauge statistic to dynamically return the + * latest value of the source. + * @param key key of this statistic + * @param source atomic int gauge + * @return the builder. + */ + public DynamicIOStatisticsBuilder withAtomicIntegerGauge(String key, + AtomicInteger source) { + withLongFunctionGauge(key, s -> source.get()); + return this; + } + + /** + * Add a new evaluator to the minimum statistics. + * @param key key of this statistic + * @param eval evaluator for the statistic + * @return the builder. + */ + public DynamicIOStatisticsBuilder withLongFunctionMinimum(String key, + ToLongFunction eval) { + activeInstance().addMinimumFunction(key, eval::applyAsLong); + return this; + } + + /** + * Add a minimum statistic to dynamically return the + * latest value of the source. + * @param key key of this statistic + * @param source atomic long minimum + * @return the builder. + */ + public DynamicIOStatisticsBuilder withAtomicLongMinimum(String key, + AtomicLong source) { + withLongFunctionMinimum(key, s -> source.get()); + return this; + } + + /** + * Add a minimum statistic to dynamically return the + * latest value of the source. + * @param key key of this statistic + * @param source atomic int minimum + * @return the builder. + */ + public DynamicIOStatisticsBuilder withAtomicIntegerMinimum(String key, + AtomicInteger source) { + withLongFunctionMinimum(key, s -> source.get()); + return this; + } + + + /** + * Add a new evaluator to the maximum statistics. + * @param key key of this statistic + * @param eval evaluator for the statistic + * @return the builder. + */ + public DynamicIOStatisticsBuilder withLongFunctionMaximum(String key, + ToLongFunction eval) { + activeInstance().addMaximumFunction(key, eval::applyAsLong); + return this; + } + + /** + * Add a maximum statistic to dynamically return the + * latest value of the source. + * @param key key of this statistic + * @param source atomic long maximum + * @return the builder. + */ + public DynamicIOStatisticsBuilder withAtomicLongMaximum(String key, + AtomicLong source) { + withLongFunctionMaximum(key, s -> source.get()); + return this; + } + + /** + * Add a maximum statistic to dynamically return the + * latest value of the source. + * @param key key of this statistic + * @param source atomic int maximum + * @return the builder. + */ + public DynamicIOStatisticsBuilder withAtomicIntegerMaximum(String key, + AtomicInteger source) { + withLongFunctionMaximum(key, s -> source.get()); + return this; + } + + /** + * Add a new evaluator to the mean statistics. + * + * This is a function which must return the mean and the sample count. + * @param key key of this statistic + * @param eval evaluator for the statistic + * @return the builder. + */ + public DynamicIOStatisticsBuilder withMeanStatisticFunction(String key, + Function eval) { + activeInstance().addMeanStatisticFunction(key, eval); + return this; + } + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/EmptyIOStatistics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/EmptyIOStatistics.java new file mode 100644 index 0000000000000..f474fc209771c --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/EmptyIOStatistics.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics.impl; + +import java.util.Map; + +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.MeanStatistic; + +import static java.util.Collections.emptyMap; + +/** + * An empty IO Statistics implementation for classes which always + * want to return a non-null set of statistics. + */ +final class EmptyIOStatistics extends AbstractIOStatisticsImpl { + + /** + * The sole instance of this class. + */ + private static final EmptyIOStatistics INSTANCE = new EmptyIOStatistics(); + + private EmptyIOStatistics() { + } + + @Override + public Map counters() { + return emptyMap(); + } + + @Override + public Map gauges() { + return emptyMap(); + } + + @Override + public Map minimums() { + return emptyMap(); + } + + @Override + public Map maximums() { + return emptyMap(); + } + + @Override + public Map meanStatistics() { + return emptyMap(); + } + + /** + * Get the single instance of this class. + * @return a shared, empty instance. + */ + public static IOStatistics getInstance() { + return INSTANCE; + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/EvaluatingStatisticsMap.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/EvaluatingStatisticsMap.java new file mode 100644 index 0000000000000..e4680f2d81fa0 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/EvaluatingStatisticsMap.java @@ -0,0 +1,202 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics.impl; + +import java.io.Serializable; +import java.util.Collection; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * A map of functions which can be invoked to dynamically + * create the value of an entry. + * @param type of entry value. + */ +final class EvaluatingStatisticsMap implements + Map { + + /** + * Functions to invoke when evaluating keys. + */ + private final Map> evaluators + = new ConcurrentHashMap<>(); + + /** + * Function to use when copying map values. + */ + private final Function copyFn; + + /** + * Construct with the copy function being simple passthrough. + */ + EvaluatingStatisticsMap() { + this(IOStatisticsBinding::passthroughFn); + } + + /** + * Construct with the copy function being that supplied in. + * @param copyFn copy function. + */ + EvaluatingStatisticsMap(final Function copyFn) { + this.copyFn = copyFn; + } + + /** + * add a mapping of a key to a function. + * @param key the key + * @param eval the evaluator + */ + void addFunction(String key, Function eval) { + evaluators.put(key, eval); + } + + @Override + public int size() { + return evaluators.size(); + } + + @Override + public boolean isEmpty() { + return evaluators.isEmpty(); + } + + @Override + public boolean containsKey(final Object key) { + return evaluators.containsKey(key); + } + + @Override + public boolean containsValue(final Object value) { + throw new UnsupportedOperationException(); + } + + @Override + public E get(final Object key) { + Function fn = evaluators.get(key); + return fn != null + ? fn.apply((String) key) + : null; + } + + @Override + public E put(final String key, final E value) { + throw new UnsupportedOperationException(); + } + + @Override + public E remove(final Object key) { + throw new UnsupportedOperationException(); + } + + @Override + public void putAll(final Map m) { + throw new UnsupportedOperationException(); + } + + @Override + public void clear() { + throw new UnsupportedOperationException(); + } + + @Override + public Set keySet() { + return evaluators.keySet(); + } + + /** + * Evaluate all the entries and provide a list of the results. + * + * This is not a snapshot, so if the evaluators actually return + * references to mutable objects (e.g. a MeanStatistic instance) + * then that value may still change. + * @return the current list of evaluated results. + */ + @Override + public Collection values() { + Set>> evalEntries = + evaluators.entrySet(); + return evalEntries.parallelStream().map((e) -> + e.getValue().apply(e.getKey())) + .collect(Collectors.toList()); + } + + /** + * Take a snapshot. + * @return a map snapshot. + */ + public Map snapshot() { + return IOStatisticsBinding.snapshotMap(this, copyFn); + } + + /** + * Creating the entry set forces an evaluation of the functions. + * + * This is not a snapshot, so if the evaluators actually return + * references to mutable objects (e.g. a MeanStatistic instance) + * then that value may still change. + * + * The evaluation may be parallelized. + * @return an evaluated set of values + */ + @Override + public synchronized Set> entrySet() { + Set>> evalEntries = + evaluators.entrySet(); + Set> r = evalEntries.parallelStream().map((e) -> + new EntryImpl<>(e.getKey(), e.getValue().apply(e.getKey()))) + .collect(Collectors.toSet()); + return r; + } + + /** + * Simple entry. + * @param entry type + */ + private static final class EntryImpl implements Entry { + + private String key; + + private E value; + + private EntryImpl(final String key, final E value) { + this.key = key; + this.value = value; + } + + @Override + public String getKey() { + return key; + } + + @Override + public E getValue() { + return value; + } + + @Override + public E setValue(final E val) { + this.value = val; + return val; + } + } + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java new file mode 100644 index 0000000000000..c3507dbc73ef4 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java @@ -0,0 +1,616 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics.impl; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.io.Serializable; +import java.util.Iterator; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.BiFunction; +import java.util.function.Function; + +import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; + +import org.apache.hadoop.fs.StorageStatistics; +import org.apache.hadoop.fs.statistics.DurationTracker; +import org.apache.hadoop.fs.statistics.DurationTrackerFactory; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.fs.statistics.MeanStatistic; +import org.apache.hadoop.util.functional.CallableRaisingIOE; +import org.apache.hadoop.util.functional.ConsumerRaisingIOE; +import org.apache.hadoop.util.functional.FunctionRaisingIOE; +import org.apache.hadoop.util.functional.InvocationRaisingIOE; + +import static org.apache.hadoop.fs.statistics.IOStatistics.MIN_UNSET_VALUE; +import static org.apache.hadoop.fs.statistics.impl.StubDurationTracker.STUB_DURATION_TRACKER; + +/** + * Support for implementing IOStatistics interfaces. + */ +public final class IOStatisticsBinding { + + /** Pattern used for each entry. */ + public static final String ENTRY_PATTERN = "(%s=%s)"; + + /** String to return when a source is null. */ + @VisibleForTesting + public static final String NULL_SOURCE = "()"; + + private IOStatisticsBinding() { + } + + /** + * Create IOStatistics from a storage statistics instance. + * + * This will be updated as the storage statistics change. + * @param storageStatistics source data. + * @return an IO statistics source. + */ + public static IOStatistics fromStorageStatistics( + StorageStatistics storageStatistics) { + DynamicIOStatisticsBuilder builder = dynamicIOStatistics(); + Iterator it = storageStatistics + .getLongStatistics(); + while (it.hasNext()) { + StorageStatistics.LongStatistic next = it.next(); + builder.withLongFunctionCounter(next.getName(), + k -> storageStatistics.getLong(k)); + } + return builder.build(); + } + + /** + * Create a builder for dynamic IO Statistics. + * @return a builder to be completed. + */ + public static DynamicIOStatisticsBuilder dynamicIOStatistics() { + return new DynamicIOStatisticsBuilder(); + } + + /** + * Get the shared instance of the immutable empty statistics + * object. + * @return an empty statistics object. + */ + public static IOStatistics emptyStatistics() { + return EmptyIOStatistics.getInstance(); + } + + /** + * Take an IOStatistics instance and wrap it in a source. + * @param statistics statistics. + * @return a source which will return the values + */ + public static IOStatisticsSource wrap(IOStatistics statistics) { + return new SourceWrappedStatistics(statistics); + } + + /** + * Create a builder for an {@link IOStatisticsStore}. + * + * @return a builder instance. + */ + public static IOStatisticsStoreBuilder iostatisticsStore() { + return new IOStatisticsStoreBuilderImpl(); + } + + /** + * Convert an entry to the string format used in logging. + * + * @param entry entry to evaluate + * @param entry type + * @return formatted string + */ + public static String entryToString( + final Map.Entry entry) { + return entryToString(entry.getKey(), entry.getValue()); + } + + /** + * Convert entry values to the string format used in logging. + * + * @param name statistic name + * @param value stat value + * @return formatted string + */ + public static String entryToString( + final String name, final E value) { + return String.format( + ENTRY_PATTERN, + name, + value); + } + + /** + * Copy into the dest map all the source entries. + * The destination is cleared first. + * @param entry type + * @param dest destination of the copy + * @param source source + * @param copyFn function to copy entries + * @return the destination. + */ + private static Map copyMap( + Map dest, + Map source, + Function copyFn) { + // we have to clone the values so that they aren't + // bound to the original values + dest.clear(); + source.entrySet() + .forEach(entry -> + dest.put(entry.getKey(), copyFn.apply(entry.getValue()))); + return dest; + } + + /** + * A passthrough copy operation suitable for immutable + * types, including numbers. + * @param src source object + * @return the source object + */ + public static E passthroughFn(E src) { + return src; + } + + /** + * Take a snapshot of a supplied map, where the copy option simply + * uses the existing value. + * + * For this to be safe, the map must refer to immutable objects. + * @param source source map + * @param type of values. + * @return a new map referencing the same values. + */ + public static Map snapshotMap( + Map source) { + return snapshotMap(source, + IOStatisticsBinding::passthroughFn); + } + + /** + * Take a snapshot of a supplied map, using the copy function + * to replicate the source values. + * @param source source map + * @param copyFn function to copy the value + * @param type of values. + * @return a concurrent hash map referencing the same values. + */ + public static + ConcurrentHashMap snapshotMap( + Map source, + Function copyFn) { + ConcurrentHashMap dest = new ConcurrentHashMap<>(); + copyMap(dest, source, copyFn); + return dest; + } + + /** + * Aggregate two maps so that the destination. + * @param type of values + * @param dest destination map. + * @param other other map + * @param aggregateFn function to aggregate the values. + * @param copyFn function to copy the value + */ + public static void aggregateMaps( + Map dest, + Map other, + BiFunction aggregateFn, + Function copyFn) { + // scan through the other hand map; copy + // any values not in the left map, + // aggregate those for which there is already + // an entry + other.entrySet().forEach(entry -> { + String key = entry.getKey(); + E rVal = entry.getValue(); + E lVal = dest.get(key); + if (lVal == null) { + dest.put(key, copyFn.apply(rVal)); + } else { + dest.put(key, aggregateFn.apply(lVal, rVal)); + } + }); + } + + /** + * Aggregate two counters. + * @param l left value + * @param r right value + * @return the aggregate value + */ + public static Long aggregateCounters(Long l, Long r) { + return Math.max(l, 0) + Math.max(r, 0); + } + + /** + * Add two gauges. + * @param l left value + * @param r right value + * @return aggregate value + */ + public static Long aggregateGauges(Long l, Long r) { + return l + r; + } + + + /** + * Aggregate two minimum values. + * @param l left + * @param r right + * @return the new minimum. + */ + public static Long aggregateMinimums(Long l, Long r) { + if (l == MIN_UNSET_VALUE) { + return r; + } else if (r == MIN_UNSET_VALUE) { + return l; + } else { + return Math.min(l, r); + } + } + + /** + * Aggregate two maximum values. + * @param l left + * @param r right + * @return the new minimum. + */ + public static Long aggregateMaximums(Long l, Long r) { + if (l == MIN_UNSET_VALUE) { + return r; + } else if (r == MIN_UNSET_VALUE) { + return l; + } else { + return Math.max(l, r); + } + } + + /** + * Aggregate the mean statistics. + * This returns a new instance. + * @param l left value + * @param r right value + * @return aggregate value + */ + public static MeanStatistic aggregateMeanStatistics( + MeanStatistic l, MeanStatistic r) { + MeanStatistic res = l.copy(); + res.add(r); + return res; + } + + /** + * Update a maximum value tracked in an atomic long. + * This is thread safe -it uses compareAndSet to ensure + * that Thread T1 whose sample is greater than the current + * value never overwrites an update from thread T2 whose + * sample was also higher -and which completed first. + * @param dest destination for all changes. + * @param sample sample to update. + */ + public static void maybeUpdateMaximum(AtomicLong dest, long sample) { + boolean done; + do { + long current = dest.get(); + if (sample > current) { + done = dest.compareAndSet(current, sample); + } else { + done = true; + } + } while (!done); + } + + /** + * Update a maximum value tracked in an atomic long. + * This is thread safe -it uses compareAndSet to ensure + * that Thread T1 whose sample is greater than the current + * value never overwrites an update from thread T2 whose + * sample was also higher -and which completed first. + * @param dest destination for all changes. + * @param sample sample to update. + */ + public static void maybeUpdateMinimum(AtomicLong dest, long sample) { + boolean done; + do { + long current = dest.get(); + if (current == MIN_UNSET_VALUE || sample < current) { + done = dest.compareAndSet(current, sample); + } else { + done = true; + } + } while (!done); + } + + /** + * Given an IOException raising function/lambda expression, + * return a new one which wraps the inner and tracks + * the duration of the operation, including whether + * it passes/fails. + * @param factory factory of duration trackers + * @param statistic statistic key + * @param inputFn input function + * @param type of argument to the input function. + * @param return type. + * @return a new function which tracks duration and failure. + */ + public static FunctionRaisingIOE trackFunctionDuration( + @Nullable DurationTrackerFactory factory, + String statistic, + FunctionRaisingIOE inputFn) { + return (x) -> { + // create the tracker outside try-with-resources so + // that failures can be set in the catcher. + DurationTracker tracker = createTracker(factory, statistic); + try { + // exec the input function and return its value + return inputFn.apply(x); + } catch (IOException | RuntimeException e) { + // input function failed: note it + tracker.failed(); + // and rethrow + throw e; + } finally { + // update the tracker. + // this is called after the catch() call would have + // set the failed flag. + tracker.close(); + } + }; + } + + /** + * Given a java function/lambda expression, + * return a new one which wraps the inner and tracks + * the duration of the operation, including whether + * it passes/fails. + * @param factory factory of duration trackers + * @param statistic statistic key + * @param inputFn input function + * @param type of argument to the input function. + * @param return type. + * @return a new function which tracks duration and failure. + */ + public static Function trackJavaFunctionDuration( + @Nullable DurationTrackerFactory factory, + String statistic, + Function inputFn) { + return (x) -> { + // create the tracker outside try-with-resources so + // that failures can be set in the catcher. + DurationTracker tracker = createTracker(factory, statistic); + try { + // exec the input function and return its value + return inputFn.apply(x); + } catch (RuntimeException e) { + // input function failed: note it + tracker.failed(); + // and rethrow + throw e; + } finally { + // update the tracker. + // this is called after the catch() call would have + // set the failed flag. + tracker.close(); + } + }; + } + + /** + * Given an IOException raising callable/lambda expression, + * execute it and update the relevant statistic. + * @param factory factory of duration trackers + * @param statistic statistic key + * @param input input callable. + * @param return type. + * @return the result of the operation. + */ + public static B trackDuration( + DurationTrackerFactory factory, + String statistic, + CallableRaisingIOE input) throws IOException { + return trackDurationOfOperation(factory, statistic, input).apply(); + } + + /** + * Given an IOException raising callable/lambda expression, + * execute it and update the relevant statistic. + * @param factory factory of duration trackers + * @param statistic statistic key + * @param input input callable. + */ + public static void trackDurationOfInvocation( + DurationTrackerFactory factory, + String statistic, + InvocationRaisingIOE input) throws IOException { + + // create the tracker outside try-with-resources so + // that failures can be set in the catcher. + DurationTracker tracker = createTracker(factory, statistic); + try { + // exec the input function and return its value + input.apply(); + } catch (IOException | RuntimeException e) { + // input function failed: note it + tracker.failed(); + // and rethrow + throw e; + } finally { + // update the tracker. + // this is called after the catch() call would have + // set the failed flag. + tracker.close(); + } + } + + /** + * Given an IOException raising callable/lambda expression, + * return a new one which wraps the inner and tracks + * the duration of the operation, including whether + * it passes/fails. + * @param factory factory of duration trackers + * @param statistic statistic key + * @param input input callable. + * @param return type. + * @return a new callable which tracks duration and failure. + */ + public static CallableRaisingIOE trackDurationOfOperation( + @Nullable DurationTrackerFactory factory, + String statistic, + CallableRaisingIOE input) { + return () -> { + // create the tracker outside try-with-resources so + // that failures can be set in the catcher. + DurationTracker tracker = createTracker(factory, statistic); + try { + // exec the input function and return its value + return input.apply(); + } catch (IOException | RuntimeException e) { + // input function failed: note it + tracker.failed(); + // and rethrow + throw e; + } finally { + // update the tracker. + // this is called after the catch() call would have + // set the failed flag. + tracker.close(); + } + }; + } + + /** + * Given an IOException raising Consumer, + * return a new one which wraps the inner and tracks + * the duration of the operation, including whether + * it passes/fails. + * @param factory factory of duration trackers + * @param statistic statistic key + * @param input input callable. + * @param return type. + * @return a new consumer which tracks duration and failure. + */ + public static ConsumerRaisingIOE trackDurationConsumer( + @Nullable DurationTrackerFactory factory, + String statistic, + ConsumerRaisingIOE input) { + return (B t) -> { + // create the tracker outside try-with-resources so + // that failures can be set in the catcher. + DurationTracker tracker = createTracker(factory, statistic); + try { + // exec the input function and return its value + input.accept(t); + } catch (IOException | RuntimeException e) { + // input function failed: note it + tracker.failed(); + // and rethrow + throw e; + } finally { + // update the tracker. + // this is called after the catch() call would have + // set the failed flag. + tracker.close(); + } + }; + } + + /** + * Given a callable/lambda expression, + * return a new one which wraps the inner and tracks + * the duration of the operation, including whether + * it passes/fails. + * @param factory factory of duration trackers + * @param statistic statistic key + * @param input input callable. + * @param return type. + * @return a new callable which tracks duration and failure. + */ + public static Callable trackDurationOfCallable( + @Nullable DurationTrackerFactory factory, + String statistic, + Callable input) { + return () -> { + // create the tracker outside try-with-resources so + // that failures can be set in the catcher. + DurationTracker tracker = createTracker(factory, statistic); + try { + // exec the input function and return its value + return input.call(); + } catch (RuntimeException e) { + // input function failed: note it + tracker.failed(); + // and rethrow + throw e; + } finally { + // update the tracker. + // this is called after any catch() call will have + // set the failed flag. + tracker.close(); + } + }; + } + + /** + * Create the tracker. If the factory is null, a stub + * tracker is returned. + * @param factory tracker factory + * @param statistic statistic to track + * @return a duration tracker. + */ + private static DurationTracker createTracker( + @Nullable final DurationTrackerFactory factory, + final String statistic) { + return factory != null + ? factory.trackDuration(statistic) + : STUB_DURATION_TRACKER; + } + + /** + * Create a DurationTrackerFactory which aggregates the tracking + * of two other factories. + * @param first first tracker factory + * @param second second tracker factory + * @return a factory + */ + public static DurationTrackerFactory pairedTrackerFactory( + final DurationTrackerFactory first, + final DurationTrackerFactory second) { + return new PairedDurationTrackerFactory(first, second); + } + + /** + * Publish the IOStatistics as a set of storage statistics. + * This is dynamic. + * @param name storage statistics name. + * @param scheme FS scheme; may be null. + * @param source IOStatistics source. + * @return a dynamic storage statistics object. + */ + public static StorageStatistics publishAsStorageStatistics( + String name, String scheme, IOStatistics source) { + return new StorageStatisticsFromIOStatistics(name, scheme, source); + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStore.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStore.java new file mode 100644 index 0000000000000..1b4139e463a9e --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStore.java @@ -0,0 +1,258 @@ +/* + * 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.time.Duration; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsAggregator; +import org.apache.hadoop.fs.statistics.DurationTrackerFactory; +import org.apache.hadoop.fs.statistics.MeanStatistic; + +/** + * Interface of an IOStatistics store intended for + * use in classes which track statistics for reporting. + */ +public interface IOStatisticsStore extends IOStatistics, + IOStatisticsAggregator, + DurationTrackerFactory { + + /** + * Increment a counter by one. + * + * No-op if the counter is unknown. + * @param key statistics key + * @return old value or, if the counter is unknown: 0 + */ + default long incrementCounter(String key) { + return incrementCounter(key, 1); + } + + /** + * Increment a counter. + * + * No-op if the counter is unknown. + * If the value is negative, it is ignored. + * @param key statistics key + * @param value value to increment + * @return the updated value or, if the counter is unknown: 0 + */ + long incrementCounter(String key, long value); + + /** + * Set a counter. + * + * No-op if the counter is unknown. + * @param key statistics key + * @param value value to set + */ + void setCounter(String key, long value); + + /** + * Set a gauge. + * + * No-op if the gauge is unknown. + * @param key statistics key + * @param value value to set + */ + void setGauge(String key, long value); + + /** + * Increment a gauge. + *

+ * No-op if the gauge is unknown. + *

+ * @param key statistics key + * @param value value to increment + * @return new value or 0 if the key is unknown + */ + long incrementGauge(String key, long value); + + /** + * Set a maximum. + * No-op if the maximum is unknown. + * @param key statistics key + * @param value value to set + */ + void setMaximum(String key, long value); + + /** + * Increment a maximum. + *

+ * No-op if the maximum is unknown. + *

+ * @param key statistics key + * @param value value to increment + * @return new value or 0 if the key is unknown + */ + long incrementMaximum(String key, long value); + + /** + * Set a minimum. + *

+ * No-op if the minimum is unknown. + *

+ * @param key statistics key + * @param value value to set + */ + void setMinimum(String key, long value); + + /** + * Increment a minimum. + *

+ * No-op if the minimum is unknown. + *

+ * @param key statistics key + * @param value value to increment + * @return new value or 0 if the key is unknown + */ + long incrementMinimum(String key, long value); + + /** + * Add a minimum sample: if less than the current value, + * updates the value. + *

+ * No-op if the minimum is unknown. + *

+ * @param key statistics key + * @param value sample value + */ + void addMinimumSample(String key, long value); + + /** + * Add a maximum sample: if greater than the current value, + * updates the value. + *

+ * No-op if the key is unknown. + *

+ * @param key statistics key + * @param value sample value + */ + void addMaximumSample(String key, long value); + + /** + * Set a mean statistic to a given value. + *

+ * No-op if the key is unknown. + *

+ * @param key statistic key + * @param value new value. + */ + void setMeanStatistic(String key, MeanStatistic value); + + /** + * Add a sample to the mean statistics. + *

+ * No-op if the key is unknown. + *

+ * @param key key + * @param value sample value. + */ + void addMeanStatisticSample(String key, long value); + + /** + * Reset all statistics. + * Unsynchronized. + */ + void reset(); + + /** + * Get a reference to the atomic instance providing the + * value for a specific counter. This is useful if + * the value is passed around. + * @param key statistic name + * @return the reference + * @throws NullPointerException if there is no entry of that name + */ + AtomicLong getCounterReference(String key); + + /** + * Get a reference to the atomic instance providing the + * value for a specific maximum. This is useful if + * the value is passed around. + * @param key statistic name + * @return the reference + * @throws NullPointerException if there is no entry of that name + */ + AtomicLong getMaximumReference(String key); + + /** + * Get a reference to the atomic instance providing the + * value for a specific minimum. This is useful if + * the value is passed around. + * @param key statistic name + * @return the reference + * @throws NullPointerException if there is no entry of that name + */ + AtomicLong getMinimumReference(String key); + + /** + * Get a reference to the atomic instance providing the + * value for a specific gauge. This is useful if + * the value is passed around. + * @param key statistic name + * @return the reference + * @throws NullPointerException if there is no entry of that name + */ + AtomicLong getGaugeReference(String key); + + /** + * Get a reference to the atomic instance providing the + * value for a specific meanStatistic. This is useful if + * the value is passed around. + * @param key statistic name + * @return the reference + * @throws NullPointerException if there is no entry of that name + */ + MeanStatistic getMeanStatistic(String key); + + /** + * Add a duration to the min/mean/max statistics, using the + * given prefix and adding a suffix for each specific value. + * + * The update is not-atomic, even though each individual statistic + * is updated thread-safely. If two threads update the values + * simultaneously, at the end of each operation the state will + * be correct. It is only during the sequence that the statistics + * may be observably inconsistent. + * @param prefix statistic prefix + * @param durationMillis duration in milliseconds. + */ + void addTimedOperation(String prefix, long durationMillis); + + /** + * Add a duration to the min/mean/max statistics, using the + * given prefix and adding a suffix for each specific value.; + * increment tha counter whose name == prefix. + * + * If any of the statistics are not registered, that part of + * the sequence will be omitted -the rest will proceed. + * + * The update is not-atomic, even though each individual statistic + * is updated thread-safely. If two threads update the values + * simultaneously, at the end of each operation the state will + * be correct. It is only during the sequence that the statistics + * may be observably inconsistent. + * @param prefix statistic prefix + * @param duration duration + */ + void addTimedOperation(String prefix, Duration duration); + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStoreBuilder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStoreBuilder.java new file mode 100644 index 0000000000000..d94a8389b7ff8 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStoreBuilder.java @@ -0,0 +1,75 @@ +/* + * 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; + +/** + * Builder of the {@link IOStatisticsStore} implementation. + */ +public interface IOStatisticsStoreBuilder { + + /** + * Declare a varargs list of counters to add. + * @param keys names of statistics. + * @return this builder. + */ + IOStatisticsStoreBuilder withCounters(String... keys); + + /** + * Declare a varargs list of gauges to add. + * @param keys names of statistics. + * @return this builder. + */ + IOStatisticsStoreBuilder withGauges(String... keys); + + /** + * Declare a varargs list of maximums to add. + * @param keys names of statistics. + * @return this builder. + */ + IOStatisticsStoreBuilder withMaximums(String... keys); + + /** + * Declare a varargs list of minimums to add. + * @param keys names of statistics. + * @return this builder. + */ + IOStatisticsStoreBuilder withMinimums(String... keys); + + /** + * Declare a varargs list of means to add. + * @param keys names of statistics. + * @return this builder. + */ + IOStatisticsStoreBuilder withMeanStatistics(String... keys); + + /** + * Add a statistic in the counter, min, max and mean maps for each + * declared statistic prefix. + * @param prefixes prefixes for the stats. + * @return this + */ + IOStatisticsStoreBuilder withDurationTracking( + String... prefixes); + + /** + * Build the collector. + * @return a new collector. + */ + IOStatisticsStore build(); +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStoreBuilderImpl.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStoreBuilderImpl.java new file mode 100644 index 0000000000000..0562271db3ef8 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStoreBuilderImpl.java @@ -0,0 +1,100 @@ +/* + * 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.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_FAILURES; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MAX; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MEAN; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MIN; + +/** + * Builder for an IOStatistics store.. + */ +final class IOStatisticsStoreBuilderImpl implements + IOStatisticsStoreBuilder { + + private final List counters = new ArrayList<>(); + + private final List gauges = new ArrayList<>(); + + private final List minimums = new ArrayList<>(); + + private final List maximums = new ArrayList<>(); + + private final List meanStatistics = new ArrayList<>(); + + @Override + public IOStatisticsStoreBuilderImpl withCounters(final String... keys) { + counters.addAll(Arrays.asList(keys)); + return this; + } + + @Override + public IOStatisticsStoreBuilderImpl withGauges(final String... keys) { + gauges.addAll(Arrays.asList(keys)); + return this; + } + + @Override + public IOStatisticsStoreBuilderImpl withMaximums(final String... keys) { + maximums.addAll(Arrays.asList(keys)); + return this; + } + + @Override + public IOStatisticsStoreBuilderImpl withMinimums(final String... keys) { + minimums.addAll(Arrays.asList(keys)); + return this; + } + + @Override + public IOStatisticsStoreBuilderImpl withMeanStatistics( + final String... keys) { + meanStatistics.addAll(Arrays.asList(keys)); + return this; + } + + @Override + public IOStatisticsStoreBuilderImpl withDurationTracking( + final String... prefixes) { + for (String p : prefixes) { + withCounters(p, p + SUFFIX_FAILURES); + withMinimums( + p + SUFFIX_MIN, + p + SUFFIX_FAILURES + SUFFIX_MIN); + withMaximums( + p + SUFFIX_MAX, + p + SUFFIX_FAILURES + SUFFIX_MAX); + withMeanStatistics( + p + SUFFIX_MEAN, + p + SUFFIX_FAILURES + SUFFIX_MEAN); + } + return this; + } + + @Override + public IOStatisticsStore build() { + return new IOStatisticsStoreImpl(counters, gauges, minimums, + maximums, meanStatistics); + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStoreImpl.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStoreImpl.java new file mode 100644 index 0000000000000..0471703b3b040 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStoreImpl.java @@ -0,0 +1,469 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics.impl; + +import javax.annotation.Nullable; +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicLong; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.statistics.DurationTracker; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.MeanStatistic; + +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.stubDurationTracker; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MAX; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MEAN; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MIN; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.aggregateMaximums; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.aggregateMinimums; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.dynamicIOStatistics; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.maybeUpdateMaximum; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.maybeUpdateMinimum; + +/** + * Implementation of {@link IOStatisticsStore}. + *

+ * A ConcurrentHashMap of each set of statistics is created; + * the AtomicLong/MeanStatistic entries are fetched as required. + * When the statistics are updated, the referenced objects + * are updated rather than new values set in the map. + *

+ */ +final class IOStatisticsStoreImpl extends WrappedIOStatistics + implements IOStatisticsStore { + + /** + * Log changes at debug. + * Noisy, but occasionally useful. + */ + private static final Logger LOG = + LoggerFactory.getLogger(IOStatisticsStoreImpl.class); + + /** All the counters are atomic longs. */ + private final Map counterMap = new ConcurrentHashMap<>(); + + /** All the gauges are atomic longs. */ + private final Map gaugeMap = new ConcurrentHashMap<>(); + + /** All the minimum values are atomic longs. */ + private final Map minimumMap = new ConcurrentHashMap<>(); + + /** All the maximum values are atomic longs. */ + private final Map maximumMap = new ConcurrentHashMap<>(); + + /** + * The mean statistics. + * Relies on the MeanStatistic operations being synchronized. + */ + private final Map meanStatisticMap + = new ConcurrentHashMap<>(); + + /** + * Constructor invoked via the builder. + * @param counters keys to use for the counter statistics. + * @param gauges names of gauges + * @param minimums names of minimums + * @param maximums names of maximums + * @param meanStatistics names of mean statistics. + */ + IOStatisticsStoreImpl( + final List counters, + final List gauges, + final List minimums, + final List maximums, + final List meanStatistics) { + // initially create the superclass with no wrapped mapping; + super(null); + + // now construct a dynamic statistics source mapping to + // the various counters, gauges etc dynamically created + // into maps + DynamicIOStatisticsBuilder builder = dynamicIOStatistics(); + if (counters != null) { + for (String key : counters) { + AtomicLong counter = new AtomicLong(); + counterMap.put(key, counter); + builder.withAtomicLongCounter(key, counter); + } + } + if (gauges != null) { + for (String key : gauges) { + AtomicLong gauge = new AtomicLong(); + gaugeMap.put(key, gauge); + builder.withAtomicLongGauge(key, gauge); + } + } + if (maximums != null) { + for (String key : maximums) { + AtomicLong maximum = new AtomicLong(MAX_UNSET_VALUE); + maximumMap.put(key, maximum); + builder.withAtomicLongMaximum(key, maximum); + } + } + if (minimums != null) { + for (String key : minimums) { + AtomicLong minimum = new AtomicLong(MIN_UNSET_VALUE); + minimumMap.put(key, minimum); + builder.withAtomicLongMinimum(key, minimum); + } + } + if (meanStatistics != null) { + for (String key : meanStatistics) { + meanStatisticMap.put(key, new MeanStatistic()); + builder.withMeanStatisticFunction(key, k -> meanStatisticMap.get(k)); + } + } + setWrapped(builder.build()); + } + + /** + * Set an atomic long to a value. + * @param aLong atomic long; may be null + * @param value value to set to + */ + private void setAtomicLong(final AtomicLong aLong, final long value) { + if (aLong != null) { + aLong.set(value); + } + } + + /** + * increment an atomic long and return its value; + * null long is no-op returning 0. + * @param aLong atomic long; may be null + * param increment amount to increment; negative for a decrement + * @return final value or 0 if the long is null + */ + private long incAtomicLong(final AtomicLong aLong, + final long increment) { + if (aLong != null) { + // optimization: zero is a get rather than addAndGet() + return increment != 0 + ? aLong.addAndGet(increment) + : aLong.get(); + } else { + return 0; + } + } + + @Override + public void setCounter(final String key, final long value) { + setAtomicLong(counterMap.get(key), value); + LOG.debug("Setting counter {} to {}", key, value); + } + + @Override + public long incrementCounter(final String key, final long value) { + AtomicLong counter = counterMap.get(key); + if (counter == null) { + LOG.debug("Ignoring counter increment for unknown counter {}", + key); + return 0; + } + if (value < 0) { + LOG.debug("Ignoring negative increment value {} for counter {}", + value, key); + // returns old value + return counter.get(); + } else { + long l = incAtomicLong(counter, value); + LOG.debug("Incrementing counter {} by {} with final value {}", + key, value, l); + return l; + } + } + + @Override + public void setMaximum(final String key, final long value) { + setAtomicLong(maximumMap.get(key), value); + } + + @Override + public long incrementMaximum(final String key, final long value) { + return incAtomicLong(maximumMap.get(key), value); + } + + @Override + public void setMinimum(final String key, final long value) { + setAtomicLong(minimumMap.get(key), value); + } + + @Override + public long incrementMinimum(final String key, final long value) { + return incAtomicLong(minimumMap.get(key), value); + } + + @Override + public void addMinimumSample(final String key, final long value) { + AtomicLong min = minimumMap.get(key); + if (min != null) { + maybeUpdateMinimum(min, value); + } + } + + @Override + public void addMaximumSample(final String key, final long value) { + AtomicLong max = maximumMap.get(key); + if (max != null) { + maybeUpdateMaximum(max, value); + } + } + + @Override + public void setGauge(final String key, final long value) { + setAtomicLong(gaugeMap.get(key), value); + } + + @Override + public long incrementGauge(final String key, final long value) { + return incAtomicLong(gaugeMap.get(key), value); + } + + @Override + public void setMeanStatistic(final String key, final MeanStatistic value) { + final MeanStatistic ref = meanStatisticMap.get(key); + if (ref != null) { + ref.set(value); + } + } + + @Override + public void addMeanStatisticSample(final String key, final long value) { + final MeanStatistic ref = meanStatisticMap.get(key); + if (ref != null) { + ref.addSample(value); + } + } + + /** + * Reset all statistics. + */ + @Override + public synchronized void reset() { + counterMap.values().forEach(a -> a.set(0)); + gaugeMap.values().forEach(a -> a.set(0)); + minimumMap.values().forEach(a -> a.set(0)); + maximumMap.values().forEach(a -> a.set(0)); + meanStatisticMap.values().forEach(a -> a.clear()); + } + + /** + * Aggregate those statistics which the store is tracking; + * ignore the rest. + * + * @param source statistics; may be null + * @return true if a statistics reference was supplied/aggregated. + */ + @Override + public synchronized boolean aggregate( + @Nullable final IOStatistics source) { + + if (source == null) { + return false; + } + // counters: addition + Map sourceCounters = source.counters(); + counterMap.entrySet(). + forEach(e -> { + Long sourceValue = lookupQuietly(sourceCounters, e.getKey()); + if (sourceValue != null) { + e.getValue().addAndGet(sourceValue); + } + }); + // gauge: add positive values only + Map sourceGauges = source.gauges(); + gaugeMap.entrySet().forEach(e -> { + Long sourceGauge = lookupQuietly(sourceGauges, e.getKey()); + if (sourceGauge != null && sourceGauge > 0) { + e.getValue().addAndGet(sourceGauge); + } + }); + // min: min of current and source + Map sourceMinimums = source.minimums(); + minimumMap.entrySet().forEach(e -> { + Long sourceValue = lookupQuietly(sourceMinimums, e.getKey()); + if (sourceValue != null) { + AtomicLong dest = e.getValue(); + dest.set(aggregateMaximums(dest.get(), sourceValue)); + dest.set(aggregateMinimums(dest.get(), sourceValue)); + } + }); + // max: max of current and source + Map sourceMaximums = source.maximums(); + maximumMap.entrySet().forEach(e -> { + Long sourceValue = lookupQuietly(sourceMaximums, e.getKey()); + if (sourceValue != null) { + AtomicLong dest = e.getValue(); + dest.set(aggregateMaximums(dest.get(), sourceValue)); + } + }); + // the most complex + Map sourceMeans = source.meanStatistics(); + meanStatisticMap.entrySet().forEach(e -> { + MeanStatistic current = e.getValue(); + MeanStatistic sourceValue = lookupQuietly( + sourceMeans, e.getKey()); + if (sourceValue != null) { + current.add(sourceValue); + } + }); + return true; + } + + /** + * Get a reference to the map type providing the + * value for a specific key, raising an exception if + * there is no entry for that key. + * @param type of map/return type. + * @param map map to look up + * @param key statistic name + * @return the value + * @throws NullPointerException if there is no entry of that name + */ + private static T lookup(final Map map, String key) { + T val = map.get(key); + requireNonNull(val, () -> ("unknown statistic " + key)); + return val; + } + + /** + * Get a reference to the map type providing the + * value for a specific key, returning null if it not found. + * @param type of map/return type. + * @param map map to look up + * @param key statistic name + * @return the value + */ + private static T lookupQuietly(final Map map, String key) { + return map.get(key); + } + + /** + * Get a reference to the atomic instance providing the + * value for a specific counter. This is useful if + * the value is passed around. + * @param key statistic name + * @return the reference + * @throws NullPointerException if there is no entry of that name + */ + @Override + public AtomicLong getCounterReference(String key) { + return lookup(counterMap, key); + } + + /** + * Get a reference to the atomic instance providing the + * value for a specific maximum. This is useful if + * the value is passed around. + * @param key statistic name + * @return the reference + * @throws NullPointerException if there is no entry of that name + */ + @Override + public AtomicLong getMaximumReference(String key) { + return lookup(maximumMap, key); + } + + /** + * Get a reference to the atomic instance providing the + * value for a specific minimum. This is useful if + * the value is passed around. + * @param key statistic name + * @return the reference + * @throws NullPointerException if there is no entry of that name + */ + @Override + public AtomicLong getMinimumReference(String key) { + return lookup(minimumMap, key); + } + + /** + * Get a reference to the atomic instance providing the + * value for a specific gauge. This is useful if + * the value is passed around. + * @param key statistic name + * @return the reference + * @throws NullPointerException if there is no entry of that name + */ + @Override + public AtomicLong getGaugeReference(String key) { + return lookup(gaugeMap, key); + } + + /** + * Get a mean statistic. + * @param key statistic name + * @return the reference + * @throws NullPointerException if there is no entry of that name + */ + @Override + public MeanStatistic getMeanStatistic(String key) { + return lookup(meanStatisticMap, key); + } + + /** + * Add a duration to the min/mean/max statistics, using the + * given prefix and adding a suffix for each specific value. + *

+ * The update is non -atomic, even though each individual statistic + * is updated thread-safely. If two threads update the values + * simultaneously, at the end of each operation the state will + * be correct. It is only during the sequence that the statistics + * may be observably inconsistent. + *

+ * @param prefix statistic prefix + * @param durationMillis duration in milliseconds. + */ + @Override + public void addTimedOperation(String prefix, long durationMillis) { + addMeanStatisticSample(prefix + SUFFIX_MEAN, durationMillis); + addMinimumSample(prefix + SUFFIX_MIN, durationMillis); + addMaximumSample(prefix + SUFFIX_MAX, durationMillis); + } + + @Override + public void addTimedOperation(String prefix, Duration duration) { + addTimedOperation(prefix, duration.toMillis()); + } + + /** + * If the store is tracking the given key, return the + * duration tracker for it. If not tracked, return the + * stub tracker. + * @param key statistic key prefix + * @param count #of times to increment the matching counter in this + * operation. + * @return a tracker. + */ + @Override + public DurationTracker trackDuration(final String key, final long count) { + if (counterMap.containsKey(key)) { + return new StatisticDurationTracker(this, key, count); + } else { + return stubDurationTracker(); + } + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/PairedDurationTrackerFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/PairedDurationTrackerFactory.java new file mode 100644 index 0000000000000..33b13f78418a9 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/PairedDurationTrackerFactory.java @@ -0,0 +1,93 @@ +/* + * 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.time.Duration; + +import org.apache.hadoop.fs.statistics.DurationTracker; +import org.apache.hadoop.fs.statistics.DurationTrackerFactory; + +/** + * A duration tracker factory which aggregates two other trackers + * to have the same lifecycle. + * + * This is to ease having instance-level tracking alongside global + * values, such as an input stream and a filesystem. + * + * It's got some inefficiencies -assuming system time is used for + * the tracking, System.currentTimeMillis will be invoked twice + * at each point of the process -and the results may actually be different. + * However, it enables multiple duration tracker factories to be given the + * opportunity to collect the statistics. + */ +final class PairedDurationTrackerFactory implements DurationTrackerFactory { + + private final DurationTrackerFactory local; + private final DurationTrackerFactory global; + + PairedDurationTrackerFactory(final DurationTrackerFactory local, + final DurationTrackerFactory global) { + this.local = local; + this.global = global; + } + + @Override + public DurationTracker trackDuration(final String key, final long count) { + return new PairedDurationTracker( + global.trackDuration(key, count), + local.trackDuration(key, count)); + } + + /** + * Tracker which wraps the two duration trackers created for the operation. + */ + private static final class PairedDurationTracker + implements DurationTracker { + private final DurationTracker firstDuration; + private final DurationTracker secondDuration; + + private PairedDurationTracker( + final DurationTracker firstDuration, + final DurationTracker secondDuration) { + this.firstDuration = firstDuration; + this.secondDuration = secondDuration; + } + + @Override + public void failed() { + firstDuration.failed(); + secondDuration.failed(); + } + + @Override + public void close() { + firstDuration.close(); + secondDuration.close(); + } + + /** + * @return the global duration + */ + @Override + public Duration asDuration() { + return firstDuration.asDuration(); + } + } + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/SourceWrappedStatistics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/SourceWrappedStatistics.java new file mode 100644 index 0000000000000..5aced7c5cddbf --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/SourceWrappedStatistics.java @@ -0,0 +1,44 @@ +/* + * 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; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; + +/** + * Wrap a statistics instance with an {@link IOStatisticsSource} + * instance which will then serve up the statistics when asked. + */ +public class SourceWrappedStatistics implements IOStatisticsSource { + + private final IOStatistics source; + + /** + * Constructor. + * @param source source of statistics. + */ + public SourceWrappedStatistics(final IOStatistics source) { + this.source = source; + } + + @Override + public IOStatistics getIOStatistics() { + return source; + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StatisticDurationTracker.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StatisticDurationTracker.java new file mode 100644 index 0000000000000..ef9e7cb107a0d --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StatisticDurationTracker.java @@ -0,0 +1,106 @@ +/* + * 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.DurationTracker; +import org.apache.hadoop.fs.statistics.StoreStatisticNames; +import org.apache.hadoop.util.OperationDuration; + +/** + * Track the duration of an object. + * + * When closed the + * min/max/mean statistics are updated. + * + * In the constructor, the counter with name of 'key' is + * incremented -default is by 1, but can be set to other + * values, including 0. + */ +public class StatisticDurationTracker extends OperationDuration + implements DurationTracker { + + /** + * Statistics to update. + */ + private final IOStatisticsStore iostats; + + /** + * Key to use as prefix of values. + */ + private final String key; + + /** + * Flag to indicate the operation failed. + */ + private boolean failed; + + /** + * Constructor -increments the counter by 1. + * @param iostats statistics to update + * @param key prefix of values. + */ + public StatisticDurationTracker( + final IOStatisticsStore iostats, + final String key) { + this(iostats, key, 1); + } + + /** + * Constructor. + * If the supplied count is greater than zero, the counter + * of the key name is updated. + * @param iostats statistics to update + * @param key Key to use as prefix of values. + * @param count #of times to increment the matching counter. + */ + public StatisticDurationTracker( + final IOStatisticsStore iostats, + final String key, + final long count) { + this.iostats = iostats; + this.key = key; + if (count > 0) { + iostats.incrementCounter(key, count); + } + } + + @Override + public void failed() { + failed = true; + } + + /** + * Set the finished time and then update the statistics. + * If the operation failed then the key + .failures counter will be + * incremented by one. + * The operation min/mean/max values will be updated with the duration; + * on a failure these will all be the .failures metrics. + */ + @Override + public void close() { + finished(); + String name = key; + if (failed) { + // failure: + name = key + StoreStatisticNames.SUFFIX_FAILURES; + iostats.incrementCounter(name); + } + iostats.addTimedOperation(name, asDuration()); + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StorageStatisticsFromIOStatistics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StorageStatisticsFromIOStatistics.java new file mode 100644 index 0000000000000..a55f04cae8c4c --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StorageStatisticsFromIOStatistics.java @@ -0,0 +1,98 @@ +/* + * 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.Iterator; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import org.apache.hadoop.fs.StorageStatistics; +import org.apache.hadoop.fs.statistics.IOStatistics; + +/** + * Returns all the counters of an IOStatistics instance as StorageStatistics. + * This is dynamic. + * The {@link #reset()} is downgraded to a no-op. + */ +public class StorageStatisticsFromIOStatistics + extends StorageStatistics + implements Iterable { + + private final IOStatistics ioStatistics; + private final String scheme; + + /** + * Instantiate. + * @param name storage statistics name. + * @param scheme FS scheme; may be null. + * @param ioStatistics IOStatistics source. + */ + public StorageStatisticsFromIOStatistics( + final String name, + final String scheme, + final IOStatistics ioStatistics) { + super(name); + this.scheme = scheme; + this.ioStatistics = ioStatistics; + } + + @Override + public Iterator iterator() { + return getLongStatistics(); + } + + /** + * Take a snapshot of the current counter values + * and return an iterator over them. + * @return all the counter statistics. + */ + @Override + public Iterator getLongStatistics() { + final Set> counters = counters() + .entrySet(); + return counters.stream().map(e -> + new StorageStatistics.LongStatistic(e.getKey(), e.getValue())) + .collect(Collectors.toSet()).iterator(); + } + + private Map counters() { + return ioStatistics.counters(); + } + + @Override + public Long getLong(final String key) { + return counters().get(key); + } + + @Override + public boolean isTracked(final String key) { + return counters().containsKey(key); + } + + @Override + public void reset() { + /* no-op */ + } + + @Override + public String getScheme() { + return scheme; + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StubDurationTracker.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StubDurationTracker.java new file mode 100644 index 0000000000000..638a9da9c7b51 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StubDurationTracker.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.statistics.impl; + +import java.time.Duration; + +import org.apache.hadoop.fs.statistics.DurationTracker; + +/** + * A simple stub duration tracker which can be issued in interfaces + * and other places where full duration tracking is not implemented. + */ +public final class StubDurationTracker implements DurationTracker { + + public static final DurationTracker STUB_DURATION_TRACKER = + new StubDurationTracker(); + + private StubDurationTracker() { + } + + @Override + public void failed() { + + } + + @Override + public void close() { + + } + + @Override + public Duration asDuration() { + return Duration.ZERO; + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StubDurationTrackerFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StubDurationTrackerFactory.java new file mode 100644 index 0000000000000..8856b6330cee6 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StubDurationTrackerFactory.java @@ -0,0 +1,44 @@ +/* + * 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.DurationTracker; +import org.apache.hadoop.fs.statistics.DurationTrackerFactory; + +/** + * This is a stub factory which always returns no-op duration + * trackers. Allows for code to always be handed a factory. + */ +public final class StubDurationTrackerFactory + implements DurationTrackerFactory { + + /** + * Single instance. + */ + public static final StubDurationTrackerFactory STUB_DURATION_TRACKER_FACTORY + = new StubDurationTrackerFactory(); + + private StubDurationTrackerFactory() { + } + + @Override + public DurationTracker trackDuration(final String key, final long count) { + return StubDurationTracker.STUB_DURATION_TRACKER; + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/WrappedIOStatistics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/WrappedIOStatistics.java new file mode 100644 index 0000000000000..4e5fc6a6a1071 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/WrappedIOStatistics.java @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics.impl; + +import java.util.Map; + +import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; + +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.MeanStatistic; + +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToString; + +/** + * Wrap IOStatistics source with another (dynamic) wrapper. + */ +public class WrappedIOStatistics extends AbstractIOStatisticsImpl { + + /** + * The wrapped statistics. + */ + private IOStatistics wrapped; + + /** + * Instantiate. + * @param wrapped nullable wrapped statistics. + */ + public WrappedIOStatistics(final IOStatistics wrapped) { + this.wrapped = wrapped; + } + + /** + * Instantiate without setting the statistics. + * This is for subclasses which build up the map during their own + * construction. + */ + protected WrappedIOStatistics() { + } + + @Override + public Map counters() { + return getWrapped().counters(); + } + + /** + * Get at the wrapped inner statistics. + * @return the wrapped value + */ + protected IOStatistics getWrapped() { + return wrapped; + } + + /** + * Set the wrapped statistics. + * Will fail if the field is already set. + * @param wrapped new value + */ + protected void setWrapped(final IOStatistics wrapped) { + Preconditions.checkState(this.wrapped == null, + "Attempted to overwrite existing wrapped statistics"); + this.wrapped = wrapped; + } + + @Override + public Map gauges() { + return getWrapped().gauges(); + } + + @Override + public Map minimums() { + return getWrapped().minimums(); + } + + @Override + public Map maximums() { + return getWrapped().maximums(); + } + + @Override + public Map meanStatistics() { + return getWrapped().meanStatistics(); + } + + /** + * Return the statistics dump of the wrapped statistics. + * @return the statistics for logging. + */ + @Override + public String toString() { + return ioStatisticsToString(wrapped); + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/package-info.java new file mode 100644 index 0000000000000..3ff7dacadce7a --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/package-info.java @@ -0,0 +1,31 @@ +/* + * 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. + */ + +/** + * Implementation support for statistics. + * For use internally; external filesystems MAY use this if the implementors + * accept that it is unstable and that incompatible changes may take + * place over minor point releases. + */ + +@InterfaceAudience.LimitedPrivate("Filesystems") +@InterfaceStability.Unstable +package org.apache.hadoop.fs.statistics.impl; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/package-info.java new file mode 100644 index 0000000000000..bf46b33a516c6 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/package-info.java @@ -0,0 +1,134 @@ +/* + * 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. + */ + +/** + * This package contains support for statistic collection and reporting. + * This is the public API; implementation classes are to be kept elsewhere. + *

+ * 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) to see if it + * implements {@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 for + * input and output streams. + *

+ * Utility classes: + *

    + *
  • + * {@link org.apache.hadoop.fs.statistics.IOStatisticsSupport}. + * General support, including the ability to take a serializable + * snapshot of the current state of an IOStatistics instance. + *
  • + *
  • + * {@link org.apache.hadoop.fs.statistics.IOStatisticsLogging}. + * Methods for robust/on-demand string conversion, designed + * for use in logging statements and {@code toString()} implementations. + *
  • + *
  • + * {@link org.apache.hadoop.fs.statistics.IOStatisticsSnapshot}. + * A static snaphot of statistics which can be marshalled via + * java serialization or as JSON via jackson. It supports + * aggregation, so can be used to generate aggregate statistics. + *
  • + *
+ * + *

+ * Implementors notes: + *

    + *
  1. + * IOStatistics keys SHOULD be standard names where possible. + *
  2. + *
  3. + * An IOStatistics instance MUST be unique to that specific instance of + * {@link org.apache.hadoop.fs.statistics.IOStatisticsSource}. + * (i.e. not shared the way StorageStatistics are) + *
  4. + *
  5. + * MUST return the same values irrespective of which thread the statistics are + * retrieved or its keys evaluated. + *
  6. + *
  7. + * MUST NOT remove keys once a statistic instance has been created. + *
  8. + *
  9. + * MUST NOT add keys once a statistic instance has been created. + *
  10. + *
  11. + * MUST NOT block for long periods of time while blocking operations + * (reads, writes) are taking place in the source. + * That is: minimal synchronization points (AtomicLongs etc.) may be + * used to share values, but retrieval of statistics should + * be fast and return values even while slow/blocking remote IO is underway. + *
  12. + *
  13. + * MUST support value enumeration and retrieval after the source has been + * closed. + *
  14. + *
  15. + * SHOULD NOT have back-references to potentially expensive objects + * (filesystem instances etc.) + *
  16. + *
  17. + * SHOULD provide statistics which can be added to generate aggregate + * statistics. + *
  18. + *
+ */ + +@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/main/java/org/apache/hadoop/io/compress/CompressionInputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionInputStream.java index 2dfa30bf76ec4..55bb132e9c87c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionInputStream.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionInputStream.java @@ -25,6 +25,10 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.PositionedReadable; import org.apache.hadoop.fs.Seekable; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.fs.statistics.IOStatisticsSupport; + /** * A compression input stream. * @@ -34,7 +38,8 @@ */ @InterfaceAudience.Public @InterfaceStability.Evolving -public abstract class CompressionInputStream extends InputStream implements Seekable { +public abstract class CompressionInputStream extends InputStream + implements Seekable, IOStatisticsSource { /** * The input stream to be compressed. */ @@ -68,7 +73,16 @@ public void close() throws IOException { } } } - + + /** + * Return any IOStatistics provided by the underlying stream. + * @return IO stats from the inner stream. + */ + @Override + public IOStatistics getIOStatistics() { + return IOStatisticsSupport.retrieveIOStatistics(in); + } + /** * Read bytes from the stream. * Made abstract to prevent leakage to underlying stream. diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionOutputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionOutputStream.java index 71c7f32e665e5..2a11ace81702c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionOutputStream.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionOutputStream.java @@ -23,13 +23,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; /** * A compression output stream. */ @InterfaceAudience.Public @InterfaceStability.Evolving -public abstract class CompressionOutputStream extends OutputStream { +public abstract class CompressionOutputStream extends OutputStream + implements IOStatisticsSource { /** * The output stream to be compressed. */ @@ -94,4 +98,12 @@ public void flush() throws IOException { */ public abstract void resetState() throws IOException; + /** + * Return any IOStatistics provided by the underlying stream. + * @return IO stats from the inner stream. + */ + @Override + public IOStatistics getIOStatistics() { + return IOStatisticsSupport.retrieveIOStatistics(out); + } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LineReader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LineReader.java index e2cd3048d5843..520ddf6bdf401 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LineReader.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LineReader.java @@ -25,6 +25,9 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; +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.Text; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY; @@ -42,7 +45,7 @@ */ @InterfaceAudience.LimitedPrivate({"MapReduce"}) @InterfaceStability.Unstable -public class LineReader implements Closeable { +public class LineReader implements Closeable, IOStatisticsSource { private static final int DEFAULT_BUFFER_SIZE = 64 * 1024; private int bufferSize = DEFAULT_BUFFER_SIZE; private InputStream in; @@ -148,7 +151,16 @@ public LineReader(InputStream in, Configuration conf, public void close() throws IOException { in.close(); } - + + /** + * Return any IOStatistics provided by the source. + * @return IO stats from the input stream. + */ + @Override + public IOStatistics getIOStatistics() { + return IOStatisticsSupport.retrieveIOStatistics(in); + } + /** * Read one line from the InputStream into the given Text. * diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/OperationDuration.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/OperationDuration.java index 3276d2138bbfc..fdd25286a2300 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/OperationDuration.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/OperationDuration.java @@ -18,48 +18,98 @@ package org.apache.hadoop.util; +import java.time.Duration; + import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; /** * Little duration counter. */ -@InterfaceAudience.Private +@InterfaceAudience.Public @InterfaceStability.Unstable public class OperationDuration { + /** + * Time in millis when the operation started. + */ private final long started; + + /** + * Time when the operation finished. + */ private long finished; + /** + * Instantiate. + * The start time and finished time are both set + * to the current clock time. + */ public OperationDuration() { started = time(); finished = started; } + /** + * Evaluate the system time. + * @return the current clock time. + */ protected long time() { return System.currentTimeMillis(); } + /** + * Update the finished time with the current system time. + */ public void finished() { finished = time(); } + /** + * Return the duration as {@link #humanTime(long)}. + * @return a printable duration. + */ public String getDurationString() { return humanTime(value()); } + /** + * Convert to a human time of minutes:seconds.millis. + * @param time time to humanize. + * @return a printable value. + */ public static String humanTime(long time) { long seconds = (time / 1000); long minutes = (seconds / 60); return String.format("%d:%02d.%03ds", minutes, seconds % 60, time % 1000); } + /** + * Return the duration as {@link #humanTime(long)}. + * @return a printable duration. + */ @Override public String toString() { return getDurationString(); } + /** + * Get the duration in milliseconds. + *

+ * This will be 0 until a call + * to {@link #finished()} has been made. + * @return the currently recorded duration. + */ public long value() { return finished -started; } + + /** + * Get the duration of an operation as a java Duration + * instance. + * @return a duration. + */ + public Duration asDuration() { + return Duration.ofMillis(value()); + } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/BiFunctionRaisingIOE.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/BiFunctionRaisingIOE.java new file mode 100644 index 0000000000000..ea17c16d01e87 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/BiFunctionRaisingIOE.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.util.functional; + +import java.io.IOException; + +/** + * Function of arity 2 which may raise an IOException. + * @param type of arg1 + * @param type of arg2 + * @param type of return value. + */ +@FunctionalInterface +public interface BiFunctionRaisingIOE { + + /** + * Apply the function. + * @param t argument 1 + * @param u argument 2 + * @return result + * @throws IOException Any IO failure + */ + R apply(T t, U u) throws IOException; +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/CallableRaisingIOE.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/CallableRaisingIOE.java new file mode 100644 index 0000000000000..65b3a63b2b9a0 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/CallableRaisingIOE.java @@ -0,0 +1,36 @@ +/* + * 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.util.functional; + +import java.io.IOException; + +/** + * This is a callable which only raises an IOException. + * @param return type + */ +@FunctionalInterface +public interface CallableRaisingIOE { + + /** + * Apply the operation. + * @return result + * @throws IOException Any IO failure + */ + R apply() throws IOException; +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/ConsumerRaisingIOE.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/ConsumerRaisingIOE.java new file mode 100644 index 0000000000000..24a3b55c58d4a --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/ConsumerRaisingIOE.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.util.functional; + +import java.io.IOException; + +/** + * Version of java.util.function.Consumer which raises + * exceptions. + * @param type of argument,. + */ +@FunctionalInterface +public interface ConsumerRaisingIOE { + + /** + * Process the argument. + * @param t type + * @throws IOException if needed + */ + void accept(T t) throws IOException; + + /** + * after calling {@link #accept(Object)}, + * invoke the next consumer in the chain. + * @param next next consumer + * @return the chain. + */ + default ConsumerRaisingIOE andThen( + ConsumerRaisingIOE next) { + return (T t) -> { + accept(t); + next.accept(t); + }; + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/FunctionRaisingIOE.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/FunctionRaisingIOE.java new file mode 100644 index 0000000000000..83e041e2b3160 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/FunctionRaisingIOE.java @@ -0,0 +1,38 @@ +/* + * 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.util.functional; + +import java.io.IOException; + +/** + * Function of arity 1 which may raise an IOException. + * @param type of arg1 + * @param type of return value. + */ +@FunctionalInterface +public interface FunctionRaisingIOE { + + /** + * Apply the function. + * @param t argument 1 + * @return result + * @throws IOException Any IO failure + */ + R apply(T t) throws IOException; +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/FutureIO.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/FutureIO.java new file mode 100644 index 0000000000000..3f7218baa759f --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/FutureIO.java @@ -0,0 +1,188 @@ +/* + * 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.util.functional; + +import java.io.IOException; +import java.io.InterruptedIOException; +import java.io.UncheckedIOException; +import java.util.concurrent.CompletionException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Future IO Helper methods. + *

+ * Contains methods promoted from + * {@link org.apache.hadoop.fs.impl.FutureIOSupport} because they + * are a key part of integrating async IO in application code. + *

+ *

+ * One key feature is that the {@link #awaitFuture(Future)} and + * {@link #awaitFuture(Future, long, TimeUnit)} calls will + * extract and rethrow exceptions raised in the future's execution, + * including extracting the inner IOException of any + * {@code UncheckedIOException} raised in the future. + * This makes it somewhat easier to execute IOException-raising + * code inside futures. + *

+ */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public final class FutureIO { + + private FutureIO() { + } + + /** + * Given a future, evaluate it. + *

+ * Any exception generated in the future is + * extracted and rethrown. + *

+ * @param future future to evaluate + * @param type of the result. + * @return the result, if all went well. + * @throws InterruptedIOException future was interrupted + * @throws IOException if something went wrong + * @throws RuntimeException any nested RTE thrown + */ + public static T awaitFuture(final Future future) + throws InterruptedIOException, IOException, RuntimeException { + try { + return future.get(); + } catch (InterruptedException e) { + throw (InterruptedIOException) new InterruptedIOException(e.toString()) + .initCause(e); + } catch (ExecutionException e) { + return raiseInnerCause(e); + } + } + + /** + * Given a future, evaluate it. + *

+ * Any exception generated in the future is + * extracted and rethrown. + *

+ * @param future future to evaluate + * @param type of the result. + * @return the result, if all went well. + * @throws InterruptedIOException future was interrupted + * @throws IOException if something went wrong + * @throws RuntimeException any nested RTE thrown + * @throws TimeoutException the future timed out. + */ + public static T awaitFuture(final Future future, + final long timeout, + final TimeUnit unit) + throws InterruptedIOException, IOException, RuntimeException, + TimeoutException { + try { + return future.get(timeout, unit); + } catch (InterruptedException e) { + throw (InterruptedIOException) new InterruptedIOException(e.toString()) + .initCause(e); + } catch (ExecutionException e) { + return raiseInnerCause(e); + } + } + + /** + * From the inner cause of an execution exception, extract the inner cause + * if it is an IOE or RTE. + * This will always raise an exception, either the inner IOException, + * an inner RuntimeException, or a new IOException wrapping the raised + * exception. + * + * @param e exception. + * @param type of return value. + * @return nothing, ever. + * @throws IOException either the inner IOException, or a wrapper around + * any non-Runtime-Exception + * @throws RuntimeException if that is the inner cause. + */ + public static T raiseInnerCause(final ExecutionException e) + throws IOException { + throw unwrapInnerException(e); + } + + /** + * Extract the cause of a completion failure and rethrow it if an IOE + * or RTE. + * @param e exception. + * @param type of return value. + * @return nothing, ever. + * @throws IOException either the inner IOException, or a wrapper around + * any non-Runtime-Exception + * @throws RuntimeException if that is the inner cause. + */ + public static T raiseInnerCause(final CompletionException e) + throws IOException { + throw unwrapInnerException(e); + } + + /** + * From the inner cause of an execution exception, extract the inner cause + * to an IOException, raising RuntimeExceptions and Errors immediately. + *
    + *
  1. If it is an IOE: Return.
  2. + *
  3. If it is a {@link UncheckedIOException}: return the cause
  4. + *
  5. Completion/Execution Exceptions: extract and repeat
  6. + *
  7. If it is an RTE or Error: throw.
  8. + *
  9. Any other type: wrap in an IOE
  10. + *
+ * + * Recursively handles wrapped Execution and Completion Exceptions in + * case something very complicated has happened. + * @param e exception. + * @return an IOException extracted or built from the cause. + * @throws RuntimeException if that is the inner cause. + * @throws Error if that is the inner cause. + */ + @SuppressWarnings("ChainOfInstanceofChecks") + public static IOException unwrapInnerException(final Throwable e) { + Throwable cause = e.getCause(); + if (cause instanceof IOException) { + return (IOException) cause; + } else if (cause instanceof UncheckedIOException) { + // this is always an IOException + return ((UncheckedIOException) cause).getCause(); + } else if (cause instanceof CompletionException) { + return unwrapInnerException(cause); + } else if (cause instanceof ExecutionException) { + return unwrapInnerException(cause); + } else if (cause instanceof RuntimeException) { + throw (RuntimeException) cause; + } else if (cause instanceof Error) { + throw (Error) cause; + } else if (cause != null) { + // other type: wrap with a new IOE + return new IOException(cause); + } else { + // this only happens if there was no cause. + return new IOException(e); + } + } + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/InvocationRaisingIOE.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/InvocationRaisingIOE.java new file mode 100644 index 0000000000000..b59dabea89ea9 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/InvocationRaisingIOE.java @@ -0,0 +1,42 @@ +/* + * 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.util.functional; + +import java.io.IOException; + +/** + * This is a lambda-expression which may raises an IOException. + * This is a recurrent design patten in the hadoop codebase, e.g + * {@code LambdaTestUtils.VoidCallable} and + * the S3A {@code Invoker.VoidOperation}}. Hopefully this should + * be the last. + * Note for implementors of methods which take this as an argument: + * don't use method overloading to determine which specific functional + * interface is to be used. + */ +@FunctionalInterface +public interface InvocationRaisingIOE { + + /** + * Apply the operation. + * @throws IOException Any IO failure + */ + void apply() throws IOException; + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RemoteIterators.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RemoteIterators.java new file mode 100644 index 0000000000000..3ac0fced1493d --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RemoteIterators.java @@ -0,0 +1,698 @@ +/* + * 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.util.functional; + +import javax.annotation.Nullable; +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.Objects; + +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.RemoteIterator; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.io.IOUtils; + +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.logIOStatisticsAtDebug; +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics; + +/** + * A set of remote iterators supporting transformation and filtering, + * with IOStatisticsSource passthrough, and of conversions of + * the iterators to lists/arrays and of performing actions + * on the values. + *

+ * This aims to make it straightforward to use lambda-expressions to + * transform the results of an iterator, without losing the statistics + * in the process, and to chain the operations together. + *

+ * The closeable operation will be passed through RemoteIterators which + * wrap other RemoteIterators. This is to support any iterator which + * can be closed to release held connections, file handles etc. + * Unless client code is written to assume that RemoteIterator instances + * may be closed, this is not likely to be broadly used. It is added + * to make it possible to adopt this feature in a managed way. + *

+ * One notable feature is that the + * {@link #foreach(RemoteIterator, ConsumerRaisingIOE)} method will + * LOG at debug any IOStatistics provided by the iterator, if such + * statistics are provided. There's no attempt at retrieval and logging + * if the LOG is not set to debug, so it is a zero cost feature unless + * the logger {@code org.apache.hadoop.fs.functional.RemoteIterators} + * is at DEBUG. + *

+ * Based on the S3A Listing code, and some some work on moving other code + * to using iterative listings so as to pick up the statistics. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public final class RemoteIterators { + + /** + * Log used for logging any statistics in + * {@link #foreach(RemoteIterator, ConsumerRaisingIOE)} + * at DEBUG. + */ + private static final Logger LOG = LoggerFactory.getLogger( + RemoteIterators.class); + + private RemoteIterators() { + } + + /** + * Create an iterator from a singleton. + * @param singleton instance + * @param type + * @return a remote iterator + */ + public static RemoteIterator remoteIteratorFromSingleton( + @Nullable T singleton) { + return new SingletonIterator<>(singleton); + } + + /** + * Create a remote iterator from a java.util.Iterator. + * @param type + * @return a remote iterator + */ + public static RemoteIterator remoteIteratorFromIterator( + Iterator iterator) { + return new WrappedJavaIterator<>(iterator); + } + + /** + * Create a remote iterator from a java.util.Iterable -e.g. a list + * or other collection. + * @param type + * @return a remote iterator + */ + public static RemoteIterator remoteIteratorFromIterable( + Iterable iterable) { + return new WrappedJavaIterator<>(iterable.iterator()); + } + + /** + * Create a remote iterator from an array. + * @param type + * @return a remote iterator + */ + public static RemoteIterator remoteIteratorFromArray(T[] array) { + return new WrappedJavaIterator<>(Arrays.stream(array).iterator()); + } + + /** + * Create an iterator from an iterator and a transformation function. + * @param source type + * @param result type + * @param iterator source + * @param mapper transformation + * @return a remote iterator + */ + public static RemoteIterator mappingRemoteIterator( + RemoteIterator iterator, + FunctionRaisingIOE mapper) { + return new MappingRemoteIterator<>(iterator, mapper); + } + + /** + * Create a RemoteIterator from a RemoteIterator, casting the + * type in the process. This is to help with filesystem API + * calls where overloading causes confusion (e.g. listStatusIterator()) + * @param source type + * @param result type + * @param iterator source + * @return a remote iterator + */ + public static RemoteIterator typeCastingRemoteIterator( + RemoteIterator iterator) { + return new TypeCastingRemoteIterator<>(iterator); + } + + /** + * Create a RemoteIterator from a RemoteIterator and a filter + * function which returns true for every element to be passed + * through. + *

+ * Elements are filtered in the hasNext() method; if not used + * the filtering will be done on demand in the {@code next()} + * call. + * @param type + * @param iterator source + * @param filter filter + * @return a remote iterator + */ + public static RemoteIterator filteringRemoteIterator( + RemoteIterator iterator, + FunctionRaisingIOE filter) { + return new FilteringRemoteIterator<>(iterator, filter); + } + + /** + * This adds an extra close operation alongside the passthrough + * to any Closeable.close() method supported by the source iterator. + * @param iterator source + * @param toClose extra object to close. + * @param source type. + * @return a new iterator + */ + public static RemoteIterator closingRemoteIterator( + RemoteIterator iterator, + Closeable toClose) { + return new CloseRemoteIterator<>(iterator, toClose); + } + + /** + * Build a list from a RemoteIterator. + * @param type + * @return a list of the values. + * @throws IOException if the source RemoteIterator raises it. + */ + public static List toList(RemoteIterator source) + throws IOException { + List l = new ArrayList<>(); + foreach(source, l::add); + return l; + } + + /** + * Build an array from a RemoteIterator. + * @param type + * @return an array of the values. + * @throws IOException if the source RemoteIterator raises it. + */ + public static T[] toArray(RemoteIterator source) throws IOException { + return (T[]) toList(source).toArray(); + } + + /** + * Apply an operation to all values of a RemoteIterator. + *

+ * If the iterator is an IOStatisticsSource returning a non-null + * set of statistics, and this classes log is set to DEBUG, + * then the statistics of the operation are evaluated and logged at + * debug. + *

+ * The number of entries processed is returned, as it is useful to + * know this, especially during tests or when reporting values + * to users. + *

+ * This does not close the iterator afterwards. + * @param source iterator source + * @param consumer consumer of the values. + * @return the number of elements processed + * @param type of source + * @throws IOException if the source RemoteIterator or the consumer raise one. + */ + public static long foreach( + RemoteIterator source, + ConsumerRaisingIOE consumer) throws IOException { + long count = 0; + + try { + while (source.hasNext()) { + count++; + consumer.accept(source.next()); + } + + // maybe log the results + logIOStatisticsAtDebug(LOG, "RemoteIterator Statistics: {}", source); + } finally { + if (source instanceof Closeable) { + // source is closeable, so close. + IOUtils.cleanupWithLogger(LOG, (Closeable) source); + } + } + + return count; + } + + /** + * A remote iterator from a singleton. It has a single next() + * value, after which hasNext() returns false and next() fails. + *

+ * If it is a source of + * remote statistics, these are returned. + * @param type. + */ + private static final class SingletonIterator + implements RemoteIterator, IOStatisticsSource { + + /** + * Single entry. + */ + private final T singleton; + + /** Has the entry been processed? */ + private boolean processed; + + /** + * Instantiate. + * @param singleton single value...may be null + */ + private SingletonIterator(@Nullable T singleton) { + this.singleton = singleton; + // if the entry is null, consider it processed. + this.processed = singleton == null; + } + + @Override + public boolean hasNext() throws IOException { + return !processed; + } + + @SuppressWarnings("NewExceptionWithoutArguments") + @Override + public T next() throws IOException { + if (hasNext()) { + processed = true; + return singleton; + } else { + throw new NoSuchElementException(); + } + } + + @Override + public IOStatistics getIOStatistics() { + return retrieveIOStatistics(singleton); + } + + @Override + public String toString() { + return "SingletonIterator{" + + (singleton != null ? singleton : "") + + '}'; + } + + } + + /** + * Create a remote iterator from a simple java.util.Iterator, or + * an iterable. + *

+ * If the iterator is a source of statistics that is passed through. + *

+ * The {@link #close()} will close the source iterator if it is + * Closeable; + * @param iterator type. + */ + private static final class WrappedJavaIterator + implements RemoteIterator, IOStatisticsSource, Closeable { + + /** + * inner iterator.. + */ + private final Iterator source; + + private final Closeable sourceToClose; + + + /** + * Construct from an interator. + * @param source source iterator. + */ + private WrappedJavaIterator(Iterator source) { + this.source = requireNonNull(source); + sourceToClose = new MaybeClose(source); + } + + @Override + public boolean hasNext() { + return source.hasNext(); + } + + @Override + public T next() { + return source.next(); + } + + @Override + public IOStatistics getIOStatistics() { + return retrieveIOStatistics(source); + } + + @Override + public String toString() { + return "FromIterator{" + source + '}'; + } + + @Override + public void close() throws IOException { + sourceToClose.close(); + + } + } + + /** + * Wrapper of another remote iterator; IOStatistics + * and Closeable methods are passed down if implemented. + * @param source type + * @param type of returned value + */ + private static abstract class WrappingRemoteIterator + implements RemoteIterator, IOStatisticsSource, Closeable { + + /** + * Source iterator. + */ + private final RemoteIterator source; + + private final Closeable sourceToClose; + + protected WrappingRemoteIterator(final RemoteIterator source) { + this.source = requireNonNull(source); + sourceToClose = new MaybeClose(source); + } + + protected RemoteIterator getSource() { + return source; + } + + @Override + public IOStatistics getIOStatistics() { + return retrieveIOStatistics(source); + } + + @Override + public void close() throws IOException { + sourceToClose.close(); + } + + /** + * Check for the source having a next element. + * If it does not, this object's close() method + * is called and false returned + * @return true if there is a new value + * @throws IOException failure to retrieve next value + */ + protected boolean sourceHasNext() throws IOException { + boolean hasNext; + try { + hasNext = getSource().hasNext(); + } catch (IOException e) { + IOUtils.cleanupWithLogger(LOG, this); + throw e; + } + if (!hasNext) { + // there is nothing less so automatically close. + close(); + } + return hasNext; + } + + /** + * Get the next source value. + * This calls {@link #sourceHasNext()} first to verify + * that there is data. + * @return the next value + * @throws IOException failure + * @throws NoSuchElementException no more data + */ + protected S sourceNext() throws IOException { + try { + if (!sourceHasNext()) { + throw new NoSuchElementException(); + } + return getSource().next(); + } catch (NoSuchElementException | IOException e) { + IOUtils.cleanupWithLogger(LOG, this); + throw e; + } + } + + @Override + public String toString() { + return source.toString(); + } + + } + + /** + * Iterator taking a source and a transformational function. + * @param source type + * @param final output type.There + */ + private static final class MappingRemoteIterator + extends WrappingRemoteIterator { + + /** + * Mapper to invoke. + */ + private final FunctionRaisingIOE mapper; + + private MappingRemoteIterator( + RemoteIterator source, + FunctionRaisingIOE mapper) { + super(source); + this.mapper = requireNonNull(mapper); + } + + @Override + public boolean hasNext() throws IOException { + return sourceHasNext(); + } + + @Override + public T next() throws IOException { + return mapper.apply(sourceNext()); + } + + @Override + public String toString() { + return "FunctionRemoteIterator{" + getSource() + '}'; + } + } + + /** + * RemoteIterator which can change the type of the input. + * This is useful in some situations. + * @param source type + * @param final output type. + */ + private static final class TypeCastingRemoteIterator + extends WrappingRemoteIterator { + + private TypeCastingRemoteIterator( + RemoteIterator source) { + super(source); + } + + @Override + public boolean hasNext() throws IOException { + return sourceHasNext(); + } + + @Override + public T next() throws IOException { + return (T)sourceNext(); + } + + @Override + public String toString() { + return getSource().toString(); + } + } + + /** + * Extend the wrapped iterator by filtering source values out. + * Only those values for which the filter predicate returns true + * will be returned. + * @param type of iterator. + */ + @SuppressWarnings("NewExceptionWithoutArguments") + private static final class FilteringRemoteIterator + extends WrappingRemoteIterator { + + /** + * Filter Predicate. + * Takes the input type or any superclass. + */ + private final FunctionRaisingIOE + filter; + + /** + * Next value; will be null if none has been evaluated, or the + * last one was already returned by next(). + */ + private S next; + + /** + * An iterator which combines filtering with transformation. + * All source elements for which filter = true are returned, + * transformed via the mapper. + * @param source source iterator. + * @param filter filter predicate. + */ + private FilteringRemoteIterator( + RemoteIterator source, + FunctionRaisingIOE filter) { + super(source); + + this.filter = requireNonNull(filter); + } + + /** + * Fetch: retrieve the next value. + * @return true if a new value was found after filtering. + * @throws IOException failure in retrieval from source or mapping + */ + private boolean fetch() throws IOException { + while (next == null && sourceHasNext()) { + S candidate = getSource().next(); + if (filter.apply(candidate)) { + next = candidate; + return true; + } + } + return false; + } + + /** + * Trigger a fetch if an entry is needed. + * @return true if there was already an entry return, + * or there was not but one could then be retrieved.set + * @throws IOException failure in fetch operation + */ + @Override + public boolean hasNext() throws IOException { + if (next != null) { + return true; + } + return fetch(); + } + + /** + * Return the next value. + * Will retrieve the next elements if needed. + * This is where the mapper takes place. + * @return true if there is another data element. + * @throws IOException failure in fetch operation or the transformation. + * @throws NoSuchElementException no more data + */ + @Override + public S next() throws IOException { + if (hasNext()) { + S result = next; + next = null; + return result; + } + throw new NoSuchElementException(); + } + + @Override + public String toString() { + return "FilteringRemoteIterator{" + getSource() + '}'; + } + } + + /** + * A wrapping remote iterator which adds another entry to + * close. This is to assist cleanup. + * @param type + */ + private static final class CloseRemoteIterator + extends WrappingRemoteIterator { + + private final MaybeClose toClose; + private boolean closed; + + private CloseRemoteIterator( + final RemoteIterator source, + final Closeable toClose) { + super(source); + this.toClose = new MaybeClose(Objects.requireNonNull(toClose)); + } + + @Override + public boolean hasNext() throws IOException { + return sourceHasNext(); + } + + @Override + public S next() throws IOException { + + return sourceNext(); + } + + @Override + public void close() throws IOException { + if (closed) { + return; + } + closed = true; + LOG.debug("Closing {}", this); + try { + super.close(); + } finally { + toClose.close(); + } + } + } + + /** + * Class to help with Closeable logic, where sources may/may not + * be closeable, only one invocation is allowed. + * On the second and later call of close(), it is a no-op. + */ + private static final class MaybeClose implements Closeable { + + private Closeable toClose; + + /** + * Construct. + * @param o object to close. + */ + private MaybeClose(Object o) { + this(o, true); + } + + /** + * Construct -close the object if it is closeable and close==true. + * @param o object to close. + * @param close should close? + */ + private MaybeClose(Object o, boolean close) { + if (close && o instanceof Closeable) { + this.toClose = (Closeable) o; + } else { + this.toClose = null; + } + } + + @Override + public void close() throws IOException { + if (toClose != null) { + try { + toClose.close(); + } finally { + toClose = null; + } + } + } + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/package-info.java new file mode 100644 index 0000000000000..1c204bb9979a8 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/package-info.java @@ -0,0 +1,41 @@ +/* + * 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. + */ + +/** + * Support for functional programming within the Hadoop APIs. + *

+ * Much of this is needed simply to cope with Java's checked exceptions and + * the fact that the java.util.function can only throw runtime exceptions. + *

+ * Pretty much all the Hadoop FS APIs raise IOExceptions, hence the need + * for these classes. If Java had made a different decision about the + * nature of exceptions, life would be better. + *

+ * Do note that the {@link org.apache.hadoop.util.functional.RemoteIterators} + * iterators go beyond that of the java ones, in terms of declaring themselves + * Closeable and implementors of + * {@link org.apache.hadoop.fs.statistics.IOStatisticsSource}; a chain + * of wrapped iterators can supply statistics of the inner iterators, and + * encourage close() to be called after use. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +package org.apache.hadoop.util.functional; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file 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..bd77dc7e0f8a7 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/iostatistics.md @@ -0,0 +1,432 @@ + + +# Statistic collection with 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 Apache Hive or Apache HBase. + +The 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 class *may* implement `IOStatisticsSource` in order to +provide statistics. + +Wrapper I/O Classes such as `FSDataInputStream` anc `FSDataOutputStream` *should* +implement the interface and forward it to the wrapped class, if they also +implement it -and return `null` if they do not. + +`IOStatisticsSource` implementations `getIOStatistics()` return an +instance of `IOStatistics` enumerating the statistics of that specific +instance. + +The `IOStatistics` Interface exports five kinds of statistic: + + +| Category | Type | Description | +|------|------|-------------| +| `counter` | `long` | a counter which may increase in value; SHOULD BE >= 0 | +| `gauge` | `long` | an arbitrary value which can down as well as up; SHOULD BE >= 0 | +| `minimum` | `long` | an minimum value; MAY BE negative | +| `maximum` | `long` | a maximum value; MAY BE negative | +| `meanStatistic` | `MeanStatistic` | an arithmetic mean and sample size; mean MAY BE negative | + +Four are simple `long` values, with the variations how they are likely to +change and how they are aggregated. + + +#### Aggregation of Statistic Values + +For the different statistic category, the result of `aggregate(x, y)` is + +| Category | Aggregation | +|------------------|-------------| +| `counter` | `max(0, x) + max(0, y)` | +| `gauge` | `max(0, x) + max(0, y)` | +| `minimum` | `min(x, y)` | +| `maximum` | `max(x, y)` | +| `meanStatistic` | calculation of the mean of `x` and `y` ) | + + +#### Class `MeanStatistic` + +## package `org.apache.hadoop.fs.statistics` + +This package contains the public statistics APIs intended +for use by applications. + + + + + +`MeanStatistic` is a tuple of `(mean, samples)` to support aggregation. + +A `MeanStatistic` with a sample of `0` is considered an empty statistic. + +All `MeanStatistic` instances where `sample = 0` are considered equal, +irrespective of the `mean` value. + +Algorithm to calculate the mean : + +```python +if x.samples = 0: + y +else if y.samples = 0 : + x +else: + samples' = x.samples + y.samples + mean' = (x.mean * x.samples) + (y.mean * y.samples) / samples' + (samples', mean') +``` + +Implicitly, this means that if both samples are empty, then the aggregate value is also empty. + +```java +public final class MeanStatistic implements Serializable, Cloneable { + /** + * Arithmetic mean. + */ + private double mean; + + /** + * Number of samples used to calculate + * the mean. + */ + private long samples; + + /** + * Get the mean value. + * @return the mean + */ + public double getMean() { + return mean; + } + + /** + * Get the sample count. + * @return the sample count; 0 means empty + */ + public long getSamples() { + return samples; + } + + /** + * Is a statistic empty? + * @return true if the sample count is 0 + */ + public boolean isEmpty() { + return samples == 0; + } + /** + * Add another mean statistic to create a new statistic. + * When adding two statistics, if either is empty then + * a copy of the non-empty statistic is returned. + * If both are empty then a new empty statistic is returned. + * + * @param other other value + * @return the aggregate mean + */ + public MeanStatistic add(final MeanStatistic other) { + /* Implementation elided. */ + } + @Override + public int hashCode() { + return Objects.hash(mean, samples); + } + + @Override + public boolean equals(final Object o) { + if (this == o) { return true; } + if (o == null || getClass() != o.getClass()) { return false; } + MeanStatistic that = (MeanStatistic) o; + if (this.isEmpty()) { + return that.isEmpty(); + } + return Double.compare(that.mean, mean) == 0 && + samples == that.samples; + } + + @Override + public MeanStatistic clone() { + return new MeanStatistic(this); + } + + public MeanStatistic copy() { + return new MeanStatistic(this); + } + +} +``` + + + + + +### 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. + +#### Invariants + +The result of `getIOStatistics()` must be one of + +* `null` +* an immutable `IOStatistics` for which each map of entries is +an empty map. +* an instance of an `IOStatistics` whose statistics MUST BE unique to that +instance of the class implementing `IOStatisticsSource`. + +Less formally: if the statistics maps returned are non-empty, all the statistics +must be collected from the current instance, and not from any other instances, the way +some of the `FileSystem` statistics are collected. + + +The result of `getIOStatistics()`, if non-null, MAY be a different instance +on every invocation. + + + + + + +### 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 { + + /** + * Map of counters. + * @return the current map of counters. + */ + Map counters(); + + /** + * Map of gauges. + * @return the current map of gauges. + */ + Map gauges(); + + /** + * Map of minumums. + * @return the current map of minumums. + */ + Map minumums(); + + /** + * Map of maximums. + * @return the current map of maximums. + */ + Map maximums(); + + /** + * Map of meanStatistics. + * @return the current map of MeanStatistic statistics. + */ + Map meanStatistics(); + +} +``` + +### Statistic Naming + +The naming policy of statistics is designed to be readable, shareable +and ideally consistent across `IOStatisticSource` implementations. + +* Characters in key names MUST match the regular expression + `[a-z|0-9|_]` with the exception of the first character, which + MUST be in the range `[a-z]`. Thus the full regular expression + for a valid statistic name is: + + [a-z][a-z|0-9|_]+ + +* Where possible, the names of statistics SHOULD be those defined + with common names. + + org.apache.hadoop.fs.statistics.StreamStatisticNames + org.apache.hadoop.fs.statistics.StoreStatisticNames + + Note 1.: these are evolving; for clients to safely reference their + statistics by name they SHOULD be copied to the application. + (i.e. for an application compiled hadoop 3.4.2 to link against hadoop 3.4.1, + copy the strings). + + Note 2: keys defined in these classes SHALL NOT be removed + from subsequent Hadoop releases. + +* A common statistic name MUST NOT be used to report any other statistic and + MUST use the pre-defined unit of measurement. + +* A statistic name in one of the maps SHOULD NOT be re-used in another map. + This aids diagnostics of logged statistics. + +### Statistic Maps + +For each map of statistics returned: + +* The operations to add/remove entries are unsupported: the map returned + MAY be mutable by the source of statistics. + +* The map MAY be empty. + +* The map keys each represent a measured statistic. + +* The set of keys in a map SHOULD remain unchanged, and MUST NOT remove keys. + +* The statistics SHOULD be dynamic: every lookup of an entry SHOULD + return the latest value. + +* The values MAY change across invocations of `Map.values()` and `Map.entries()` + +* The update MAY be in the `iterable()` calls of the iterators returned, + or MAY be in the actual `iterable.next()` operation. That is: there is + no guarantee as to when the evaluation takes place. + +* The returned `Map.Entry` instances MUST return the same value on + repeated `getValue()` calls. (i.e once you have the entry, it is immutable). + +* Queries of statistics SHOULD be fast and non-blocking to the extent + that if invoked during a long operation, they will prioritize + returning fast over most timely values. + +* The statistics MAY lag; especially for statistics collected in separate + operations (e.g stream IO statistics as provided by a filesystem + instance). + +* Statistics which represent time SHOULD use milliseconds as their unit. + +* Statistics which represent time and use a different unit MUST document + the unit used. + +### Thread Model + +1. An instance of `IOStatistics` can be shared across threads; + +1. Read access to the supplied statistics maps MUST be thread safe. + +1. Iterators returned from the maps MUST NOT be shared across threads. + +1. The statistics collected MUST include all operations which took + place across all threads performing work for the monitored object. + +1. The statistics reported MUST NOT be local to the active thread. + +This is different from the `FileSystem.Statistics` behavior where per-thread statistics +are collected and reported. + +That mechanism supports collecting limited read/write statistics for different +worker threads sharing the same FS instance, but as the collection is thread local, +it invariably under-reports IO performed in other threads on behalf of a worker thread. + + +## Statisic Snapshot + +A snapshot of the current statistic values MAY be obtained by calling +`IOStatisticsSupport.snapshotIOStatistics()` + +```java + public static X + snapshotIOStatistics(IOStatistics statistics) +``` + +This snapshot is serializable through Java serialization and through +Jackson to/from JSON. + +## Helper Classes + + +### class `org.apache.hadoop.fs.statistics.IOStatisticsSupport` + +This provides helper methods to work with IOStatistics sources and instances. + +Consult the javadocs for its operations. + +### class `org.apache.hadoop.fs.statistics.IOStatisticsLogging` + +Support for efficiently logging `IOStatistics`/`IOStatisticsSource` +instances. + +These are intended for assisting logging, including only enumerating the +state of an `IOStatistics` instance when the log level needs it. + +```java +LOG.info("IOStatistics after upload: {}", demandStringify(iostats)); + +// or even better, as it results in only a single object creations +Object latest = demandStringify(iostats); +LOG.info("IOStatistics : {}", latest); +/* do some work. */ +LOG.info("IOStatistics : {}", latest); + +``` + +## Package `org.apache.hadoop.fs.statistics.impl` + +This contains implementation classes to support providing statistics to applications. + +These MUST NOT BE used by applications. If a feature is needed from this package then +the provisioning of a public implementation MAY BE raised via the Hadoop development +channels. + +These MAY be used by those implementations of the Hadoop `FileSystem`, `AbstractFileSystem` +and related classes which are not in the hadoop source tree. Implementors MUST BE +aware that the implementation this code is unstable and may change across +minor point releases of Hadoop. diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMultipartUploaderTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMultipartUploaderTest.java index d0b067ad6e6ae..90e12a84487a0 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMultipartUploaderTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMultipartUploaderTest.java @@ -51,6 +51,7 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyPathExists; import static org.apache.hadoop.fs.impl.FutureIOSupport.awaitFuture; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsSourceToString; import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; import static org.apache.hadoop.test.LambdaTestUtils.eventually; import static org.apache.hadoop.test.LambdaTestUtils.intercept; @@ -106,6 +107,8 @@ public void teardown() throws Exception { CompletableFuture f = uploader.abortUploadsUnderPath(teardown); f.get(); + LOG.info("Statistics {}", + ioStatisticsSourceToString(uploader)); } catch (Exception e) { LOG.warn("Exeception in teardown", e); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractStreamIOStatisticsTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractStreamIOStatisticsTest.java new file mode 100644 index 0000000000000..89b21c497083b --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractStreamIOStatisticsTest.java @@ -0,0 +1,313 @@ +/* + * 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.contract; + +import java.util.Collections; +import java.util.List; + +import org.assertj.core.api.Assertions; +import org.junit.AfterClass; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.extractStatistics; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.demandStringifyIOStatisticsSource; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString; +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.snapshotIOStatistics; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BYTES; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_WRITE_BYTES; + +/** + * Tests {@link IOStatistics} support in input and output streams. + *

+ * Requires both the input and output streams to offer the basic + * bytes read/written statistics. + *

+ * If the IO is buffered, that information must be provided, + * especially the input buffer size. + */ +public abstract class AbstractContractStreamIOStatisticsTest + extends AbstractFSContractTestBase { + + private static final Logger LOG = + LoggerFactory.getLogger(AbstractContractStreamIOStatisticsTest.class); + + /** + * FileSystem statistics are collected across every test case. + */ + protected static final IOStatisticsSnapshot FILESYSTEM_IOSTATS = + snapshotIOStatistics(); + + @Override + public void teardown() throws Exception { + final FileSystem fs = getFileSystem(); + if (fs instanceof IOStatisticsSource) { + FILESYSTEM_IOSTATS.aggregate(((IOStatisticsSource)fs).getIOStatistics()); + } + super.teardown(); + } + + /** + * Dump the filesystem statistics after the class if contains any values. + */ + @AfterClass + public static void dumpFileSystemIOStatistics() { + if (!FILESYSTEM_IOSTATS.counters().isEmpty()) { + // if there is at least one counter + LOG.info("Aggregate FileSystem Statistics {}", + ioStatisticsToPrettyString(FILESYSTEM_IOSTATS)); + } + } + + @Test + public void testOutputStreamStatisticKeys() throws Throwable { + describe("Look at the statistic keys of an output stream"); + Path path = methodPath(); + FileSystem fs = getFileSystem(); + fs.mkdirs(path.getParent()); + try (FSDataOutputStream out = fs.create(path, true)) { + IOStatistics statistics = extractStatistics(out); + final List keys = outputStreamStatisticKeys(); + Assertions.assertThat(statistics.counters().keySet()) + .describedAs("statistic keys of %s", statistics) + .containsAll(keys); + Assertions.assertThat(keys) + .describedAs("Statistics supported by the stream %s", out) + .contains(STREAM_WRITE_BYTES); + } finally { + fs.delete(path, false); + } + } + + /** + * If the stream writes in blocks, then counters during the write may be + * zero until a whole block is written -or the write has finished. + * @return true if writes are buffered into whole blocks. + */ + public boolean streamWritesInBlocks() { + return false; + } + + @Test + public void testWriteSingleByte() throws Throwable { + describe("Write a byte to a file and verify" + + " the stream statistics are updated"); + Path path = methodPath(); + FileSystem fs = getFileSystem(); + fs.mkdirs(path.getParent()); + boolean writesInBlocks = streamWritesInBlocks(); + try (FSDataOutputStream out = fs.create(path, true)) { + IOStatistics statistics = extractStatistics(out); + // before a write, no bytes + verifyStatisticCounterValue(statistics, STREAM_WRITE_BYTES, 0); + out.write('0'); + verifyStatisticCounterValue(statistics, STREAM_WRITE_BYTES, + writesInBlocks ? 0 : 1); + // close the stream + out.close(); + // statistics are still valid after the close + // always call the output stream to check that behavior + statistics = extractStatistics(out); + final String strVal = statistics.toString(); + LOG.info("Statistics = {}", strVal); + verifyStatisticCounterValue(statistics, STREAM_WRITE_BYTES, 1); + } finally { + fs.delete(path, false); + } + } + + @Test + public void testWriteByteArrays() throws Throwable { + describe("Write byte arrays to a file and verify" + + " the stream statistics are updated"); + Path path = methodPath(); + FileSystem fs = getFileSystem(); + fs.mkdirs(path.getParent()); + boolean writesInBlocks = streamWritesInBlocks(); + try (FSDataOutputStream out = fs.create(path, true)) { + Object demandStatsString = demandStringifyIOStatisticsSource(out); + // before a write, no bytes + final byte[] bytes = ContractTestUtils.toAsciiByteArray( + "statistically-speaking"); + final long len = bytes.length; + out.write(bytes); + out.flush(); + LOG.info("stats {}", demandStatsString); + IOStatistics statistics = extractStatistics(out); + verifyStatisticCounterValue(statistics, STREAM_WRITE_BYTES, + writesInBlocks ? 0 : len); + out.write(bytes); + out.flush(); + verifyStatisticCounterValue(statistics, STREAM_WRITE_BYTES, + writesInBlocks ? 0 : len * 2); + // close the stream + out.close(); + LOG.info("stats {}", demandStatsString); + // statistics are still valid after the close + // always call the output stream to check that behavior + statistics = extractStatistics(out); + verifyStatisticCounterValue(statistics, STREAM_WRITE_BYTES, len * 2); + // the to string value must contain the same counterHiCable you mean + Assertions.assertThat(demandStatsString.toString()) + .contains(Long.toString(len * 2)); + } finally { + fs.delete(path, false); + } + } + + @Test + public void testInputStreamStatisticKeys() throws Throwable { + describe("Look at the statistic keys of an input stream"); + Path path = methodPath(); + FileSystem fs = getFileSystem(); + ContractTestUtils.touch(fs, path); + try (FSDataInputStream in = fs.open(path)) { + IOStatistics statistics = extractStatistics(in); + final List keys = inputStreamStatisticKeys(); + Assertions.assertThat(statistics.counters().keySet()) + .describedAs("statistic keys of %s", statistics) + .containsAll(keys); + Assertions.assertThat(keys) + .describedAs("Statistics supported by the stream %s", in) + .contains(STREAM_READ_BYTES); + verifyStatisticCounterValue(statistics, STREAM_READ_BYTES, 0); + } finally { + fs.delete(path, false); + } + } + + @Test + public void testInputStreamStatisticRead() throws Throwable { + describe("Read Data from an input stream"); + Path path = methodPath(); + FileSystem fs = getFileSystem(); + final int fileLen = 1024; + final byte[] ds = dataset(fileLen, 'a', 26); + ContractTestUtils.writeDataset(fs, path, ds, fileLen, 8_000, true); + + try (FSDataInputStream in = fs.open(path)) { + long current = 0; + IOStatistics statistics = extractStatistics(in); + verifyStatisticCounterValue(statistics, STREAM_READ_BYTES, 0); + Assertions.assertThat(in.read()).isEqualTo('a'); + int bufferSize = readBufferSize(); + // either a single byte was read or a whole block + current = verifyBytesRead(statistics, current, 1, bufferSize); + final int bufferLen = 128; + byte[] buf128 = new byte[bufferLen]; + in.read(buf128); + current = verifyBytesRead(statistics, current, bufferLen, bufferSize); + in.readFully(buf128); + current = verifyBytesRead(statistics, current, bufferLen, bufferSize); + in.readFully(0, buf128); + current = verifyBytesRead(statistics, current, bufferLen, bufferSize); + // seek must not increment the read counter + in.seek(256); + verifyBytesRead(statistics, current, 0, bufferSize); + + // if a stream implements lazy-seek the seek operation + // may be postponed until the read + final int sublen = 32; + Assertions.assertThat(in.read(buf128, 0, sublen)) + .isEqualTo(sublen); + current = verifyBytesRead(statistics, current, sublen, bufferSize); + + // perform some read operations near the end of the file such that + // the buffer will not be completely read. + // skip these tests for buffered IO as it is too complex to work out + if (bufferSize == 0) { + final int pos = fileLen - sublen; + in.seek(pos); + Assertions.assertThat(in.read(buf128)) + .describedAs("Read overlapping EOF") + .isEqualTo(sublen); + current = verifyStatisticCounterValue(statistics, STREAM_READ_BYTES, + current + sublen); + Assertions.assertThat(in.read(pos, buf128, 0, bufferLen)) + .describedAs("Read(buffer) overlapping EOF") + .isEqualTo(sublen); + verifyStatisticCounterValue(statistics, STREAM_READ_BYTES, + current + sublen); + } + } finally { + fs.delete(path, false); + } + } + + /** + * Verify the bytes read value, taking into account block size. + * @param statistics stats + * @param current current count + * @param bytesRead bytes explicitly read + * @param bufferSize buffer size of stream + * @return the current count of bytes read ignoring block size + */ + public long verifyBytesRead(final IOStatistics statistics, + final long current, + final int bytesRead, final int bufferSize) { + // final position. for unbuffered read, this is the expected value + long finalPos = current + bytesRead; + long expected = finalPos; + if (bufferSize > 0) { + // buffered. count of read is number of buffers already read + // plus the current buffer, multiplied by that buffer size + expected = bufferSize * (1 + (current / bufferSize)); + } + verifyStatisticCounterValue(statistics, STREAM_READ_BYTES, expected); + return finalPos; + } + + /** + * Buffer size for reads. + * Filesystems performing block reads (checksum, etc) + * must return their buffer value is + * @return buffer capacity; 0 for unbuffered + */ + public int readBufferSize() { + return 0; + } + + /** + * Keys which the output stream must support. + * @return a list of keys + */ + public List outputStreamStatisticKeys() { + return Collections.singletonList(STREAM_WRITE_BYTES); + } + + /** + * Keys which the input stream must support. + * @return a list of keys + */ + public List inputStreamStatisticKeys() { + return Collections.singletonList(STREAM_READ_BYTES); + } + +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractStreamIOStatistics.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractStreamIOStatistics.java new file mode 100644 index 0000000000000..642baec502d2e --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractStreamIOStatistics.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.contract.localfs; + +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 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; + +/** + * Test IOStatistics through the local FS. + */ +public class TestLocalFSContractStreamIOStatistics extends + AbstractContractStreamIOStatisticsTest { + + @Override + protected AbstractFSContract createContract(Configuration conf) { + return new LocalFSContract(conf); + } + + /** + * Keys which the input stream must support. + * @return a list of keys + */ + public List inputStreamStatisticKeys() { + return Arrays.asList(STREAM_READ_BYTES, + STREAM_READ_EXCEPTIONS, + STREAM_READ_SEEK_OPERATIONS, + STREAM_READ_SKIP_OPERATIONS, + STREAM_READ_SKIP_BYTES); + } + + /** + * Keys which the output stream must support. + * @return a list of keys + */ + @Override + public List outputStreamStatisticKeys() { + return Arrays.asList(STREAM_WRITE_BYTES, + STREAM_WRITE_EXCEPTIONS); + } + + @Override + public int readBufferSize() { + return 1024; + } + + @Override + public boolean streamWritesInBlocks() { + return true; + } + + +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/IOStatisticAssertions.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/IOStatisticAssertions.java new file mode 100644 index 0000000000000..22f6c33d2e260 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/IOStatisticAssertions.java @@ -0,0 +1,528 @@ +/* + * 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.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.ObjectStreamClass; +import java.io.Serializable; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import org.assertj.core.api.AbstractLongAssert; +import org.assertj.core.api.ObjectAssert; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Assertions and any other support for IOStatistics testing. + * If used downstream: know it is unstable. + */ + +@InterfaceAudience.Private +@InterfaceStability.Unstable +public final class IOStatisticAssertions { + + private static final String COUNTER = "Counter"; + + private static final String GAUGE = "Gauge"; + + private static final String MINIMUM = "Minimum"; + + private static final String MAXIMUM = "Maxiumum"; + + private static final String MEAN = "Mean"; + + private IOStatisticAssertions() { + } + + /** + * Get a required counter statistic. + * @param stats statistics source + * @param key statistic key + * @return the value + */ + public static long lookupCounterStatistic( + final IOStatistics stats, + final String key) { + return lookupStatistic(COUNTER, key, + verifyStatisticsNotNull(stats).counters()); + } + + /** + * Given an IOStatistics instance, verify it is not null, + * and return the value for continued use in a test. + * @param stats statistics source. + * @param type of statistics + * @return the value passed in. + */ + public static T + verifyStatisticsNotNull(final T stats) { + assertThat(stats) + .describedAs("IO Statistics reference") + .isNotNull(); + return stats; + } + + /** + * Get a required gauge statistic. + * @param stats statistics source + * @param key statistic key + * @return the value + */ + public static long lookupGaugeStatistic( + final IOStatistics stats, + final String key) { + return lookupStatistic(GAUGE, key, + verifyStatisticsNotNull(stats).gauges()); + } + + /** + * Get a required maximum statistic. + * @param stats statistics source + * @param key statistic key + * @return the value + */ + public static long lookupMaximumStatistic( + final IOStatistics stats, + final String key) { + return lookupStatistic(MAXIMUM, key, + verifyStatisticsNotNull(stats).maximums()); + } + + /** + * Get a required minimum statistic. + * @param stats statistics source + * @param key statistic key + * @return the value + */ + public static long lookupMinimumStatistic( + final IOStatistics stats, + final String key) { + return lookupStatistic(MINIMUM, key, + verifyStatisticsNotNull(stats).minimums()); + } + + /** + * Get a required mean statistic. + * @param stats statistics source + * @param key statistic key + * @return the value + */ + public static MeanStatistic lookupMeanStatistic( + final IOStatistics stats, + final String key) { + return lookupStatistic(MEAN, key, + verifyStatisticsNotNull(stats).meanStatistics()); + } + + /** + * Get a required counter statistic. + * @param type of map element + * @param type type for error text + * @param key statistic key + * @param map map to probe + * @return the value + */ + private static E lookupStatistic( + final String type, + final String key, + final Map map) { + final E statistic = map.get(key); + assertThat(statistic) + .describedAs("%s named %s", type, key) + .isNotNull(); + return statistic; + } + + /** + * Assert that a counter has an expected value. + * @param stats statistics source + * @param key statistic key + * @param value expected value. + * @return the value (which always equals the expected value) + */ + public static long verifyStatisticCounterValue( + final IOStatistics stats, + final String key, + final long value) { + return verifyStatisticValue(COUNTER, key, + verifyStatisticsNotNull(stats).counters(), value); + } + + /** + * Assert that a gauge has an expected value. + * @param stats statistics source + * @param key statistic key + * @param value expected value. + * @return the value (which always equals the expected value) + */ + public static long verifyStatisticGaugeValue( + final IOStatistics stats, + final String key, + final long value) { + return verifyStatisticValue(GAUGE, key, + verifyStatisticsNotNull(stats).gauges(), value); + } + + /** + * Assert that a maximum has an expected value. + * @param stats statistics source + * @param key statistic key + * @param value expected value. + * @return the value (which always equals the expected value) + */ + public static long verifyStatisticMaximumValue( + final IOStatistics stats, + final String key, + final long value) { + return verifyStatisticValue(MAXIMUM, key, + verifyStatisticsNotNull(stats).maximums(), value); + } + + /** + * Assert that a minimum has an expected value. + * @param stats statistics source + * @param key statistic key + * @param value expected value. + * @return the value (which always equals the expected value) + */ + public static long verifyStatisticMinimumValue( + final IOStatistics stats, + final String key, + final long value) { + return verifyStatisticValue(MINIMUM, key, + verifyStatisticsNotNull(stats).minimums(), value); + } + + /** + * Assert that a mean has an expected value. + * @param stats statistics source + * @param key statistic key + * @param value expected value. + * @return the value (which always equals the expected value) + */ + public static MeanStatistic verifyStatisticMeanValue( + final IOStatistics stats, + final String key, + final MeanStatistic value) { + return verifyStatisticValue(MEAN, key, + verifyStatisticsNotNull(stats).meanStatistics(), value); + } + + /** + * Assert that a given statistic has an expected value. + * @param type type for error text + * @param key statistic key + * @param map map to look up + * @param value expected value. + * @param type of map element + * @return the value (which always equals the expected value) + */ + private static E verifyStatisticValue( + final String type, + final String key, + final Map map, + final E value) { + final E statistic = lookupStatistic(type, key, map); + assertThat(statistic) + .describedAs("%s named %s with expected value %s", type, + key, value) + .isEqualTo(value); + return statistic; + } + + + /** + * Assert that a given statistic has an expected value. + * @param type of map element + * @param type type for error text + * @param key statistic key + * @param map map to look up + * @return an ongoing assertion + */ + private static ObjectAssert assertThatStatistic( + final String type, + final String key, + final Map map) { + final E statistic = lookupStatistic(type, key, map); + return assertThat(statistic) + .describedAs("%s named %s", type, key); + } + + /** + * Assert that a given statistic has an expected value. + * @param type of map element + * @param type type for error text + * @param key statistic key + * @param map map to look up + * @return an ongoing assertion + */ + private static AbstractLongAssert assertThatStatisticLong( + final String type, + final String key, + final Map map) { + final long statistic = lookupStatistic(type, key, map); + return assertThat(statistic) + .describedAs("%s named %s", type, key); + } + + /** + * Start an assertion chain on + * a required counter statistic. + * @param stats statistics source + * @param key statistic key + * @return an ongoing assertion + */ + public static AbstractLongAssert assertThatStatisticCounter( + final IOStatistics stats, + final String key) { + return assertThatStatisticLong(COUNTER, key, + verifyStatisticsNotNull(stats).counters()); + } + + /** + * Start an assertion chain on + * a required gauge statistic. + * @param stats statistics source + * @param key statistic key + * @return an ongoing assertion + */ + public static AbstractLongAssert assertThatStatisticGauge( + final IOStatistics stats, + final String key) { + return assertThatStatisticLong(GAUGE, key, + verifyStatisticsNotNull(stats).gauges()); + } + + /** + * Start an assertion chain on + * a required minimum statistic. + * @param stats statistics source + * @param key statistic key + * @return an ongoing assertion + */ + public static AbstractLongAssert assertThatStatisticMinimum( + final IOStatistics stats, + final String key) { + return assertThatStatisticLong(MINIMUM, key, + verifyStatisticsNotNull(stats).minimums()); + } + + /** + * Start an assertion chain on + * a required maximum statistic. + * @param stats statistics source + * @param key statistic key + * @return an ongoing assertion + */ + public static AbstractLongAssert assertThatStatisticMaximum( + final IOStatistics stats, + final String key) { + return assertThatStatisticLong(MAXIMUM, key, + verifyStatisticsNotNull(stats).maximums()); + } + + /** + * Start an assertion chain on + * a required mean statistic. + * @param stats statistics source + * @param key statistic key + * @return an ongoing assertion + */ + public static ObjectAssert assertThatStatisticMean( + final IOStatistics stats, + final String key) { + return assertThatStatistic(MEAN, key, + verifyStatisticsNotNull(stats).meanStatistics()); + } + + /** + * Start an assertion chain on + * a required mean statistic with the initial validation on the + * sample count and sum. + * @param stats statistics source + * @param key statistic key + * @return an ongoing assertion + */ + public static ObjectAssert assertThatStatisticMeanMatches( + final IOStatistics stats, + final String key, + final long samples, + final long sum) { + return assertThatStatisticMean(stats, key) + .matches(p -> (p.getSamples() == samples), + "samples == " + samples) + .matches(p -> (p.getSum() == sum), + "sum == " + sum); + } + + /** + * Assert that a given counter statistic is untracked. + * @param stats statistics source + * @param type type for error text + * @param key statistic key + * @param map map to probe + */ + private static void assertUntracked(final IOStatistics stats, + final String type, + final String key, + final Map map) { + assertThat(map.containsKey(key)) + .describedAs("%s %s is tracked in %s", type, key, stats) + .isFalse(); + } + + /** + * Assert that a given counter statistic is untracked. + * @param stats statistics source + * @param type type for error text + * @param key statistic key + * @param map map to probe + */ + private static void assertTracked(final IOStatistics stats, + final String type, + final String key, + final Map map) { + assertThat(map.containsKey(key)) + .describedAs("%s %s is not tracked in %s", type, key, stats) + .isTrue(); + } + + /** + * Assert that a given statistic is tracked. + * @param stats statistics source + * @param key statistic key + */ + public static void assertStatisticCounterIsTracked( + final IOStatistics stats, + final String key) { + assertTracked(stats, COUNTER, key, + verifyStatisticsNotNull(stats).counters()); + } + + /** + * Assert that a given counter statistic is untracked. + * @param stats statistics source + * @param key statistic key + */ + public static void assertStatisticCounterIsUntracked( + final IOStatistics stats, + final String key) { + assertUntracked(stats, COUNTER, key, + verifyStatisticsNotNull(stats).counters()); + } + + /** + * Assert that an object is a statistics source and that the + * statistics is not null. + * @param source source object. + */ + public static void assertIsStatisticsSource(Object source) { + assertThat(source) + .describedAs("Object %s", source) + .isInstanceOf(IOStatisticsSource.class) + .extracting(o -> ((IOStatisticsSource) o).getIOStatistics()) + .isNotNull(); + } + + /** + * Query the source for the statistics; fails if the statistics + * returned are null or the class does not implement the API. + * @param source source object. + * @return the statistics it provides. + */ + public static IOStatistics extractStatistics(Object source) { + assertThat(source) + .describedAs("Object %s", source) + .isInstanceOf(IOStatisticsSource.class); + IOStatisticsSource ios = (IOStatisticsSource) source; + return extractStatistics(ios); + } + + /** + * Get the non-null statistics. + * @param ioStatisticsSource source + * @return the statistics, guaranteed to be non null + */ + private static IOStatistics extractStatistics( + final IOStatisticsSource ioStatisticsSource) { + IOStatistics statistics = ioStatisticsSource.getIOStatistics(); + assertThat(statistics) + .describedAs("Statistics from %s", ioStatisticsSource) + .isNotNull(); + return statistics; + } + + /** + * Perform a serialization round trip on a statistics instance. + * @param stat statistic + * @return the deserialized version. + */ + public static IOStatistics statisticsJavaRoundTrip(final IOStatistics stat) + throws IOException, ClassNotFoundException { + assertThat(stat).isInstanceOf(Serializable.class); + ByteArrayOutputStream baos = new ByteArrayOutputStream(1024); + try (ObjectOutputStream oos = new ObjectOutputStream(baos)) { + oos.writeObject(stat); + } + ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray()); + IOStatistics deser; + try (ObjectInputStream ois = new RestrictedInput(bais, + IOStatisticsSnapshot.requiredSerializationClasses())) { + deser = (IOStatistics) ois.readObject(); + } + return deser; + } + + private static final class RestrictedInput extends ObjectInputStream { + + private final List allowedClasses; + + private RestrictedInput(final InputStream in, + final List allowedClasses) throws IOException { + + super(in); + this.allowedClasses = allowedClasses.stream() + .map(Class::getName) + .collect(Collectors.toList()); + } + + @Override + protected Class resolveClass(final ObjectStreamClass desc) + throws IOException, ClassNotFoundException { + final String classname = desc.getName(); + if (!allowedClasses.contains(classname)) { + throw new ClassNotFoundException("Class " + classname + + " Not in list of allowed classes"); + } + + return super.resolveClass(desc); + } + } + +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestDurationTracking.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestDurationTracking.java new file mode 100644 index 0000000000000..8258b62c1f759 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestDurationTracking.java @@ -0,0 +1,361 @@ +/* + * 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.FileNotFoundException; +import java.io.IOException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.impl.FutureIOSupport; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; +import org.apache.hadoop.test.AbstractHadoopTestBase; +import org.apache.hadoop.util.functional.FunctionRaisingIOE; +import org.apache.hadoop.util.functional.FutureIO; + +import static org.apache.hadoop.fs.statistics.DurationStatisticSummary.fetchDurationSummary; +import static org.apache.hadoop.fs.statistics.DurationStatisticSummary.fetchSuccessSummary; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.*; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.*; +import static org.apache.hadoop.fs.statistics.impl.StubDurationTrackerFactory.STUB_DURATION_TRACKER_FACTORY; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Test the IOStatistic DurationTracker logic. + */ +public class TestDurationTracking extends AbstractHadoopTestBase { + + private static final Logger LOG = + LoggerFactory.getLogger(TestDurationTracking.class); + + private static final String REQUESTS = "requests"; + + public static final String UNKNOWN = "unknown"; + + private IOStatisticsStore stats; + + private final AtomicInteger invocationCounter = new AtomicInteger(0); + + @Before + public void setup() { + stats = iostatisticsStore() + .withDurationTracking(REQUESTS) + .build(); + } + + @After + public void teardown() { + LOG.info("stats {}", stats); + } + + /** + * Duration tracking. + */ + @Test + public void testDurationTryWithResources() throws Throwable { + DurationTracker tracker = + stats.trackDuration(REQUESTS); + verifyStatisticCounterValue(stats, REQUESTS, 1L); + sleep(); + tracker.close(); + try (DurationTracker ignored = + stats.trackDuration(REQUESTS)) { + sleep(); + } + LOG.info("Statistics: {}", stats); + DurationStatisticSummary summary = fetchSuccessSummary(stats, REQUESTS); + assertSummaryValues(summary, 2, 1, 1); + assertSummaryMean(summary, 2, 0); + } + + /** + * A little sleep method; exceptions are swallowed. + * Increments {@link #invocationCounter}. + * Increments {@inheritDoc #atomicCounter}. + */ + public void sleep() { + sleepf(10); + } + + /** + * A little sleep function; exceptions are swallowed. + * Increments {@link #invocationCounter}. + */ + protected int sleepf(final int millis) { + invocationCounter.incrementAndGet(); + try { + Thread.sleep(millis); + } catch (InterruptedException ignored) { + } + return millis; + } + + /** + * Assert that the sleep counter has been invoked + * the expected number of times. + * @param expected expected value + */ + private void assertCounterValue(final int expected) { + assertThat(invocationCounter.get()) + .describedAs("Sleep invocation Counter") + .isEqualTo(expected); + } + + /** + * Test that a function raising an IOE can be wrapped. + */ + @Test + public void testDurationFunctionIOE() throws Throwable { + FunctionRaisingIOE fn = + trackFunctionDuration(stats, REQUESTS, + (Integer x) -> invocationCounter.getAndSet(x)); + assertThat(fn.apply(1)).isEqualTo(0); + assertCounterValue(1); + assertSummaryValues( + fetchSuccessSummary(stats, REQUESTS), + 1, 0, 0); + } + + /** + * Trigger a failure and verify its the failure statistics + * which go up. + */ + @Test + public void testDurationFunctionIOEFailure() throws Throwable { + FunctionRaisingIOE fn = + trackFunctionDuration(stats, REQUESTS, + (Integer x) -> { + sleep(); + return 100 / x; + }); + intercept(ArithmeticException.class, + () -> fn.apply(0)); + assertSummaryValues( + fetchSuccessSummary(stats, REQUESTS), + 1, -1, -1); + + DurationStatisticSummary failures = fetchDurationSummary(stats, REQUESTS, + false); + assertSummaryValues(failures, 1, 0, 0); + assertSummaryMean(failures, 1, 0); + } + + /** + * Trigger a failure and verify its the failure statistics + * which go up. + */ + @Test + public void testDurationJavaFunctionFailure() throws Throwable { + Function fn = + trackJavaFunctionDuration(stats, REQUESTS, + (Integer x) -> { + return 100 / x; + }); + intercept(ArithmeticException.class, + () -> fn.apply(0)); + assertSummaryValues( + fetchSuccessSummary(stats, REQUESTS), + 1, -1, -1); + + DurationStatisticSummary failures = fetchDurationSummary(stats, REQUESTS, + false); + assertSummaryValues(failures, 1, 0, 0); + } + + /** + * Test trackDurationOfCallable. + */ + @Test + public void testCallableDuration() throws Throwable { + // call the operation + assertThat( + trackDurationOfCallable(stats, REQUESTS, () -> sleepf(100)).call()) + .isEqualTo(100); + DurationStatisticSummary summary = fetchSuccessSummary(stats, REQUESTS); + assertSummaryValues(summary, 1, 0, 0); + assertSummaryMean(summary, 1, 0); + } + + /** + * Callable raising an RTE after a sleep; failure + * stats will be updated and the execution count will be + * 1. + */ + @Test + public void testCallableFailureDuration() throws Throwable { + + intercept(RuntimeException.class, + trackDurationOfCallable(stats, REQUESTS, () -> { + sleepf(100); + throw new RuntimeException("oops"); + })); + assertCounterValue(1); + assertSummaryValues( + fetchSuccessSummary(stats, REQUESTS), + 1, -1, -1); + + assertSummaryValues(fetchDurationSummary(stats, REQUESTS, false), + 1, 0, 0); + } + + /** + * Duration of the successful execution of a InvocationRaisingIOE. + */ + @Test + public void testInvocationDuration() throws Throwable { + // call the operation + trackDurationOfInvocation(stats, REQUESTS, () -> { + sleepf(100); + }); + assertCounterValue(1); + DurationStatisticSummary summary = fetchSuccessSummary(stats, REQUESTS); + assertSummaryValues(summary, 1, 0, 0); + assertSummaryMean(summary, 1, 0); + } + + /** + * Duration of the successful execution of a CallableRaisingIOE. + */ + @Test + public void testCallableIOEDuration() throws Throwable { + // call the operation + assertThat( + trackDuration(stats, REQUESTS, () -> sleepf(100))) + .isEqualTo(100); + DurationStatisticSummary summary = fetchSuccessSummary(stats, REQUESTS); + assertSummaryValues(summary, 1, 0, 0); + assertSummaryMean(summary, 1, 0); + } + + /** + * Track the duration of an IOE raising callable which fails. + */ + @Test + public void testCallableIOEFailureDuration() throws Throwable { + intercept(IOException.class, + () -> + trackDuration(stats, REQUESTS, () -> { + sleepf(100); + throw new IOException("oops"); + })); + assertSummaryValues( + fetchSuccessSummary(stats, REQUESTS), + 1, -1, -1); + + assertSummaryValues(fetchDurationSummary(stats, REQUESTS, false), + 1, 0, 0); + } + + + /** + * Track the duration of an IOE raising callable which fails. + */ + @Test + public void testDurationThroughEval() throws Throwable { + CompletableFuture eval = FutureIOSupport.eval( + trackDurationOfOperation(stats, REQUESTS, () -> { + sleepf(100); + throw new FileNotFoundException("oops"); + })); + intercept(FileNotFoundException.class, "oops", () -> + FutureIO.awaitFuture(eval)); + assertSummaryValues(fetchDurationSummary(stats, REQUESTS, false), + 1, 0, 0); + } + + /** + * It's OK to track a duration against an unknown statistic. + */ + @Test + public void testUnknownDuration() throws Throwable { + trackDurationOfCallable(stats, UNKNOWN, () -> sleepf(1)).call(); + DurationStatisticSummary summary = fetchSuccessSummary(stats, UNKNOWN); + assertSummaryValues(summary, 0, -1, -1); + assertThat(summary.getMean()).isNull(); + } + + /** + * The stub duration tracker factory can be supplied as an input. + */ + @Test + public void testTrackDurationWithStubFactory() throws Throwable { + trackDuration(STUB_DURATION_TRACKER_FACTORY, UNKNOWN, () -> sleepf(1)); + } + + /** + * Make sure the tracker returned from the stub factory + * follows the basic lifecycle. + */ + @Test + public void testStubDurationLifecycle() throws Throwable { + DurationTracker tracker = STUB_DURATION_TRACKER_FACTORY + .trackDuration("k", 1); + tracker.failed(); + tracker.close(); + tracker.close(); + } + + /** + * Assert that a statistics summary has the specific values. + * @param summary summary data + * @param count count -must match exactly. + * @param minBase minimum value for the minimum field (inclusive) + * @param maxBase minimum value for the maximum field (inclusive) + */ + protected void assertSummaryValues( + final DurationStatisticSummary summary, + final int count, + final int minBase, + final int maxBase) { + assertThat(summary) + .matches(s -> s.getCount() == count, "Count value") + .matches(s -> s.getMax() >= maxBase, "Max value") + .matches(s -> s.getMin() >= minBase, "Min value"); + } + + /** + * Assert that at a summary has a matching mean value. + * @param summary summary data. + * @param expectedSampleCount sample count -which must match + * @param meanGreaterThan the mean must be greater than this value. + */ + protected void assertSummaryMean( + final DurationStatisticSummary summary, + final int expectedSampleCount, + final double meanGreaterThan) { + String description = "mean of " + summary; + assertThat(summary.getMean()) + .describedAs(description) + .isNotNull(); + assertThat(summary.getMean().getSamples()) + .describedAs(description) + .isEqualTo(expectedSampleCount); + assertThat(summary.getMean().mean()) + .describedAs(description) + .isGreaterThan(meanGreaterThan); + } +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestDynamicIOStatistics.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestDynamicIOStatistics.java new file mode 100644 index 0000000000000..9b929ac82ff11 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestDynamicIOStatistics.java @@ -0,0 +1,311 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics; + +import java.util.Iterator; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; + +import org.assertj.core.api.Assertions; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.statistics.impl.SourceWrappedStatistics; +import org.apache.hadoop.metrics2.MetricsInfo; +import org.apache.hadoop.metrics2.lib.MutableCounterLong; +import org.apache.hadoop.test.AbstractHadoopTestBase; + +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertStatisticCounterIsTracked; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertStatisticCounterIsUntracked; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.demandStringifyIOStatistics; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.demandStringifyIOStatisticsSource; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToString; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.ENTRY_PATTERN; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.NULL_SOURCE; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.dynamicIOStatistics; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.emptyStatistics; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** + * verify dynamic statistics are dynamic, except when you iterate through + * them, along with other tests of the class's behavior. + */ +public class TestDynamicIOStatistics extends AbstractHadoopTestBase { + + private static final Logger LOG = + LoggerFactory.getLogger(TestDynamicIOStatistics.class); + + private static final String ALONG = "along"; + + private static final String AINT = "aint"; + + private static final String COUNT = "count"; + + private static final String EVAL = "eval"; + + /** + * The statistics. + */ + private IOStatistics statistics = emptyStatistics(); + + /** + * A source of these statistics. + */ + private IOStatisticsSource statsSource; + + private final AtomicLong aLong = new AtomicLong(); + + private final AtomicInteger aInt = new AtomicInteger(); + + private final MutableCounterLong counter = new MutableCounterLong( + new Info("counter"), 0); + + private long evalLong; + + private static final String[] KEYS = new String[]{ALONG, AINT, COUNT, EVAL}; + + @Before + public void setUp() throws Exception { + statistics = dynamicIOStatistics() + .withAtomicLongCounter(ALONG, aLong) + .withAtomicIntegerCounter(AINT, aInt) + .withMutableCounter(COUNT, counter) + .withLongFunctionCounter(EVAL, x -> evalLong) + .build(); + statsSource = new SourceWrappedStatistics(statistics); + } + + /** + * The eval operation is foundational. + */ + @Test + public void testEval() throws Throwable { + verifyStatisticCounterValue(statistics, EVAL, 0); + evalLong = 10; + verifyStatisticCounterValue(statistics, EVAL, 10); + } + + /** + * Atomic Long statistic. + */ + @Test + public void testAlong() throws Throwable { + verifyStatisticCounterValue(statistics, ALONG, 0); + aLong.addAndGet(1); + verifyStatisticCounterValue(statistics, ALONG, 1); + } + + /** + * Atomic Int statistic. + */ + @Test + public void testAint() throws Throwable { + verifyStatisticCounterValue(statistics, AINT, 0); + aInt.addAndGet(1); + verifyStatisticCounterValue(statistics, AINT, 1); + } + + /** + * Metrics2 counter. + */ + @Test + public void testCounter() throws Throwable { + verifyStatisticCounterValue(statistics, COUNT, 0); + counter.incr(); + verifyStatisticCounterValue(statistics, COUNT, 1); + } + + /** + * keys() returns all the keys. + */ + @Test + public void testKeys() throws Throwable { + Assertions.assertThat(statistics.counters().keySet()) + .describedAs("statistic keys of %s", statistics) + .containsExactlyInAnyOrder(KEYS); + } + + @Test + public void testIteratorHasAllKeys() throws Throwable { + // go through the statistics iterator and assert that it contains exactly + // the values. + assertThat(statistics.counters().keySet()) + .containsExactlyInAnyOrder(KEYS); + } + + /** + * Verify that the iterator is taken from + * a snapshot of the values. + */ + @Test + public void testIteratorIsSnapshot() throws Throwable { + // set the counters all to 1 + incrementAllCounters(); + // take the snapshot + final Iterator> it = + statistics.counters().entrySet().iterator(); + // increment the counters + incrementAllCounters(); + // now assert that all the iterator values are of value 1 + while (it.hasNext()) { + Map.Entry next = it.next(); + assertThat(next.getValue()) + .describedAs("Value of entry %s", next) + .isEqualTo(1); + } + } + + @Test + public void testUnknownStatistic() throws Throwable { + assertStatisticCounterIsUntracked(statistics, "anything"); + } + + @Test + public void testStatisticsTrackedAssertion() throws Throwable { + // expect an exception to be raised when an assertion + // is made that an unknown statistic is tracked,. + assertThatThrownBy(() -> + assertStatisticCounterIsTracked(statistics, "anything")) + .isInstanceOf(AssertionError.class); + } + + @Test + public void testStatisticsValueAssertion() throws Throwable { + // expect an exception to be raised when + // an assertion is made about the value of an unknown statistics + assertThatThrownBy(() -> + verifyStatisticCounterValue(statistics, "anything", 0)) + .isInstanceOf(AssertionError.class); + } + + /** + * Serialization round trip will preserve all the values. + */ + @Test + public void testSerDeser() throws Throwable { + incrementAllCounters(); + IOStatistics stat = IOStatisticsSupport.snapshotIOStatistics(statistics); + incrementAllCounters(); + IOStatistics deser = IOStatisticAssertions.statisticsJavaRoundTrip(stat); + assertThat(deser.counters().keySet()) + .containsExactlyInAnyOrder(KEYS); + for (Map.Entry e : deser.counters().entrySet()) { + assertThat(e.getValue()) + .describedAs("Value of entry %s", e) + .isEqualTo(1); + } + } + + @Test + public void testStringification() throws Throwable { + assertThat(ioStatisticsToString(statistics)) + .isNotBlank() + .contains(KEYS); + } + + @Test + public void testDemandStringification() throws Throwable { + String counterPattern = ENTRY_PATTERN; + // this is not yet evaluated + Object demand = demandStringifyIOStatistics(statistics); + // nor is this. + Object demandSource = demandStringifyIOStatisticsSource(statsSource); + + // show it evaluates + String formatted1 = String.format(counterPattern, ALONG, aLong.get()); + assertThat(demand + .toString()) + .contains(formatted1); + assertThat(demandSource + .toString()) + .contains(formatted1); + + // when the counters are incremented + incrementAllCounters(); + incrementAllCounters(); + // there are new values to expect + String formatted2 = String.format(counterPattern, ALONG, aLong.get()); + assertThat(demand + .toString()) + .doesNotContain(formatted1) + .contains(formatted2); + assertThat(demandSource + .toString()) + .doesNotContain(formatted1) + .contains(formatted2); + } + + @Test + public void testNullSourceStringification() throws Throwable { + assertThat(demandStringifyIOStatisticsSource((IOStatisticsSource) null) + .toString()) + .isEqualTo(NULL_SOURCE); + } + + @Test + public void testNullStatStringification() throws Throwable { + assertThat(demandStringifyIOStatistics((IOStatistics) null) + .toString()) + .isEqualTo(NULL_SOURCE); + } + + @Test + public void testStringLogging() throws Throwable { + LOG.info("Output {}", demandStringifyIOStatistics(statistics)); + } + + /** + * Increment all the counters from their current value. + */ + private void incrementAllCounters() { + aLong.incrementAndGet(); + aInt.incrementAndGet(); + evalLong += 1; + counter.incr(); + } + + /** + * Needed to provide a metrics info instance for the counter + * constructor. + */ + private static final class Info implements MetricsInfo { + + private final String name; + + private Info(final String name) { + this.name = name; + } + + @Override + public String name() { + return name; + } + + @Override + public String description() { + return name; + } + } + +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestEmptyIOStatistics.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestEmptyIOStatistics.java new file mode 100644 index 0000000000000..296470abaa9bf --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestEmptyIOStatistics.java @@ -0,0 +1,110 @@ +/* + * 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.junit.Test; + +import org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding; +import org.apache.hadoop.test.AbstractHadoopTestBase; + +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertStatisticCounterIsTracked; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertStatisticCounterIsUntracked; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToString; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.emptyStatistics; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** + * Test handling of the empty IO statistics class. + */ +public class TestEmptyIOStatistics extends AbstractHadoopTestBase { + + private final IOStatistics empty = emptyStatistics(); + + @Test + public void testUnknownStatistic() throws Throwable { + assertStatisticCounterIsUntracked(empty, "anything"); + } + + @Test + public void testStatisticsTrackedAssertion() throws Throwable { + // expect an exception to be raised when an assertion + // is made that an unknown statistic is tracked,. + assertThatThrownBy(() -> + assertStatisticCounterIsTracked(empty, "anything")) + .isInstanceOf(AssertionError.class); + } + + @Test + public void testStatisticsValueAssertion() throws Throwable { + // expect an exception to be raised when + // an assertion is made about the value of an unknown statistics + assertThatThrownBy(() -> + verifyStatisticCounterValue(empty, "anything", 0)) + .isInstanceOf(AssertionError.class); + } + + @Test + public void testEmptySnapshot() throws Throwable { + final IOStatistics stat = IOStatisticsSupport.snapshotIOStatistics(empty); + assertThat(stat.counters().keySet()) + .describedAs("keys of snapshot") + .isEmpty(); + IOStatistics deser = IOStatisticAssertions.statisticsJavaRoundTrip(stat); + assertThat(deser.counters().keySet()) + .describedAs("keys of deserialized snapshot") + .isEmpty(); + } + + @Test + public void testStringification() throws Throwable { + assertThat(ioStatisticsToString(empty)) + .isNotBlank(); + } + + @Test + public void testWrap() throws Throwable { + IOStatisticsSource statisticsSource = IOStatisticsBinding.wrap(empty); + assertThat(statisticsSource.getIOStatistics()) + .isSameAs(empty); + } + + @Test + public void testStringifyNullSource() throws Throwable { + assertThat(IOStatisticsLogging.ioStatisticsSourceToString(null)) + .isEmpty(); + } + + @Test + public void testStringifyNullStats() throws Throwable { + assertThat( + IOStatisticsLogging.ioStatisticsSourceToString( + IOStatisticsBinding.wrap(null))) + .isEmpty(); + } + + @Test + public void testStringificationNull() throws Throwable { + assertThat(ioStatisticsToString(null)) + .describedAs("Null statistics should stringify to \"\"") + .isEmpty(); + } + +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestIOStatisticsSnapshot.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestIOStatisticsSnapshot.java new file mode 100644 index 0000000000000..41e9bffefe834 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestIOStatisticsSnapshot.java @@ -0,0 +1,147 @@ +/* + * 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.assertj.core.api.Assertions; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding; +import org.apache.hadoop.test.AbstractHadoopTestBase; +import org.apache.hadoop.util.JsonSerialization; + +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.*; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToString; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** + * Test handling of the {@link IOStatisticsSnapshot} class. + */ +public class TestIOStatisticsSnapshot extends AbstractHadoopTestBase { + + private static final Logger LOG = + LoggerFactory.getLogger(TestIOStatisticsSnapshot.class); + + /** + * Simple snapshot built up in test setup. + */ + private final IOStatisticsSnapshot snapshot = new IOStatisticsSnapshot(); + + /** Saved to the snapshot as "mean01". */ + private MeanStatistic mean0; + + /** Saved to the snapshot as "mean1". */ + private MeanStatistic mean1; + + @Before + public void setup() throws Exception { + snapshot.counters().put("c1", 0L); + snapshot.gauges().put("g1", 1L); + snapshot.minimums().put("m1", -1L); + mean1 = new MeanStatistic(1, 1); + snapshot.meanStatistics().put("mean1", + mean1); + mean0 = new MeanStatistic(0, 1); + snapshot.meanStatistics().put("mean0", + mean0); + } + + @Test + public void testTrackedValues() throws Throwable { + verifyStatisticCounterValue(snapshot, "c1", 0L); + verifyStatisticGaugeValue(snapshot, "g1", 1L); + verifyStatisticMinimumValue(snapshot, "m1", -1L); + verifyStatisticMeanValue(snapshot, "mean0", + new MeanStatistic(0, 1)); + } + + @Test + public void testStatisticsValueAssertion() throws Throwable { + // expect an exception to be raised when + // an assertion is made about the value of an unknown statistics + assertThatThrownBy(() -> + verifyStatisticCounterValue(snapshot, "anything", 0)) + .isInstanceOf(AssertionError.class); + } + + @Test + public void testStringification() throws Throwable { + assertThat(ioStatisticsToString(snapshot)) + .isNotBlank(); + } + + @Test + public void testStringification2() throws Throwable { + + String ss = snapshot.toString(); + LOG.info("original {}", ss); + Assertions.assertThat(ss) + .describedAs("snapshot toString()") + .contains("c1=0") + .contains("g1=1"); + } + + @Test + public void testWrap() throws Throwable { + IOStatisticsSource statisticsSource = IOStatisticsBinding.wrap(snapshot); + assertThat(statisticsSource.getIOStatistics()) + .isSameAs(snapshot); + } + + @Test + public void testJsonRoundTrip() throws Throwable { + JsonSerialization serializer + = IOStatisticsSnapshot.serializer(); + + String json = serializer.toJson(snapshot); + LOG.info("serialized form\n{}", json); + IOStatisticsSnapshot deser = serializer.fromJson(json); + verifyDeserializedInstance(deser); + } + + /** + * Verify the deserialized instance's data + * matches the expected values. + * @param deser deserialized vlaue. + */ + public void verifyDeserializedInstance( + final IOStatistics deser) { + LOG.info("deserialized {}", deser); + verifyStatisticCounterValue(deser, "c1", 0L); + verifyStatisticGaugeValue(deser, "g1", 1L); + verifyStatisticMinimumValue(deser, "m1", -1L); + verifyStatisticMeanValue(deser, "mean0", + new MeanStatistic(0, 1)); + verifyStatisticMeanValue(deser, "mean1", + snapshot.meanStatistics().get("mean1")); + } + + @Test + public void testJavaRoundTrip() throws Throwable { + verifyDeserializedInstance( + IOStatisticAssertions.statisticsJavaRoundTrip( + snapshot)); + + + } + +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestIOStatisticsStore.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestIOStatisticsStore.java new file mode 100644 index 0000000000000..778eab8315aa5 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestIOStatisticsStore.java @@ -0,0 +1,177 @@ +/* + * 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.assertj.core.api.Assertions; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; +import org.apache.hadoop.test.AbstractHadoopTestBase; +import org.apache.hadoop.util.JsonSerialization; + +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticMeanMatches; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticGaugeValue; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticMaximumValue; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticMinimumValue; +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.snapshotIOStatistics; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore; + +/** + * Test the IOStatisticStore implementation. + */ +public class TestIOStatisticsStore extends AbstractHadoopTestBase { + + private static final Logger LOG = + LoggerFactory.getLogger(TestIOStatisticsStore.class); + + + private static final String COUNT = "count"; + + private static final String GAUGE = "gauge"; + + private static final String MIN = "min"; + + private static final String MAX = "max"; + + private static final String MEAN = "mean"; + + public static final String UNKNOWN = "unknown"; + + private IOStatisticsStore stats; + + @Before + public void setup() { + stats = iostatisticsStore() + .withCounters(COUNT) + .withGauges(GAUGE) + .withMinimums(MIN) + .withMaximums(MAX) + .withMeanStatistics(MEAN) + .build(); + } + + @After + public void teardown() { + LOG.info("stats {}", stats); + } + + /** + * Gauges go up and down. + */ + @Test + public void testGauges() throws Throwable { + stats.setGauge(GAUGE, 1); + verifyStatisticGaugeValue(stats, GAUGE, 1); + stats.incrementGauge(GAUGE, 1); + verifyStatisticGaugeValue(stats, GAUGE, 2); + stats.setGauge(GAUGE, -1); + verifyStatisticGaugeValue(stats, GAUGE, -1); + Assertions.assertThat(stats.incrementGauge(GAUGE, -1)) + .isEqualTo(-2); + verifyStatisticGaugeValue(stats, GAUGE, -2); + Assertions.assertThat(stats.getGaugeReference(GAUGE).get()) + .isEqualTo(-2); + stats.setGauge(UNKNOWN, 1); + Assertions.assertThat(stats.incrementGauge(UNKNOWN, 1)) + .isEqualTo(0); + } + + @Test + public void testMinimums() throws Throwable { + stats.setMinimum(MIN, 100); + verifyStatisticMinimumValue(stats, MIN, 100); + stats.setMinimum(MIN, 100); + // will do nothing as it is higher + stats.addMinimumSample(MIN, 200); + verifyStatisticMinimumValue(stats, MIN, 100); + stats.addMinimumSample(MIN, 10); + verifyStatisticMinimumValue(stats, MIN, 10); + stats.setMinimum(UNKNOWN, 100); + stats.addMinimumSample(UNKNOWN, 200); + } + + @Test + public void testMaximums() throws Throwable { + stats.setMaximum(MAX, 100); + verifyStatisticMaximumValue(stats, MAX, 100); + stats.setMaximum(MAX, 100); + stats.addMaximumSample(MAX, 200); + verifyStatisticMaximumValue(stats, MAX, 200); + stats.addMaximumSample(MAX, 10); + verifyStatisticMaximumValue(stats, MAX, 200); + stats.setMaximum(UNKNOWN, 100); + stats.addMaximumSample(UNKNOWN, 200); + } + + @Test + public void testMeans() throws Throwable { + stats.setMeanStatistic(MEAN, + new MeanStatistic(1, 1)); + + assertThatStatisticMeanMatches(stats, MEAN, 1, 1) + .matches(p -> p.mean() == 1, "mean"); + stats.addMeanStatisticSample(MEAN, 9); + assertThatStatisticMeanMatches(stats, MEAN, 2, 10) + .matches(p -> p.mean() == 5, "mean"); + } + + @Test + public void testRoundTrip() throws Throwable { + JsonSerialization serializer + = IOStatisticsSnapshot.serializer(); + stats.incrementCounter(COUNT); + stats.setGauge(GAUGE, -1); + stats.addMaximumSample(MAX, 200); + stats.addMinimumSample(MIN, -100); + stats.addMeanStatisticSample(MEAN, 1); + stats.addMeanStatisticSample(MEAN, 9); + + String json = serializer.toJson(snapshotIOStatistics(stats)); + LOG.info("serialized form\n{}", json); + IOStatisticsSnapshot deser = serializer.fromJson(json); + LOG.info("deserialized {}", deser); + verifyStatisticCounterValue(deser, COUNT, 1L); + verifyStatisticGaugeValue(deser, GAUGE, -1); + verifyStatisticMaximumValue(deser, MAX, 200); + verifyStatisticMinimumValue(deser, MIN, -100); + assertThatStatisticMeanMatches(deser, MEAN, 2, 10) + .matches(p -> p.mean() == 5, "mean"); + + } + + @Test + public void testUnknownCounter() throws Throwable { + Assertions.assertThat(stats.incrementCounter("unknown", -10)) + .isEqualTo(0); + } + + @Test + public void testNegativeCounterIncrementIgnored() throws Throwable { + Assertions.assertThat(stats.incrementCounter(COUNT, 2)) + .isEqualTo(2); + Assertions.assertThat(stats.incrementCounter(COUNT, -10)) + .isEqualTo(2); + } + +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestMeanStatistic.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestMeanStatistic.java new file mode 100644 index 0000000000000..749a6ee4d9eb4 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestMeanStatistic.java @@ -0,0 +1,219 @@ +/* + * 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.assertj.core.api.Assertions; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.test.AbstractHadoopTestBase; +import org.apache.hadoop.util.JsonSerialization; + +/** + * Test the {@link MeanStatistic} class. + */ +public class TestMeanStatistic extends AbstractHadoopTestBase { + + private static final Logger LOG = + LoggerFactory.getLogger(TestMeanStatistic.class); + + private static final int TEN = 10; + + private static final double ZEROD = 0.0d; + + private static final double TEND = 10.0d; + + private final MeanStatistic empty = new MeanStatistic(0, 0); + + private final MeanStatistic tenFromOne = new MeanStatistic(1, TEN); + + private final MeanStatistic tenFromTen = new MeanStatistic(TEN, TEN); + + @Test + public void testEmptiness() throws Throwable { + Assertions.assertThat(empty) + .matches(MeanStatistic::isEmpty, "is empty") + .isEqualTo(new MeanStatistic(0, TEN)) + .isEqualTo(new MeanStatistic()) + .isNotEqualTo(tenFromOne); + Assertions.assertThat(empty.mean()) + .isEqualTo(ZEROD); + Assertions.assertThat(empty.toString()) + .contains("0.0"); + } + + @Test + public void testTenFromOne() throws Throwable { + Assertions.assertThat(tenFromOne) + .matches(p -> !p.isEmpty(), "is not empty") + .isEqualTo(tenFromOne) + .isNotEqualTo(tenFromTen); + Assertions.assertThat(tenFromOne.mean()) + .isEqualTo(TEND); + } + + @Test + public void testNegativeSamplesAreEmpty() throws Throwable { + MeanStatistic stat = new MeanStatistic(-10, 1); + Assertions.assertThat(stat) + .describedAs("stat with negative samples") + .matches(MeanStatistic::isEmpty, "is empty") + .isEqualTo(empty) + .extracting(MeanStatistic::mean) + .isEqualTo(ZEROD); + Assertions.assertThat(stat.toString()) + .contains("0.0"); + + } + + @Test + public void testCopyNonEmpty() throws Throwable { + MeanStatistic stat = tenFromOne.copy(); + Assertions.assertThat(stat) + .describedAs("copy of " + tenFromOne) + .isEqualTo(tenFromOne) + .isNotSameAs(tenFromOne); + } + + @Test + public void testCopyEmpty() throws Throwable { + MeanStatistic stat = empty.copy(); + Assertions.assertThat(stat) + .describedAs("copy of " + empty) + .isEqualTo(empty) + .isNotSameAs(empty); + } + + @Test + public void testDoubleSamples() throws Throwable { + MeanStatistic stat = tenFromOne.copy(); + Assertions.assertThat(stat.add(tenFromOne)) + .isEqualTo(new MeanStatistic(2, 20)) + .extracting(MeanStatistic::mean) + .isEqualTo(TEND); + } + + @Test + public void testAddEmptyR() throws Throwable { + MeanStatistic stat = tenFromOne.copy(); + Assertions.assertThat(stat.add(empty)) + .isEqualTo(tenFromOne); + } + + @Test + public void testAddEmptyL() throws Throwable { + MeanStatistic stat = empty.copy(); + Assertions.assertThat(stat.add(tenFromOne)) + .isEqualTo(tenFromOne); + } + + @Test + public void testAddEmptyLR() throws Throwable { + MeanStatistic stat = empty.copy(); + Assertions.assertThat(stat.add(empty)) + .isEqualTo(empty); + } + + @Test + public void testAddSampleToEmpty() throws Throwable { + MeanStatistic stat = empty.copy(); + stat.addSample(TEN); + Assertions.assertThat(stat) + .isEqualTo(tenFromOne); + } + + @Test + public void testAddZeroValueSamples() throws Throwable { + MeanStatistic stat = tenFromOne.copy(); + for (int i = 0; i < 9; i++) { + stat.addSample(0); + } + Assertions.assertThat(stat) + .isEqualTo(tenFromTen); + } + + @Test + public void testSetSamples() throws Throwable { + MeanStatistic stat = tenFromOne.copy(); + stat.setSamples(10); + Assertions.assertThat(stat) + .isEqualTo(tenFromTen); + } + + @Test + public void testSetSums() throws Throwable { + MeanStatistic stat = tenFromOne.copy(); + stat.setSum(100); + stat.setSamples(20); + Assertions.assertThat(stat) + .isEqualTo(new MeanStatistic(20, 100)) + .extracting(MeanStatistic::mean) + .isEqualTo(5.0d); + } + + @Test + public void testSetNegativeSamplesMakesEmpty() throws Throwable { + MeanStatistic stat = tenFromOne.copy(); + stat.setSamples(-3); + Assertions.assertThat(stat) + .isEqualTo(empty); + } + + @Test + public void testJsonRoundTrip() throws Throwable { + JsonSerialization serializer = serializer(); + + String json = serializer.toJson(tenFromTen); + LOG.info("serialized form\n{}", json); + Assertions.assertThat(json) + .describedAs("JSON form of %s", tenFromTen) + .doesNotContain("empty") + .doesNotContain("mean"); + + MeanStatistic deser = serializer.fromJson(json); + LOG.info("deserialized {}", deser); + Assertions.assertThat(deser) + .isEqualTo(tenFromTen); + } + + /** + * negative sample counts in the json convert the stat to being empty. + */ + @Test + public void testHandleMaliciousStat() throws Throwable { + String json = "{\n" + + " \"sum\" : 10,\n" + + " \"samples\" : -10\n" + + "}"; + JsonSerialization serializer = serializer(); + MeanStatistic deser = serializer.fromJson(json); + LOG.info("deserialized {}", deser); + Assertions.assertThat(deser) + .isEqualTo(empty); + } + + /** + * Get a JSON serializer. + * @return a serializer. + */ + public static JsonSerialization serializer() { + return new JsonSerialization<>(MeanStatistic.class, true, true); + } +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/functional/TestRemoteIterators.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/functional/TestRemoteIterators.java new file mode 100644 index 0000000000000..8cd5c58585e6f --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/functional/TestRemoteIterators.java @@ -0,0 +1,469 @@ +/* + * 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.util.functional; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Iterator; +import java.util.NoSuchElementException; + +import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.test.AbstractHadoopTestBase; + +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.extractStatistics; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.apache.hadoop.util.functional.RemoteIterators.*; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Test for {@link RemoteIterators}. + * + */ +public class TestRemoteIterators extends AbstractHadoopTestBase { + + private static final Logger LOG = LoggerFactory.getLogger( + TestRemoteIterators.class); + + private static final String[] DATA = {"a", "b", "c"}; + + /** Counter for lambda-expressions. */ + private int counter; + + @Test + public void testIterateArray() throws Throwable { + verifyInvoked(remoteIteratorFromArray(DATA), DATA.length, + (s) -> LOG.info(s)); + } + + @Test + public void testIterateArrayMapped() throws Throwable { + verifyInvoked( + mappingRemoteIterator( + remoteIteratorFromArray(DATA), + (d) -> { + counter += d.length(); + return d; + }), + DATA.length, + this::log); + assertCounterValue(3); + } + + public void log(Object o) { + LOG.info("{}", o); + } + + /** + * Singleton is iterated through once. + * The toString() call is passed through. + */ + @Test + public void testSingleton() throws Throwable { + StringBuffer result = new StringBuffer(); + String name = "singleton"; + RemoteIterator it = remoteIteratorFromSingleton(name); + assertStringValueContains(it, "SingletonIterator"); + assertStringValueContains(it, name); + verifyInvoked( + it, + 1, + (s) -> result.append(s)); + assertThat(result.toString()) + .isEqualTo(name); + } + + @Test + public void testSingletonNotClosed() throws Throwable { + CloseCounter closeCounter = new CloseCounter(); + RemoteIterator it = remoteIteratorFromSingleton(closeCounter); + verifyInvoked(it, 1, this::log); + close(it); + closeCounter.assertCloseCount(0); + } + + /** + * A null singleton is not an error. + */ + @Test + public void testNullSingleton() throws Throwable { + verifyInvoked(remoteIteratorFromSingleton(null), 0, this::log); + } + + + /** + * If you create a singleton iterator and it is an IOStatisticsSource, + * then that is the statistics which can be extracted from the + * iterator. + */ + @Test + public void testSingletonStats() throws Throwable { + IOStatsInstance singleton = new IOStatsInstance(); + RemoteIterator it + = remoteIteratorFromSingleton(singleton); + extractStatistics(it); + } + + /** + * The mapping remote iterator passes IOStatistics + * calls down. + */ + @Test + public void testMappedSingletonStats() throws Throwable { + IOStatsInstance singleton = new IOStatsInstance(); + RemoteIterator it + = mappingRemoteIterator(remoteIteratorFromSingleton(singleton), + Object::toString); + verifyInvoked(it, 1, this::log); + extractStatistics(it); + } + + /** + * Close() calls are passed through. + */ + @Test + public void testClosePassthrough() throws Throwable { + CountdownRemoteIterator countdown = new CountdownRemoteIterator(0); + RemoteIterator it = mappingRemoteIterator( + countdown, + i -> i); + verifyInvoked(it, 0, this::log); + // the foreach() operation called close() + countdown.assertCloseCount(1); + extractStatistics(countdown); + ((Closeable)it).close(); + countdown.assertCloseCount(1); + } + + @Test + public void testMapping() throws Throwable { + CountdownRemoteIterator countdown = new CountdownRemoteIterator(100); + RemoteIterator it = mappingRemoteIterator( + countdown, + i -> i); + verifyInvoked(it, 100, c -> counter++); + assertCounterValue(100); + extractStatistics(it); + assertStringValueContains(it, "CountdownRemoteIterator"); + close(it); + countdown.assertCloseCount(1); + } + + @Test + public void testFiltering() throws Throwable { + CountdownRemoteIterator countdown = new CountdownRemoteIterator(100); + // only even numbers are passed through + RemoteIterator it = filteringRemoteIterator( + countdown, + i -> (i % 2) == 0); + verifyInvoked(it, 50, c -> counter++); + assertCounterValue(50); + extractStatistics(it); + close(it); + countdown.assertCloseCount(1); + } + + /** + * A filter which accepts nothing results in + * an empty iteration. + */ + @Test + public void testFilterNoneAccepted() throws Throwable { + // nothing gets through + RemoteIterator it = filteringRemoteIterator( + new CountdownRemoteIterator(100), + i -> false); + verifyInvoked(it, 0, c -> counter++); + assertCounterValue(0); + extractStatistics(it); + } + + @Test + public void testFilterAllAccepted() throws Throwable { + // nothing gets through + RemoteIterator it = filteringRemoteIterator( + new CountdownRemoteIterator(100), + i -> true); + verifyInvoked(it, 100, c -> counter++); + assertStringValueContains(it, "CountdownRemoteIterator"); + } + + @Test + public void testJavaIteratorSupport() throws Throwable { + CountdownIterator countdownIterator = new CountdownIterator(100); + RemoteIterator it = remoteIteratorFromIterator( + countdownIterator); + verifyInvoked(it, 100, c -> counter++); + assertStringValueContains(it, "CountdownIterator"); + extractStatistics(it); + close(it); + countdownIterator.assertCloseCount(1); + } + + @Test + public void testJavaIterableSupport() throws Throwable { + CountdownIterable countdown = new CountdownIterable(100); + RemoteIterator it = remoteIteratorFromIterable( + countdown); + verifyInvoked(it, 100, c -> counter++); + assertStringValueContains(it, "CountdownIterator"); + extractStatistics(it); + // close the iterator + close(it); + countdown.assertCloseCount(0); + // and a new iterator can be crated + verifyInvoked(remoteIteratorFromIterable(countdown), + 100, c -> counter++); + } + + /** + * If a RemoteIterator is constructed from an iterable + * and that is to be closed, we close it. + */ + @Test + public void testJavaIterableClose() throws Throwable { + CountdownIterable countdown = new CountdownIterable(100); + RemoteIterator it = closingRemoteIterator( + remoteIteratorFromIterable(countdown), + countdown); + verifyInvoked(it, 100, c -> counter++); + assertStringValueContains(it, "CountdownIterator"); + extractStatistics(it); + + // verify the iterator was self closed in hasNext() + countdown.assertCloseCount(1); + + // explicitly close the iterator + close(it); + countdown.assertCloseCount(1); + // and a new iterator cannot be created + intercept(IllegalStateException.class, () -> + remoteIteratorFromIterable(countdown)); + } + + /** + * If a RemoteIterator is constructed from an iterable + * and that is to be closed, we close it. + */ + @SuppressWarnings("InfiniteLoopStatement") + @Test + public void testJavaIterableCloseInNextLoop() throws Throwable { + CountdownIterable countdown = new CountdownIterable(100); + RemoteIterator it = closingRemoteIterator( + remoteIteratorFromIterable(countdown), + countdown); + try { + while(true) { + it.next(); + } + } catch (NoSuchElementException expected) { + + } + // verify the iterator was self closed in next() + countdown.assertCloseCount(1); + + } + + /** + * assert that the string value of an object contains the + * expected text. + * @param o object + * @param expected expected text + */ + protected void assertStringValueContains( + final Object o, + final String expected) { + assertThat(o.toString()) + .describedAs("Object string value") + .contains(expected); + } + + /** + * Assert that the counter field is at a specific value. + * @param expected counter + */ + protected void assertCounterValue(final int expected) { + assertThat(counter) + .describedAs("Counter value") + .isEqualTo(expected); + } + + /** + * Verify that the iteration completes with a given size. + * @param it iterator + * @param type. + * @param length expected size + * @param consumer consumer + */ + protected void verifyInvoked(final RemoteIterator it, + int length, + ConsumerRaisingIOE consumer) + throws IOException { + assertThat(foreach(it, consumer)) + .describedAs("Scan through iterator %s", it) + .isEqualTo(length); + } + + /** + * Close an iterator if it is iterable. + * @param it iterator + * @param type. + */ + private void close(final RemoteIterator it) throws IOException { + if (it instanceof Closeable) { + ((Closeable) it).close(); + } + } + + /** + * Class whose close() call increments a counter. + */ + private static class CloseCounter extends + IOStatsInstance implements Closeable { + + private int closeCount; + + @Override + public void close() throws IOException { + closeCount++; + LOG.info("close ${}", closeCount); + } + + public int getCloseCount() { + return closeCount; + } + + public void reset() { + closeCount = 0; + } + + public void assertCloseCount(int expected) { + assertThat(closeCount) + .describedAs("Close count") + .isEqualTo(expected); + } + + } + + /** + * Simple class to implement IOStatistics. + */ + private static class IOStatsInstance implements IOStatisticsSource { + + private IOStatisticsSnapshot stats = new IOStatisticsSnapshot(); + + @Override + public IOStatistics getIOStatistics() { + return stats; + } + + } + + /** + * Iterator which counts down. + */ + private static final class CountdownRemoteIterator extends CloseCounter + implements RemoteIterator { + + private int limit; + + private CountdownRemoteIterator(final int limit) { + this.limit = limit; + } + + @Override + public boolean hasNext() throws IOException { + return limit > 0; + } + + @Override + public Integer next() throws IOException { + return limit--; + } + + @Override + public String toString() { + return "CountdownRemoteIterator{" + + "limit=" + limit + + '}'; + } + } + + /** + * Iterator which counts down. + */ + private static final class CountdownIterator extends CloseCounter + implements Iterator { + + private int limit; + + private CountdownIterator(final int limit) { + this.limit = limit; + } + + @Override + public boolean hasNext() { + return limit > 0; + } + + @Override + public Integer next() { + if (!hasNext()) { + throw new NoSuchElementException("limit reached"); + } + return limit--; + } + + @Override + public String toString() { + return "CountdownIterator{" + + "limit=" + limit + + '}'; + } + } + + /** + * Iterable for countdown iterators. + * Once closed, calls to iterator() raise an exception. + */ + private static final class CountdownIterable extends CloseCounter + implements Iterable { + + private int limit; + + private CountdownIterable(final int limit) { + this.limit = limit; + } + + @Override + public Iterator iterator() { + Preconditions.checkState(getCloseCount() == 0); + + return new CountdownIterator(limit); + } + } + +} From a9acb280c77a278d4b3a0d62728d4728fdcc9dc1 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 31 Dec 2020 16:02:10 +0000 Subject: [PATCH 2/4] MAPREDUCE-7315. LocatedFileStatusFetcher to collect/publish IOStatistics. (#2579) Part of the HADOOP-16830 IOStatistics API feature. If the source FileSystem's listing RemoteIterators implement IOStatisticsSource, these are collected and served through the IOStatisticsSource API. If they are not: getIOStatistics() returns null. Only the listing statistics are collected; FileSystem.globStatus() doesn't provide any, so IO use there is not included in the aggregate results. Contributed by Steve Loughran. Change-Id: Iff1485297c2c7e181b54eaf1d2c4f80faeee7cfa --- .../mapred/LocatedFileStatusFetcher.java | 60 ++++++++++++++++++- 1 file changed, 58 insertions(+), 2 deletions(-) diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LocatedFileStatusFetcher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LocatedFileStatusFetcher.java index 74ade77a4f6c9..5810698607c41 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LocatedFileStatusFetcher.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LocatedFileStatusFetcher.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.LinkedList; import java.util.List; +import java.util.StringJoiner; import java.util.concurrent.BlockingQueue; import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; @@ -37,6 +38,9 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; @@ -52,6 +56,9 @@ import org.apache.hadoop.util.concurrent.HadoopExecutors; +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics; +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.snapshotIOStatistics; + /** * Utility class to fetch block locations for specified Input paths using a * configured number of threads. @@ -60,7 +67,7 @@ * configuration. */ @Private -public class LocatedFileStatusFetcher { +public class LocatedFileStatusFetcher implements IOStatisticsSource { public static final Logger LOG = LoggerFactory.getLogger(LocatedFileStatusFetcher.class.getName()); @@ -87,6 +94,12 @@ public class LocatedFileStatusFetcher { private volatile Throwable unknownError; + /** + * Demand created IO Statistics: only if the filesystem + * returns statistics does this fetch collect them. + */ + private IOStatisticsSnapshot iostats; + /** * Instantiate. * The newApi switch is only used to configure what exception is raised @@ -226,7 +239,46 @@ private void decrementRunningAndCheckCompletion() { lock.unlock(); } } - + + /** + * Return any IOStatistics collected during listing. + * @return IO stats accrued. + */ + @Override + public synchronized IOStatistics getIOStatistics() { + return iostats; + } + + /** + * Add the statistics of an individual thread's scan. + * @param stats possibly null statistics. + */ + private void addResultStatistics(IOStatistics stats) { + if (stats != null) { + // demand creation of IO statistics. + synchronized (this) { + LOG.debug("Adding IOStatistics: {}", stats); + if (iostats == null) { + // demand create the statistics + iostats = snapshotIOStatistics(stats); + } else { + iostats.aggregate(stats); + } + } + } + } + + @Override + public String toString() { + final IOStatistics ioStatistics = getIOStatistics(); + StringJoiner stringJoiner = new StringJoiner(", ", + LocatedFileStatusFetcher.class.getSimpleName() + "[", "]"); + if (ioStatistics != null) { + stringJoiner.add("IOStatistics=" + ioStatistics); + } + return stringJoiner.toString(); + } + /** * Retrieves block locations for the given @link {@link FileStatus}, and adds * additional paths to the process queue if required. @@ -265,6 +317,8 @@ public Result call() throws Exception { } } } + // aggregate any stats + result.stats = retrieveIOStatistics(iter); } else { result.locatedFileStatuses.add(fileStatus); } @@ -275,6 +329,7 @@ private static class Result { private List locatedFileStatuses = new LinkedList<>(); private List dirsNeedingRecursiveCalls = new LinkedList<>(); private FileSystem fs; + private IOStatistics stats; } } @@ -289,6 +344,7 @@ private class ProcessInputDirCallback implements @Override public void onSuccess(ProcessInputDirCallable.Result result) { try { + addResultStatistics(result.stats); if (!result.locatedFileStatuses.isEmpty()) { resultQueue.add(result.locatedFileStatuses); } From 6a5523298c9543f8da6077fe81387e150a0a8175 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 31 Dec 2020 21:55:39 +0000 Subject: [PATCH 3/4] HADOOP-17271. S3A connector to support IOStatistics. (#2580) S3A connector to support the IOStatistics API of HADOOP-16830, This is a major rework of the S3A Statistics collection to * Embrace the IOStatistics APIs * Move from direct references of S3AInstrumention statistics collectors to interface/implementation classes in new packages. * Ubiquitous support of IOStatistics, including: S3AFileSystem, input and output streams, RemoteIterator instances provided in list calls. * Adoption of new statistic names from hadoop-common Regarding statistic collection, as well as all existing statistics, the connector now records min/max/mean durations of HTTP GET and HEAD requests, and those of LIST operations. Contributed by Steve Loughran. Change-Id: I182d34b6ac39e017a8b4a221dad8e930882b39cf --- .../dev-support/findbugs-exclude.xml | 10 + hadoop-tools/hadoop-aws/pom.xml | 4 + .../org/apache/hadoop/fs/s3a/Constants.java | 7 + .../hadoop/fs/s3a/DefaultS3ClientFactory.java | 207 ++- .../fs/s3a/InconsistentS3ClientFactory.java | 14 +- .../org/apache/hadoop/fs/s3a/Invoker.java | 48 +- .../org/apache/hadoop/fs/s3a/Listing.java | 288 +--- .../hadoop/fs/s3a/S3ABlockOutputStream.java | 64 +- .../apache/hadoop/fs/s3a/S3ADataBlocks.java | 23 +- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 305 ++-- .../apache/hadoop/fs/s3a/S3AInputStream.java | 61 +- .../hadoop/fs/s3a/S3AInstrumentation.java | 1514 +++++++++++------ .../apache/hadoop/fs/s3a/S3AOpContext.java | 32 +- .../hadoop/fs/s3a/S3AReadOpContext.java | 10 +- .../hadoop/fs/s3a/S3AStorageStatistics.java | 99 +- .../org/apache/hadoop/fs/s3a/S3AUtils.java | 8 +- .../apache/hadoop/fs/s3a/S3ClientFactory.java | 5 +- .../org/apache/hadoop/fs/s3a/Statistic.java | 640 +++++-- .../hadoop/fs/s3a/WriteOperationHelper.java | 17 +- .../apache/hadoop/fs/s3a/WriteOperations.java | 5 +- .../auth/delegation/S3ADelegationTokens.java | 14 +- .../fs/s3a/commit/AbstractS3ACommitter.java | 61 +- .../fs/s3a/commit/CommitOperations.java | 52 +- .../fs/s3a/commit/MagicCommitIntegration.java | 10 +- .../hadoop/fs/s3a/commit/PutTracker.java | 5 +- .../fs/s3a/commit/files/PendingSet.java | 38 +- .../s3a/commit/files/SinglePendingCommit.java | 31 +- .../fs/s3a/commit/files/SuccessData.java | 31 +- .../s3a/commit/magic/MagicCommitTracker.java | 23 +- .../commit/magic/MagicS3GuardCommitter.java | 6 + .../fs/s3a/impl/ActiveOperationContext.java | 92 + .../fs/s3a/impl/BulkDeleteRetryHandler.java | 5 +- .../hadoop/fs/s3a/impl/CallableSupplier.java | 8 +- .../hadoop/fs/s3a/impl/ChangeTracker.java | 19 +- .../hadoop/fs/s3a/impl/ErrorTranslation.java | 12 + .../hadoop/fs/s3a/impl/InternalConstants.java | 7 + .../s3a/impl/ListingOperationCallbacks.java | 10 +- .../hadoop/fs/s3a/impl/NetworkBinding.java | 18 +- .../fs/s3a/impl/S3AMultipartUploader.java | 24 +- .../s3a/impl/S3AMultipartUploaderBuilder.java | 2 +- .../hadoop/fs/s3a/impl/StoreContext.java | 15 +- .../fs/s3a/impl/StoreContextBuilder.java | 6 +- .../fs/s3a/s3guard/DynamoDBMetadataStore.java | 20 +- .../fs/s3a/s3guard/InternalIterators.java | 91 - .../fs/s3a/s3guard/RetryingCollection.java | 10 +- .../fs/s3a/select/SelectInputStream.java | 12 +- .../BlockOutputStreamStatistics.java | 137 ++ .../statistics/ChangeTrackerStatistics.java | 36 + .../s3a/statistics/CommitterStatistics.java | 66 + .../fs/s3a/statistics/CountersAndGauges.java | 72 + .../statistics/DelegationTokenStatistics.java | 28 + .../statistics/S3AInputStreamStatistics.java | 191 +++ .../S3AMultipartUploaderStatistics.java | 7 +- .../s3a/statistics/S3AStatisticInterface.java | 32 + .../s3a/statistics/S3AStatisticsContext.java | 69 + .../fs/s3a/statistics/StatisticTypeEnum.java | 46 + .../s3a/statistics/StatisticsFromAwsSdk.java | 81 + .../impl/AbstractS3AStatisticsSource.java | 111 ++ .../impl/AwsStatisticsCollector.java | 129 ++ .../impl/BondedS3AStatisticsContext.java | 234 +++ .../impl/CountingChangeTracker.java | 53 + .../impl/EmptyS3AStatisticsContext.java | 597 +++++++ .../S3AMultipartUploaderStatisticsImpl.java | 46 +- .../impl/StatisticsFromAwsSdkImpl.java | 88 + .../fs/s3a/statistics/impl/package-info.java | 29 + .../fs/s3a/statistics/package-info.java | 31 + .../hadoop/fs/s3a/tools/MarkerTool.java | 12 +- .../contract/s3a/ITestS3AContractDistCp.java | 12 +- .../hadoop/fs/s3a/AbstractS3ATestBase.java | 23 + .../fs/s3a/ITestLocatedFileStatusFetcher.java | 266 ++- .../fs/s3a/ITestS3ABlockOutputArray.java | 9 +- .../apache/hadoop/fs/s3a/ITestS3AMetrics.java | 31 +- .../hadoop/fs/s3a/ITestS3AMiscOperations.java | 8 +- .../hadoop/fs/s3a/ITestS3AUnbuffer.java | 105 +- .../hadoop/fs/s3a/ITestS3GuardEmptyDirs.java | 2 +- .../hadoop/fs/s3a/MockS3AFileSystem.java | 20 +- .../hadoop/fs/s3a/MockS3ClientFactory.java | 5 +- .../apache/hadoop/fs/s3a/S3ATestUtils.java | 11 +- .../org/apache/hadoop/fs/s3a/TestListing.java | 54 +- .../fs/s3a/TestS3ABlockOutputStream.java | 8 +- .../fs/s3a/TestStreamChangeTracker.java | 7 +- .../hadoop/fs/s3a/auth/ITestAssumeRole.java | 13 +- .../s3a/auth/ITestRestrictedReadAccess.java | 2 +- .../hadoop/fs/s3a/auth/ProgressCounter.java | 8 +- .../ITestSessionDelegationInFileystem.java | 17 +- .../fs/s3a/commit/AbstractCommitITest.java | 9 +- .../s3a/commit/AbstractITCommitProtocol.java | 25 +- .../integration/ITestS3ACommitterMRJob.java | 1 - .../staging/MockedStagingCommitter.java | 4 +- .../s3a/impl/ITestPartialRenamesDeletes.java | 11 +- .../fs/s3a/impl/TestNetworkBinding.java | 105 ++ .../hadoop/fs/s3a/impl/TestNeworkBinding.java | 61 - .../s3a/performance/AbstractS3ACostTest.java | 72 +- .../s3a/performance/ITestS3ADeleteCost.java | 39 +- .../s3a/performance/ITestS3ARenameCost.java | 30 +- .../fs/s3a/performance/OperationCost.java | 2 + .../performance/OperationCostValidator.java | 36 +- ...ynamoDBMetadataStoreAuthoritativeMode.java | 8 +- .../ITestDynamoDBMetadataStoreScale.java | 4 +- .../s3a/scale/AbstractSTestS3AHugeFiles.java | 35 +- .../ILoadTestS3ABulkDeleteThrottling.java | 11 - .../scale/ITestS3ADirectoryPerformance.java | 39 +- .../scale/ITestS3AInputStreamPerformance.java | 94 +- .../hadoop/fs/s3a/scale/S3AScaleTestBase.java | 3 +- .../hadoop/fs/s3a/select/ITestS3Select.java | 34 +- .../fs/s3a/select/ITestS3SelectLandsat.java | 6 +- .../fs/s3a/select/ITestS3SelectMRJob.java | 4 +- .../ITestAggregateIOStatistics.java | 100 ++ .../ITestS3AContractStreamIOStatistics.java | 90 + .../MinimalListingOperationCallbacks.java | 12 +- .../src/test/resources/log4j.properties | 8 + 111 files changed, 5829 insertions(+), 1693 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ActiveOperationContext.java delete mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/InternalIterators.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/BlockOutputStreamStatistics.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/ChangeTrackerStatistics.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/CommitterStatistics.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/CountersAndGauges.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/DelegationTokenStatistics.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AInputStreamStatistics.java rename hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/{impl => }/statistics/S3AMultipartUploaderStatistics.java (88%) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AStatisticInterface.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AStatisticsContext.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/StatisticTypeEnum.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/StatisticsFromAwsSdk.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/AbstractS3AStatisticsSource.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/AwsStatisticsCollector.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/BondedS3AStatisticsContext.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/CountingChangeTracker.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/EmptyS3AStatisticsContext.java rename hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/{impl/statistics => statistics/impl}/S3AMultipartUploaderStatisticsImpl.java (60%) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/StatisticsFromAwsSdkImpl.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/package-info.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/package-info.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNetworkBinding.java delete mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNeworkBinding.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestAggregateIOStatistics.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestS3AContractStreamIOStatistics.java diff --git a/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml b/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml index 2bdd63d9bfa9e..9d598be856a16 100644 --- a/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml +++ b/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml @@ -80,4 +80,14 @@ + + + + + + diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml index 3e4e37e0c20a4..8dea1a384c78d 100644 --- a/hadoop-tools/hadoop-aws/pom.xml +++ b/hadoop-tools/hadoop-aws/pom.xml @@ -200,6 +200,8 @@ **/ITestMarkerToolRootOperations.java **/ITestS3GuardDDBRootOperations.java + + **/ITestAggregateIOStatistics.java @@ -249,6 +251,8 @@ **/ITestS3AContractRootDir.java **/ITestS3GuardDDBRootOperations.java + + **/ITestAggregateIOStatistics.java diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java index a1c1d969a8258..fcaec509290c8 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java @@ -549,6 +549,13 @@ private Constants() { @InterfaceStability.Unstable public static final String INPUT_FADV_RANDOM = "random"; + /** + * Gauge name for the input policy : {@value}. + * This references an enum currently exclusive to the S3A stream. + */ + public static final String STREAM_READ_GAUGE_INPUT_POLICY = + "stream_read_gauge_input_policy"; + @InterfaceAudience.Private @InterfaceStability.Unstable public static final String S3_CLIENT_FACTORY_IMPL = diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java index 276961bf8b7e1..96d16e8b1b8e0 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java @@ -23,16 +23,26 @@ import com.amazonaws.ClientConfiguration; import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.client.builder.AwsClientBuilder; +import com.amazonaws.metrics.RequestMetricCollector; import com.amazonaws.services.s3.AmazonS3; import com.amazonaws.services.s3.AmazonS3Client; +import com.amazonaws.services.s3.AmazonS3ClientBuilder; import com.amazonaws.services.s3.S3ClientOptions; +import com.amazonaws.services.s3.internal.ServiceUtils; +import com.amazonaws.util.AwsHostNameUtils; +import com.amazonaws.util.RuntimeHttpUtils; +import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; +import org.apache.hadoop.fs.s3a.statistics.StatisticsFromAwsSdk; +import org.apache.hadoop.fs.s3a.statistics.impl.AwsStatisticsCollector; import static org.apache.hadoop.fs.s3a.Constants.EXPERIMENTAL_AWS_INTERNAL_THROTTLING; import static org.apache.hadoop.fs.s3a.Constants.ENDPOINT; @@ -41,7 +51,7 @@ /** * The default {@link S3ClientFactory} implementation. - * This which calls the AWS SDK to configure and create an + * This calls the AWS SDK to configure and create an * {@link AmazonS3Client} that communicates with the S3 service. */ @InterfaceAudience.Private @@ -49,13 +59,30 @@ public class DefaultS3ClientFactory extends Configured implements S3ClientFactory { - protected static final Logger LOG = S3AFileSystem.LOG; + private static final String S3_SERVICE_NAME = "s3"; + private static final String S3_SIGNER = "S3SignerType"; + private static final String S3_V4_SIGNER = "AWSS3V4SignerType"; + /** + * Subclasses refer to this. + */ + protected static final Logger LOG = + LoggerFactory.getLogger(DefaultS3ClientFactory.class); + + /** + * Create the client. + *

+ * 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,124 @@ 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 + */ + @SuppressWarnings("deprecation") + 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 endPoint s3 endpoint, may be empty + * @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 +226,7 @@ private static AmazonS3 configureAmazonS3Client(AmazonS3 s3, throw new IllegalArgumentException(msg, e); } } - return applyS3ClientOptions(s3, conf); + return applyS3ClientOptions(s3, pathStyleAccess); } /** @@ -119,23 +234,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 +259,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..ddc492235dba8 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 metric collector + * @param endpoint AWS endpoint + * @param pathStyleAccess should path style access be supported? * @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/Invoker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java index a79737df3763d..19cd6c985b531 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java @@ -30,17 +30,19 @@ import org.slf4j.LoggerFactory; import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.io.retry.RetryPolicy; import org.apache.hadoop.util.DurationInfo; +import org.apache.hadoop.util.functional.CallableRaisingIOE; /** * Class to provide lambda expression invocation of AWS operations. * * The core retry logic is in - * {@link #retryUntranslated(String, boolean, Retried, Operation)}; + * {@link #retryUntranslated(String, boolean, Retried, CallableRaisingIOE)}; * the other {@code retry() and retryUntranslated()} calls are wrappers. * - * The static {@link #once(String, String, Operation)} and + * The static {@link #once(String, String, CallableRaisingIOE)} and * {@link #once(String, String, VoidOperation)} calls take an operation and * return it with AWS exceptions translated to IOEs of some form. * @@ -56,11 +58,13 @@ * These callbacks can be used for reporting and incrementing statistics. * * The static {@link #quietly(String, String, VoidOperation)} and - * {@link #quietlyEval(String, String, Operation)} calls exist to take any - * operation and quietly catch and log at debug. The return value of - * {@link #quietlyEval(String, String, Operation)} is a java 8 optional, + * {@link #quietlyEval(String, String, CallableRaisingIOE)} calls exist to + * take any operation and quietly catch and log at debug. + * The return value of {@link #quietlyEval(String, String, CallableRaisingIOE)} + * is a java 8 optional, * which can then be used in java8-expressions. */ +@InterfaceAudience.Private public class Invoker { private static final Logger LOG = LoggerFactory.getLogger(Invoker.class); @@ -104,10 +108,11 @@ public Retried getRetryCallback() { * @throws IOException any IOE raised, or translated exception */ @Retries.OnceTranslated - public static T once(String action, String path, Operation operation) + public static T once(String action, String path, + CallableRaisingIOE operation) throws IOException { try (DurationInfo ignored = new DurationInfo(LOG, false, "%s", action)) { - return operation.execute(); + return operation.apply(); } catch (AmazonClientException e) { throw S3AUtils.translateException(action, path, e); } @@ -143,7 +148,7 @@ public static void ignoreIOExceptions( Logger log, String action, String path, - Operation operation) { + CallableRaisingIOE operation) { try { once(action, path, operation); } catch (IOException e) { @@ -280,7 +285,7 @@ public void maybeRetry( public T retry(String action, @Nullable String path, boolean idempotent, - Operation operation) + CallableRaisingIOE operation) throws IOException { return retry(action, path, idempotent, retryCallback, operation); @@ -288,7 +293,7 @@ public T retry(String action, /** * Execute a function with retry processing. - * Uses {@link #once(String, String, Operation)} as the inner + * Uses {@link #once(String, String, CallableRaisingIOE)} as the inner * invocation mechanism before retry logic is performed. * @param type of return value * @param action action to execute (used in error messages) @@ -306,7 +311,7 @@ public T retry( @Nullable String path, boolean idempotent, Retried retrying, - Operation operation) + CallableRaisingIOE operation) throws IOException { return retryUntranslated( toDescription(action, path), @@ -317,7 +322,7 @@ public T retry( /** * Execute a function with retry processing when doRetry=true, else just once. - * Uses {@link #once(String, String, Operation)} as the inner + * Uses {@link #once(String, String, CallableRaisingIOE)} as the inner * invocation mechanism before retry logic is performed. * @param type of return value * @param doRetry true if retries should be performed @@ -337,7 +342,7 @@ public T maybeRetry( @Nullable String path, boolean idempotent, Retried retrying, - Operation operation) + CallableRaisingIOE operation) throws IOException { if (doRetry) { return retryUntranslated( @@ -366,7 +371,7 @@ public T maybeRetry( public T retryUntranslated( String text, boolean idempotent, - Operation operation) throws IOException { + CallableRaisingIOE operation) throws IOException { return retryUntranslated(text, idempotent, retryCallback, operation); } @@ -391,7 +396,7 @@ public T retryUntranslated( String text, boolean idempotent, Retried retrying, - Operation operation) throws IOException { + CallableRaisingIOE operation) throws IOException { Preconditions.checkArgument(retrying != null, "null retrying argument"); int retryCount = 0; @@ -404,7 +409,7 @@ public T retryUntranslated( LOG.debug("retry #{}", retryCount); } // execute the operation, returning if successful - return operation.execute(); + return operation.apply(); } catch (IOException | SdkBaseException e) { caught = e; } @@ -490,7 +495,7 @@ public static void quietly(String action, */ public static Optional quietlyEval(String action, String path, - Operation operation) { + CallableRaisingIOE operation) { try { return Optional.of(once(action, path, operation)); } catch (Exception e) { @@ -510,15 +515,6 @@ private static String toDescription(String action, @Nullable String path) { (StringUtils.isNotEmpty(path) ? (" on " + path) : ""); } - /** - * Arbitrary operation throwing an IOException. - * @param return type - */ - @FunctionalInterface - public interface Operation { - T execute() throws IOException; - } - /** * Void operation which may raise an IOException. */ diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java index 8b35edaf59bba..3cb3d5d832df3 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java @@ -38,8 +38,11 @@ import org.apache.hadoop.fs.s3a.s3guard.MetadataStoreListFilesIterator; import org.apache.hadoop.fs.s3a.s3guard.PathMetadata; import org.apache.hadoop.fs.s3a.s3guard.S3Guard; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; +import org.apache.hadoop.util.functional.RemoteIterators; -import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.slf4j.Logger; import java.io.FileNotFoundException; @@ -48,7 +51,6 @@ import java.time.OffsetDateTime; import java.time.ZoneOffset; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -57,6 +59,7 @@ import java.util.NoSuchElementException; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.StringJoiner; import static org.apache.hadoop.fs.impl.FutureIOSupport.awaitFuture; import static org.apache.hadoop.fs.s3a.Constants.S3N_FOLDER_SUFFIX; @@ -67,6 +70,12 @@ import static org.apache.hadoop.fs.s3a.S3AUtils.stringify; import static org.apache.hadoop.fs.s3a.S3AUtils.translateException; import static org.apache.hadoop.fs.s3a.auth.RoleModel.pathToKey; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_CONTINUE_LIST_REQUEST; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_LIST_REQUEST; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore; +import static org.apache.hadoop.util.functional.RemoteIterators.filteringRemoteIterator; +import static org.apache.hadoop.util.functional.RemoteIterators.remoteIteratorFromArray; +import static org.apache.hadoop.util.functional.RemoteIterators.remoteIteratorFromSingleton; /** * Place for the S3A listing classes; keeps all the small classes under control. @@ -96,11 +105,14 @@ public Listing(ListingOperationCallbacks listingOperationCallbacks, * @param acceptor the file status acceptor * @return the file status iterator */ - ProvidedFileStatusIterator createProvidedFileStatusIterator( + RemoteIterator createProvidedFileStatusIterator( S3AFileStatus[] fileStatuses, PathFilter filter, FileStatusAcceptor acceptor) { - return new ProvidedFileStatusIterator(fileStatuses, filter, acceptor); + return filteringRemoteIterator( + remoteIteratorFromArray(fileStatuses), + status -> + filter.accept(status.getPath()) && acceptor.accept(status)); } /** @@ -109,11 +121,11 @@ ProvidedFileStatusIterator createProvidedFileStatusIterator( * @return the file status iterator. */ @VisibleForTesting - public static ProvidedFileStatusIterator toProvidedFileStatusIterator( + public static RemoteIterator toProvidedFileStatusIterator( S3AFileStatus[] fileStatuses) { - return new ProvidedFileStatusIterator(fileStatuses, - ACCEPT_ALL, - Listing.ACCEPT_ALL_BUT_S3N); + return filteringRemoteIterator( + remoteIteratorFromArray(fileStatuses), + Listing.ACCEPT_ALL_BUT_S3N::accept); } /** @@ -185,9 +197,11 @@ public ObjectListingIterator createObjectListingIterator( * @return a new remote iterator */ @VisibleForTesting - public LocatedFileStatusIterator createLocatedFileStatusIterator( + public RemoteIterator createLocatedFileStatusIterator( RemoteIterator statusIterator) { - return new LocatedFileStatusIterator(statusIterator); + return RemoteIterators.mappingRemoteIterator( + statusIterator, + listingOperationCallbacks::toLocatedFileStatus); } /** @@ -199,11 +213,28 @@ public LocatedFileStatusIterator createLocatedFileStatusIterator( * @return a new remote iterator. */ @VisibleForTesting - TombstoneReconcilingIterator createTombstoneReconcilingIterator( - RemoteIterator iterator, Set tombstones) { - return new TombstoneReconcilingIterator(iterator, tombstones); + RemoteIterator createTombstoneReconcilingIterator( + RemoteIterator iterator, + @Nullable Set tombstones) { + if (tombstones == null || tombstones.isEmpty()) { + // no need to filter. + return iterator; + } else { + return filteringRemoteIterator( + iterator, + candidate -> !tombstones.contains(candidate.getPath())); + } } + /** + * Create a remote iterator from a single status entry. + * @param status status + * @return iterator. + */ + public RemoteIterator createSingleStatusIterator( + S3ALocatedFileStatus status) { + return remoteIteratorFromSingleton(status); + } /** * List files under a path assuming the path to be a directory. @@ -369,7 +400,7 @@ public RemoteIterator getLocatedFileStatusIteratorForDir( allowAuthoritative); // In auth mode return directly with auth flag. if (allowAuthoritative && dirMeta != null && dirMeta.isAuthoritative()) { - ProvidedFileStatusIterator mfsItr = createProvidedFileStatusIterator( + RemoteIterator mfsItr = createProvidedFileStatusIterator( S3Guard.dirMetaToStatuses(dirMeta), ACCEPT_ALL, Listing.ACCEPT_ALL_BUT_S3N); @@ -429,105 +460,6 @@ interface FileStatusAcceptor { boolean accept(FileStatus status); } - /** - * A remote iterator which only iterates over a single `LocatedFileStatus` - * value. - * - * If the status value is null, the iterator declares that it has no data. - * This iterator is used to handle {@link S3AFileSystem#listStatus(Path)} - * calls where the path handed in refers to a file, not a directory: - * this is the iterator returned. - */ - static final class SingleStatusRemoteIterator - implements RemoteIterator { - - /** - * The status to return; set to null after the first iteration. - */ - private S3ALocatedFileStatus status; - - /** - * Constructor. - * @param status status value: may be null, in which case - * the iterator is empty. - */ - SingleStatusRemoteIterator(S3ALocatedFileStatus status) { - this.status = status; - } - - /** - * {@inheritDoc} - * @return true if there is a file status to return: this is always false - * for the second iteration, and may be false for the first. - * @throws IOException never - */ - @Override - public boolean hasNext() throws IOException { - return status != null; - } - - /** - * {@inheritDoc} - * @return the non-null status element passed in when the instance was - * constructed, if it ha not already been retrieved. - * @throws IOException never - * @throws NoSuchElementException if this is the second call, or it is - * the first call and a null {@link LocatedFileStatus} entry was passed - * to the constructor. - */ - @Override - public S3ALocatedFileStatus next() throws IOException { - if (hasNext()) { - S3ALocatedFileStatus s = this.status; - status = null; - return s; - } else { - throw new NoSuchElementException(); - } - } - } - - /** - * This wraps up a provided non-null list of file status as a remote iterator. - * - * It firstly filters the provided list and later {@link #next} call will get - * from the filtered list. This suffers from scalability issues if the - * provided list is too large. - * - * There is no remote data to fetch. - */ - static class ProvidedFileStatusIterator - implements RemoteIterator { - private final ArrayList filteredStatusList; - private int index = 0; - - ProvidedFileStatusIterator(S3AFileStatus[] fileStatuses, PathFilter filter, - FileStatusAcceptor acceptor) { - Preconditions.checkArgument(fileStatuses != null, "Null status list!"); - - filteredStatusList = new ArrayList<>(fileStatuses.length); - for (S3AFileStatus status : fileStatuses) { - if (filter.accept(status.getPath()) && acceptor.accept(status)) { - filteredStatusList.add(status); - } - } - filteredStatusList.trimToSize(); - } - - @Override - public boolean hasNext() throws IOException { - return index < filteredStatusList.size(); - } - - @Override - public S3AFileStatus next() throws IOException { - if (!hasNext()) { - throw new NoSuchElementException(); - } - return filteredStatusList.get(index++); - } - } - /** * Wraps up object listing into a remote iterator which will ask for more * listing data if needed. @@ -555,7 +487,7 @@ public S3AFileStatus next() throws IOException { * Thread safety: None. */ class FileStatusListingIterator - implements RemoteIterator { + implements RemoteIterator, IOStatisticsSource { /** Source of objects. */ private final ObjectListingIterator source; @@ -758,6 +690,23 @@ private boolean buildNextStatusBatch(S3ListResult objects) { public int getBatchSize() { return batchSize; } + + /** + * Return any IOStatistics provided by the underlying stream. + * @return IO stats from the inner stream. + */ + @Override + public IOStatistics getIOStatistics() { + return source.getIOStatistics(); + } + + @Override + public String toString() { + return new StringJoiner(", ", + FileStatusListingIterator.class.getSimpleName() + "[", "]") + .add(source.toString()) + .toString(); + } } /** @@ -780,7 +729,8 @@ public int getBatchSize() { * * Thread safety: none. */ - class ObjectListingIterator implements RemoteIterator { + class ObjectListingIterator implements RemoteIterator, + IOStatisticsSource { /** The path listed. */ private final Path listPath; @@ -805,6 +755,8 @@ class ObjectListingIterator implements RemoteIterator { */ private int maxKeys; + private final IOStatisticsStore iostats; + /** * Future to store current batch listing result. */ @@ -828,10 +780,14 @@ class ObjectListingIterator implements RemoteIterator { S3ListRequest request) throws IOException { this.listPath = listPath; this.maxKeys = listingOperationCallbacks.getMaxKeys(); - this.s3ListResultFuture = listingOperationCallbacks - .listObjectsAsync(request); this.request = request; this.objectsPrev = null; + this.iostats = iostatisticsStore() + .withDurationTracking(OBJECT_LIST_REQUEST) + .withDurationTracking(OBJECT_CONTINUE_LIST_REQUEST) + .build(); + this.s3ListResultFuture = listingOperationCallbacks + .listObjectsAsync(request, iostats); } /** @@ -895,7 +851,7 @@ private void fetchNextBatchAsyncIfPresent() throws IOException { LOG.debug("[{}], Requesting next {} objects under {}", listingCount, maxKeys, listPath); s3ListResultFuture = listingOperationCallbacks - .continueListObjectsAsync(request, objects); + .continueListObjectsAsync(request, objects, iostats); } } @@ -903,7 +859,13 @@ private void fetchNextBatchAsyncIfPresent() throws IOException { public String toString() { return "Object listing iterator against " + listPath + "; listing count "+ listingCount - + "; isTruncated=" + objects.isTruncated(); + + "; isTruncated=" + objects.isTruncated() + + "; " + iostats; + } + + @Override + public IOStatistics getIOStatistics() { + return iostats; } /** @@ -966,89 +928,6 @@ public boolean accept(FileStatus status) { } } - /** - * Take a remote iterator over a set of {@link FileStatus} instances and - * return a remote iterator of {@link LocatedFileStatus} instances. - */ - class LocatedFileStatusIterator - implements RemoteIterator { - private final RemoteIterator statusIterator; - - /** - * Constructor. - * @param statusIterator an iterator over the remote status entries - */ - LocatedFileStatusIterator(RemoteIterator statusIterator) { - this.statusIterator = statusIterator; - } - - @Override - public boolean hasNext() throws IOException { - return statusIterator.hasNext(); - } - - @Override - public S3ALocatedFileStatus next() throws IOException { - return listingOperationCallbacks - .toLocatedFileStatus(statusIterator.next()); - } - } - - /** - * Wraps another iterator and filters out files that appear in the provided - * set of tombstones. Will read ahead in the iterator when necessary to - * ensure that emptiness is detected early enough if only deleted objects - * remain in the source iterator. - */ - static class TombstoneReconcilingIterator implements - RemoteIterator { - private S3ALocatedFileStatus next = null; - private final RemoteIterator iterator; - private final Set tombstones; - - /** - * @param iterator Source iterator to filter - * @param tombstones set of tombstone markers to filter out of results - */ - TombstoneReconcilingIterator(RemoteIterator - iterator, Set tombstones) { - this.iterator = iterator; - if (tombstones != null) { - this.tombstones = tombstones; - } else { - this.tombstones = Collections.emptySet(); - } - } - - private boolean fetch() throws IOException { - while (next == null && iterator.hasNext()) { - S3ALocatedFileStatus candidate = iterator.next(); - if (!tombstones.contains(candidate.getPath())) { - next = candidate; - return true; - } - } - return false; - } - - public boolean hasNext() throws IOException { - if (next != null) { - return true; - } - return fetch(); - } - - public S3ALocatedFileStatus next() throws IOException { - if (hasNext()) { - S3ALocatedFileStatus result = next; - next = null; - fetch(); - return result; - } - throw new NoSuchElementException(); - } - } - /** * Accept all entries except those which map to S3N pseudo directory markers. */ @@ -1117,4 +996,9 @@ public boolean accept(FileStatus status) { } } + public static RemoteIterator toLocatedFileStatusIterator( + RemoteIterator iterator) { + return (RemoteIterator < LocatedFileStatus >) iterator; + } + } 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 b17412f211d08..0fdad2150b6c7 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 @@ -20,6 +20,8 @@ import java.io.IOException; import java.io.OutputStream; +import java.time.Duration; +import java.time.Instant; import java.util.ArrayList; import java.util.List; import java.util.Locale; @@ -49,10 +51,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.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 +74,7 @@ @InterfaceAudience.Private @InterfaceStability.Unstable class S3ABlockOutputStream extends OutputStream implements - StreamCapabilities { + StreamCapabilities, IOStatisticsSource { private static final Logger LOG = LoggerFactory.getLogger(S3ABlockOutputStream.class); @@ -81,6 +88,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 +119,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 +156,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 +165,10 @@ class S3ABlockOutputStream extends OutputStream implements this.blockFactory = blockFactory; this.blockSize = (int) blockSize; this.statistics = statistics; + // test instantiations may not provide statistics; + this.iostatistics = statistics != null + ? statistics.getIOStatistics() + : emptyStatistics(); this.writeOperationHelper = writeOperationHelper; this.putTracker = putTracker; Preconditions.checkArgument(blockSize >= Constants.MULTIPART_MIN_SIZE, @@ -282,6 +296,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(); @@ -382,7 +397,8 @@ public void close() throws IOException { // then complete the operation if (putTracker.aboutToComplete(multiPartUpload.getUploadId(), partETags, - bytes)) { + bytes, + iostatistics)) { multiPartUpload.complete(partETags); } else { LOG.info("File {} will be visible when the job is committed", key); @@ -432,10 +448,9 @@ private int putObject() throws IOException { writeOperationHelper.createPutObjectRequest(key, uploadData.getFile()) : writeOperationHelper.createPutObjectRequest(key, uploadData.getUploadStream(), size); - long transferQueueTime = now(); BlockUploadProgress callback = new BlockUploadProgress( - block, progressListener, transferQueueTime); + block, progressListener, now()); putObjectRequest.setGeneralProgressListener(callback); statistics.blockUploadQueued(size); ListenableFuture putObjectResult = @@ -473,6 +488,8 @@ public String toString() { if (block != null) { sb.append(", activeBlock=").append(block); } + sb.append(" Statistics=") + .append(IOStatisticsLogging.ioStatisticsSourceToString(this)); sb.append('}'); return sb.toString(); } @@ -485,15 +502,15 @@ private void incrementWriteOperations() { * Current time in milliseconds. * @return time */ - private long now() { - return System.currentTimeMillis(); + private Instant now() { + return Instant.now(); } /** * Get the statistics for this stream. * @return stream statistics */ - S3AInstrumentation.OutputStreamStatistics getStatistics() { + BlockOutputStreamStatistics getStatistics() { return statistics; } @@ -520,11 +537,20 @@ public boolean hasCapability(String capability) { case StreamCapabilities.HSYNC: return false; + // yes, we do statistics. + case StreamCapabilities.IOSTATISTICS: + return true; + default: return false; } } + @Override + public IOStatistics getIOStatistics() { + return iostatistics; + } + /** * Multiple partition upload. */ @@ -636,10 +662,9 @@ private void uploadBlockAsync(final S3ADataBlocks.DataBlock block) noteUploadFailure(e); throw e; } - long transferQueueTime = now(); BlockUploadProgress callback = new BlockUploadProgress( - block, progressListener, transferQueueTime); + block, progressListener, now()); request.setGeneralProgressListener(callback); statistics.blockUploadQueued(block.dataSize()); ListenableFuture partETagFuture = @@ -754,8 +779,8 @@ public void abort() { private final class BlockUploadProgress implements ProgressListener { private final S3ADataBlocks.DataBlock block; private final ProgressListener nextListener; - private final long transferQueueTime; - private long transferStartTime; + private final Instant transferQueueTime; + private Instant transferStartTime; /** * Track the progress of a single block upload. @@ -766,7 +791,7 @@ private final class BlockUploadProgress implements ProgressListener { */ private BlockUploadProgress(S3ADataBlocks.DataBlock block, ProgressListener nextListener, - long transferQueueTime) { + Instant transferQueueTime) { this.block = block; this.transferQueueTime = transferQueueTime; this.nextListener = nextListener; @@ -787,17 +812,22 @@ public void progressChanged(ProgressEvent progressEvent) { case TRANSFER_PART_STARTED_EVENT: transferStartTime = now(); - statistics.blockUploadStarted(transferStartTime - transferQueueTime, + statistics.blockUploadStarted( + Duration.between(transferQueueTime, transferStartTime), size); incrementWriteOperations(); break; case TRANSFER_PART_COMPLETED_EVENT: - statistics.blockUploadCompleted(now() - transferStartTime, size); + statistics.blockUploadCompleted( + Duration.between(transferStartTime, now()), + size); break; case TRANSFER_PART_FAILED_EVENT: - statistics.blockUploadFailed(now() - transferStartTime, size); + statistics.blockUploadFailed( + Duration.between(transferStartTime, now()), + size); LOG.warn("Transfer failure of block {}", block); break; 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 fa38f246dd4ec..250317706900c 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.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 beb4b975236e7..4e1a67f86f9fb 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 @@ -117,11 +117,14 @@ import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum; import org.apache.hadoop.fs.s3a.impl.StoreContext; import org.apache.hadoop.fs.s3a.impl.StoreContextBuilder; -import org.apache.hadoop.fs.s3a.impl.statistics.S3AMultipartUploaderStatisticsImpl; import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; import org.apache.hadoop.fs.s3a.select.InternalSelectConstants; import org.apache.hadoop.fs.s3a.tools.MarkerToolOperations; import org.apache.hadoop.fs.s3a.tools.MarkerToolOperationsImpl; +import org.apache.hadoop.fs.statistics.DurationTracker; +import org.apache.hadoop.fs.statistics.DurationTrackerFactory; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.Text; import org.apache.hadoop.security.token.DelegationTokenIssuer; @@ -159,6 +162,11 @@ import org.apache.hadoop.fs.s3a.s3guard.PathMetadata; import org.apache.hadoop.fs.s3a.s3guard.S3Guard; import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider; +import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics; +import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics; +import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext; +import org.apache.hadoop.fs.s3a.statistics.StatisticsFromAwsSdk; +import org.apache.hadoop.fs.s3a.statistics.impl.BondedS3AStatisticsContext; import org.apache.hadoop.fs.s3native.S3xLoginHelper; import org.apache.hadoop.io.retry.RetryPolicies; import org.apache.hadoop.fs.store.EtagChecksum; @@ -170,10 +178,12 @@ import org.apache.hadoop.util.SemaphoredDelegatingExecutor; import org.apache.hadoop.util.concurrent.HadoopExecutors; +import static java.util.Objects.requireNonNull; import static org.apache.hadoop.fs.impl.AbstractFSBuilderImpl.rejectUnknownMandatoryKeys; import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs; import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.Invoker.*; +import static org.apache.hadoop.fs.s3a.Listing.toLocatedFileStatusIterator; import static org.apache.hadoop.fs.s3a.S3AUtils.*; import static org.apache.hadoop.fs.s3a.Statistic.*; import static org.apache.commons.lang3.StringUtils.isNotEmpty; @@ -185,12 +195,20 @@ import static org.apache.hadoop.fs.s3a.commit.CommitConstants.FS_S3A_COMMITTER_STAGING_ABORT_PENDING_UPLOADS; 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.isObjectNotFound; 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; import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.dirMetaToStatuses; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_CONTINUE_LIST_REQUEST; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_LIST_REQUEST; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.pairedTrackerFactory; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfInvocation; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfOperation; import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; +import static org.apache.hadoop.util.functional.RemoteIterators.typeCastingRemoteIterator; /** * The core S3A Filesystem implementation. @@ -208,7 +226,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. */ @@ -260,9 +278,13 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, * is no encryption. */ private EncryptionSecrets encryptionSecrets = new EncryptionSecrets(); + /** The core instrumentation. */ private S3AInstrumentation instrumentation; - private final S3AStorageStatistics storageStatistics = - createStorageStatistics(); + /** Accessors to statistics for this FS. */ + private S3AStatisticsContext statisticsContext; + /** Storage Statistics Bonded to the instrumentation. */ + private S3AStorageStatistics storageStatistics; + private long readAhead; private S3AInputPolicy inputPolicy; private ChangeDetectionPolicy changeDetectionPolicy; @@ -369,6 +391,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(); @@ -378,7 +401,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); @@ -532,6 +556,33 @@ 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() { + storageStatistics = createStorageStatistics( + requireNonNull(getIOStatistics())); + 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 @@ -566,13 +617,15 @@ private void initThreadPools(Configuration conf) { /** * Create the storage statistics or bind to an existing one. - * @return a storage statistics instance. + * @param ioStatistics IOStatistics to build the storage statistics from. + * @return a storage statistics instance; expected to be that of the FS. */ - protected static S3AStorageStatistics createStorageStatistics() { + protected static S3AStorageStatistics createStorageStatistics( + final IOStatistics ioStatistics) { return (S3AStorageStatistics) GlobalStorageStatistics.INSTANCE .put(S3AStorageStatistics.NAME, - () -> new S3AStorageStatistics()); + () -> new S3AStorageStatistics(ioStatistics)); } /** @@ -611,6 +664,7 @@ protected void verifyBucketExistsV2() * Get S3A Instrumentation. For test purposes. * @return this instance's instrumentation. */ + @VisibleForTesting public S3AInstrumentation getInstrumentation() { return instrumentation; } @@ -678,8 +732,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: HADOOP-16830 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); } /** @@ -1177,7 +1239,7 @@ private S3AReadOpContext createReadContext( invoker, s3guardInvoker, statistics, - instrumentation, + statisticsContext, fileStatus, seekPolicy, changePolicy, @@ -1274,15 +1336,19 @@ public FSDataOutputStream create(Path f, FsPermission permission, PutTracker putTracker = committerIntegration.createTracker(path, key); String destKey = putTracker.getDestKey(); + final BlockOutputStreamStatistics outputStreamStatistics + = statisticsContext.newOutputStreamStatistics(); return new FSDataOutputStream( new S3ABlockOutputStream(this, destKey, - new SemaphoredDelegatingExecutor(boundedThreadPool, - blockOutputActiveBlocks, true), + new SemaphoredDelegatingExecutor( + boundedThreadPool, + blockOutputActiveBlocks, + true), progress, partSize, blockFactory, - instrumentation.newOutputStreamStatistics(statistics), + outputStreamStatistics, getWriteOperationHelper(), putTracker), null); @@ -1656,20 +1722,26 @@ protected class ListingOperationCallbacksImpl implements @Override @Retries.RetryRaw public CompletableFuture listObjectsAsync( - S3ListRequest request) + S3ListRequest request, + DurationTrackerFactory trackerFactory) throws IOException { - return submit(unboundedThreadPool, - () -> listObjects(request)); + return submit(unboundedThreadPool, () -> + listObjects(request, + pairedTrackerFactory(trackerFactory, + getDurationTrackerFactory()))); } @Override @Retries.RetryRaw public CompletableFuture continueListObjectsAsync( - S3ListRequest request, - S3ListResult prevResult) + S3ListRequest request, + S3ListResult prevResult, + DurationTrackerFactory trackerFactory) throws IOException { return submit(unboundedThreadPool, - () -> continueListObjects(request, prevResult)); + () -> continueListObjects(request, prevResult, + pairedTrackerFactory(trackerFactory, + getDurationTrackerFactory()))); } @Override @@ -1818,8 +1890,7 @@ protected void incrementStatistic(Statistic statistic) { * @param count the count to increment */ protected void incrementStatistic(Statistic statistic, long count) { - instrumentation.incrementCounter(statistic, count); - storageStatistics.incrementCounter(statistic, count); + statisticsContext.incrementCounter(statistic, count); } /** @@ -1828,7 +1899,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); } /** @@ -1837,7 +1908,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); } /** @@ -1850,6 +1921,7 @@ public void operationRetried(Exception ex) { if (isThrottleException(ex)) { operationThrottled(false); } else { + incrementStatistic(STORE_IO_RETRY); incrementStatistic(IGNORED_ERRORS); } } @@ -1901,11 +1973,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); } } @@ -1918,6 +1990,27 @@ public S3AStorageStatistics getStorageStatistics() { return storageStatistics; } + /** + * Get the instrumentation's IOStatistics. + * @return statistics + */ + @Override + public IOStatistics getIOStatistics() { + return instrumentation != null + ? instrumentation.getIOStatistics() + : null; + } + + /** + * Get the factory for duration tracking. + * @return a factory from the instrumentation. + */ + protected DurationTrackerFactory getDurationTrackerFactory() { + return instrumentation != null ? + instrumentation.getDurationTrackerFactory() + : null; + } + /** * Request object metadata; increments counters in the process. * Retry policy: retry untranslated. @@ -1955,15 +2048,30 @@ protected ObjectMetadata getObjectMetadata(String key, ObjectMetadata meta = changeInvoker.retryUntranslated("GET " + key, true, () -> { incrementStatistic(OBJECT_METADATA_REQUESTS); - LOG.debug("HEAD {} with change tracker {}", key, changeTracker); - if (changeTracker != null) { - changeTracker.maybeApplyConstraint(request); - } - ObjectMetadata objectMetadata = s3.getObjectMetadata(request); - if (changeTracker != null) { - changeTracker.processMetadata(objectMetadata, operation); + DurationTracker duration = getDurationTrackerFactory() + .trackDuration(ACTION_HTTP_HEAD_REQUEST.getSymbol()); + try { + LOG.debug("HEAD {} with change tracker {}", key, changeTracker); + if (changeTracker != null) { + changeTracker.maybeApplyConstraint(request); + } + ObjectMetadata objectMetadata = s3.getObjectMetadata(request); + if (changeTracker != null) { + changeTracker.processMetadata(objectMetadata, operation); + } + return objectMetadata; + } catch(AmazonServiceException ase) { + if (!isObjectNotFound(ase)) { + // file not found is not considered a failure of the call, + // so only switch the duration tracker to update failure + // metrics on other exception outcomes. + duration.failed(); + } + throw ase; + } finally { + // update the tracker. + duration.close(); } - return objectMetadata; }); incrementReadOperations(); return meta; @@ -1975,13 +2083,15 @@ protected ObjectMetadata getObjectMetadata(String key, * * Retry policy: retry untranslated. * @param request request to initiate + * @param trackerFactory duration tracking * @return the results * @throws IOException if the retry invocation raises one (it shouldn't). */ @Retries.RetryRaw - protected S3ListResult listObjects(S3ListRequest request) throws IOException { + protected S3ListResult listObjects(S3ListRequest request, + @Nullable final DurationTrackerFactory trackerFactory) + throws IOException { incrementReadOperations(); - incrementStatistic(OBJECT_LIST_REQUESTS); LOG.debug("LIST {}", request); validateListArguments(request); try(DurationInfo ignored = @@ -1989,13 +2099,15 @@ protected S3ListResult listObjects(S3ListRequest request) throws IOException { return invoker.retryUntranslated( request.toString(), true, - () -> { - if (useListV1) { - return S3ListResult.v1(s3.listObjects(request.getV1())); - } else { - return S3ListResult.v2(s3.listObjectsV2(request.getV2())); - } - }); + trackDurationOfOperation(trackerFactory, + OBJECT_LIST_REQUEST, + () -> { + if (useListV1) { + return S3ListResult.v1(s3.listObjects(request.getV1())); + } else { + return S3ListResult.v2(s3.listObjectsV2(request.getV2())); + } + })); } } @@ -2016,12 +2128,14 @@ private void validateListArguments(S3ListRequest request) { * Retry policy: retry untranslated. * @param request last list objects request to continue * @param prevResult last paged result to continue from + * @param trackerFactory duration tracking * @return the next result object * @throws IOException none, just there for retryUntranslated. */ @Retries.RetryRaw protected S3ListResult continueListObjects(S3ListRequest request, - S3ListResult prevResult) throws IOException { + S3ListResult prevResult, + final DurationTrackerFactory trackerFactory) throws IOException { incrementReadOperations(); validateListArguments(request); try(DurationInfo ignored = @@ -2029,17 +2143,19 @@ protected S3ListResult continueListObjects(S3ListRequest request, return invoker.retryUntranslated( request.toString(), true, - () -> { - incrementStatistic(OBJECT_CONTINUE_LIST_REQUESTS); - if (useListV1) { - return S3ListResult.v1( - s3.listNextBatchOfObjects(prevResult.getV1())); - } else { - request.getV2().setContinuationToken(prevResult.getV2() - .getNextContinuationToken()); - return S3ListResult.v2(s3.listObjectsV2(request.getV2())); - } - }); + trackDurationOfOperation( + trackerFactory, + OBJECT_CONTINUE_LIST_REQUEST, + () -> { + if (useListV1) { + return S3ListResult.v1( + s3.listNextBatchOfObjects(prevResult.getV1())); + } else { + request.getV2().setContinuationToken(prevResult.getV2() + .getNextContinuationToken()); + return S3ListResult.v2(s3.listObjectsV2(request.getV2())); + } + })); } } @@ -2084,9 +2200,10 @@ protected void deleteObject(String key) invoker.retryUntranslated(String.format("Delete %s:/%s", bucket, key), DELETE_CONSIDERED_IDEMPOTENT, ()-> { - incrementStatistic(OBJECT_DELETE_REQUESTS); incrementStatistic(OBJECT_DELETE_OBJECTS); - s3.deleteObject(bucket, key); + trackDurationOfInvocation(getDurationTrackerFactory(), + OBJECT_DELETE_REQUEST.getSymbol(), + () -> s3.deleteObject(bucket, key)); return null; }); } @@ -2170,11 +2287,12 @@ private DeleteObjectsResult deleteObjects(DeleteObjectsRequest deleteRequest) // handle the failure retryHandler.bulkDeleteRetried(deleteRequest, e); }, - () -> { - incrementStatistic(OBJECT_DELETE_REQUESTS, 1); - incrementStatistic(OBJECT_DELETE_OBJECTS, keyCount); - return s3.deleteObjects(deleteRequest); - }); + // duration is tracked in the bulk delete counters + trackDurationOfOperation(getDurationTrackerFactory(), + OBJECT_BULK_DELETE_REQUEST.getSymbol(), () -> { + incrementStatistic(OBJECT_DELETE_OBJECTS, keyCount); + return s3.deleteObjects(deleteRequest); + })); } catch (MultiObjectDeleteException e) { // one or more of the keys could not be deleted. // log and rethrow @@ -2666,17 +2784,7 @@ public RemoteIterator listStatusIterator(Path p) throws FileNotFoundException, IOException { RemoteIterator listStatusItr = once("listStatus", p.toString(), () -> innerListStatus(p)); - return new RemoteIterator() { - @Override - public boolean hasNext() throws IOException { - return listStatusItr.hasNext(); - } - - @Override - public FileStatus next() throws IOException { - return listStatusItr.next(); - } - }; + return typeCastingRemoteIterator(listStatusItr); } /** @@ -3202,8 +3310,8 @@ S3AFileStatus s3GetFileStatus(final Path path, S3ListRequest request = createListObjectsRequest(dirKey, "/", listSize); // execute the request - S3ListResult listResult = listObjects(request); - + S3ListResult listResult = listObjects(request, + getDurationTrackerFactory()); if (listResult.hasPrefixesOrObjects(contextAccessors, tombstones)) { if (LOG.isDebugEnabled()) { @@ -3608,8 +3716,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 + ")"; @@ -4103,12 +4212,10 @@ public String toString() { sb.append(", delegation tokens=") .append(delegationTokens.map(Objects::toString).orElse("disabled")); sb.append(", ").append(directoryPolicy); - sb.append(", statistics {") - .append(statistics) - .append("}"); - if (instrumentation != null) { - sb.append(", metrics {") - .append(instrumentation.dump("{", "=", "} ", true)) + // if logging at debug, toString returns the entire IOStatistics set. + if (getInstrumentation() != null) { + sb.append(", instrumentation {") + .append(getInstrumentation().toString()) .append("}"); } sb.append('}'); @@ -4308,21 +4415,6 @@ public RemoteIterator listFiles(Path f, new Listing.AcceptFilesOnly(qualify(f)), null, true, false)); } - private static RemoteIterator toLocatedFileStatusIterator( - RemoteIterator iterator) { - return new RemoteIterator() { - @Override - public boolean hasNext() throws IOException { - return iterator.hasNext(); - } - - @Override - public LocatedFileStatus next() throws IOException { - return iterator.next(); - } - }; - } - /** * Recursive List of files and empty directories. * @param f path to list from @@ -4405,7 +4497,7 @@ private RemoteIterator innerListFiles( if (status != null && status.isFile()) { // simple case: File LOG.debug("Path is a file: {}", path); - return new Listing.SingleStatusRemoteIterator( + return listing.createSingleStatusIterator( toLocatedFileStatus(status)); } // Assuming the path to be a directory @@ -4425,7 +4517,7 @@ private RemoteIterator innerListFiles( ? status : (S3AFileStatus) getFileStatus(path); if (fileStatus.isFile()) { - return new Listing.SingleStatusRemoteIterator( + return listing.createSingleStatusIterator( toLocatedFileStatus(fileStatus)); } } @@ -4484,7 +4576,7 @@ public RemoteIterator listLocatedStatus(final Path f, if (fileStatus.isFile()) { // simple case: File LOG.debug("Path is a file"); - return new Listing.SingleStatusRemoteIterator( + return listing.createSingleStatusIterator( filter.accept(path) ? toLocatedFileStatus(fileStatus) : null); @@ -4631,8 +4723,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") @@ -4766,8 +4858,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 @@ -4914,7 +5007,7 @@ public S3AMultipartUploaderBuilder createMultipartUploader( getWriteOperationHelper(), ctx, basePath, - new S3AMultipartUploaderStatisticsImpl(ctx::incrementStatistic)); + statisticsContext.createMultipartUploaderStatistics()); } /** @@ -4934,7 +5027,7 @@ public StoreContext createStoreContext() { .setExecutor(boundedThreadPool) .setExecutorCapacity(executorCapacity) .setInvoker(invoker) - .setInstrumentation(getInstrumentation()) + .setInstrumentation(statisticsContext) .setStorageStatistics(getStorageStatistics()) .setInputPolicy(getInputPolicy()) .setChangeDetectionPolicy(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 bd8adad035d7a..c725fdf37ff8a 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,14 @@ 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.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.apache.hadoop.fs.statistics.DurationTracker; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -68,7 +72,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"; @@ -105,7 +109,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; @@ -131,6 +135,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 @@ -154,13 +163,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()); @@ -210,8 +221,21 @@ private synchronized void reopen(String reason, long targetPos, long length, String text = String.format("%s %s at %d", operation, uri, targetPos); changeTracker.maybeApplyConstraint(request); - object = Invoker.once(text, uri, - () -> client.getObject(request)); + DurationTracker tracker = streamStatistics.initiateGetRequest(); + try { + object = Invoker.once(text, uri, + () -> client.getObject(request)); + } catch(IOException e) { + // input function failed: note it + tracker.failed(); + // and rethrow + throw e; + } finally { + // update the tracker. + // this is called after any catch() call will have + // set the failed flag. + tracker.close(); + } changeTracker.processResponse(object, operation, targetPos); @@ -294,13 +318,11 @@ private void seekInStream(long targetPos, long length) throws IOException { if (skipForward) { // the forward seek range is within the limits LOG.debug("Forward seek on {}, of {} bytes", uri, diff); - streamStatistics.seekForwards(diff); long skipped = wrappedStream.skip(diff); if (skipped > 0) { pos += skipped; - // as these bytes have been read, they are included in the counter - incrementBytesRead(diff); } + streamStatistics.seekForwards(diff, skipped); if (pos == targetPos) { // all is well @@ -312,6 +334,9 @@ private void seekInStream(long targetPos, long length) throws IOException { LOG.warn("Failed to seek on {} to {}. Current position {}", uri, targetPos, pos); } + } else { + // not attempting to read any bytes from the stream + streamStatistics.seekForwards(diff, 0); } } else if (diff < 0) { // backwards seek @@ -356,7 +381,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 @@ -720,7 +745,7 @@ public String toString() { sb.append(" contentRangeFinish=").append(contentRangeFinish); sb.append(" remainingInCurrentRequest=") .append(remainingInCurrentRequest()); - sb.append(changeTracker); + sb.append(" ").append(changeTracker); sb.append('\n').append(s); sb.append('}'); return sb.toString(); @@ -774,7 +799,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; } @@ -858,13 +883,14 @@ public synchronized void unbuffer() { try { closeStream("unbuffer()", contentRangeFinish, false); } finally { - streamStatistics.merge(false); + streamStatistics.unbuffered(); } } @Override public boolean hasCapability(String capability) { switch (toLowerCase(capability)) { + case StreamCapabilities.IOSTATISTICS: case StreamCapabilities.READAHEAD: case StreamCapabilities.UNBUFFER: return true; @@ -877,4 +903,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 19f42cf9e399f..982611a098401 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,34 @@ package org.apache.hadoop.fs.s3a; +import javax.annotation.Nullable; + import org.apache.hadoop.thirdparty.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.s3guard.MetastoreInstrumentation; +import org.apache.hadoop.fs.s3a.statistics.impl.AbstractS3AStatisticsSource; +import org.apache.hadoop.fs.s3a.statistics.ChangeTrackerStatistics; +import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics; +import org.apache.hadoop.fs.s3a.statistics.CountersAndGauges; +import org.apache.hadoop.fs.s3a.statistics.impl.CountingChangeTracker; +import org.apache.hadoop.fs.s3a.statistics.DelegationTokenStatistics; +import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; +import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics; +import org.apache.hadoop.fs.s3a.statistics.StatisticTypeEnum; +import org.apache.hadoop.fs.statistics.DurationTrackerFactory; +import org.apache.hadoop.fs.statistics.IOStatisticsLogging; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; +import org.apache.hadoop.fs.statistics.StreamStatisticNames; +import org.apache.hadoop.fs.statistics.DurationTracker; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStoreBuilder; import org.apache.hadoop.metrics2.AbstractMetric; import org.apache.hadoop.metrics2.MetricStringBuilder; import org.apache.hadoop.metrics2.MetricsCollector; @@ -43,216 +63,176 @@ import java.io.Closeable; import java.net.URI; +import java.time.Duration; +import java.util.Arrays; +import java.util.EnumSet; import java.util.HashMap; +import java.util.List; 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.s3a.Constants.STREAM_READ_GAUGE_INPUT_POLICY; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.demandStringifyIOStatistics; +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.snapshotIOStatistics; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.ACTION_EXECUTOR_ACQUIRED; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.ACTION_HTTP_GET_REQUEST; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_FAILURES; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_UNBUFFERED; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore; import static org.apache.hadoop.fs.s3a.Statistic.*; /** - * Instrumentation of S3a. - * Derived from the {@code AzureFileSystemInstrumentation}. - * + * Instrumentation of S3A. + *

+ * History + *

+ *
    + *
  1. + * HADOOP-13028. Initial implementation. + * Derived from the {@code AzureFileSystemInstrumentation}. + *
  2. + *
  3. + * Broadly (and directly) used in S3A. + * The use of direct references causes "problems" in mocking tests. + *
  4. + *
  5. + * HADOOP-16830. IOStatistics. Move to an interface and implementation + * design for the different inner classes. + *
  6. + *
+ *

* 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, * the operations to increment/query metric values are designed to handle * lookup failures. + *

+ *

+ * S3AFileSystem StorageStatistics are dynamically derived from + * the IOStatistics. + *

+ *

+ * The toString() operation includes the entire IOStatistics when this + * class's log is set to DEBUG. This keeps the logs somewhat manageable + * on normal runs, but allows for more reporting. + *

*/ @InterfaceAudience.Private @InterfaceStability.Evolving -public class S3AInstrumentation implements Closeable, MetricsSource { +public class S3AInstrumentation implements Closeable, MetricsSource, + CountersAndGauges, IOStatisticsSource, DurationTrackerFactory { 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"; // metricsSystemLock must be used to synchronize modifications to // metricsSystem and the following counters. - private static Object metricsSystemLock = new Object(); + private static final Object METRICS_SYSTEM_LOCK = new Object(); private static MetricsSystem metricsSystem = null; private static int metricsSourceNameCounter = 0; private static int metricsSourceActiveCounter = 0; + private final DurationTrackerFactory durationTrackerFactory; + private String metricsSourceName; private final MetricsRegistry registry = new MetricsRegistry("s3aFileSystem").setContext(CONTEXT); - private final MutableCounterLong streamOpenOperations; - private final MutableCounterLong streamCloseOperations; - private final MutableCounterLong streamClosed; - private final MutableCounterLong streamAborted; - private final MutableCounterLong streamSeekOperations; - private final MutableCounterLong streamReadExceptions; - private final MutableCounterLong streamForwardSeekOperations; - private final MutableCounterLong streamBackwardSeekOperations; - private final MutableCounterLong streamBytesSkippedOnSeek; - private final MutableCounterLong streamBytesBackwardsOnSeek; - private final MutableCounterLong streamBytesRead; - private final MutableCounterLong streamReadOperations; - private final MutableCounterLong streamReadFullyOperations; - private final MutableCounterLong streamReadsIncomplete; - private final MutableCounterLong streamBytesReadInClose; - private final MutableCounterLong streamBytesDiscardedInAbort; - private final MutableCounterLong ignoredErrors; private final MutableQuantiles putLatencyQuantile; private final MutableQuantiles throttleRateQuantile; private final MutableQuantiles s3GuardThrottleRateQuantile; - private final MutableCounterLong numberOfFilesCreated; - private final MutableCounterLong numberOfFilesCopied; - private final MutableCounterLong bytesOfFilesCopied; - private final MutableCounterLong numberOfFilesDeleted; - private final MutableCounterLong numberOfFakeDirectoryDeletes; - private final MutableCounterLong numberOfDirectoriesCreated; - private final MutableCounterLong numberOfDirectoriesDeleted; /** Instantiate this without caring whether or not S3Guard is enabled. */ private final S3GuardInstrumentation s3GuardInstrumentation = new S3GuardInstrumentation(); - private static final Statistic[] COUNTERS_TO_CREATE = { - INVOCATION_COPY_FROM_LOCAL_FILE, - INVOCATION_CREATE, - INVOCATION_CREATE_NON_RECURSIVE, - INVOCATION_DELETE, - INVOCATION_EXISTS, - INVOCATION_GET_DELEGATION_TOKEN, - INVOCATION_GET_FILE_CHECKSUM, - INVOCATION_GET_FILE_STATUS, - INVOCATION_GLOB_STATUS, - INVOCATION_IS_DIRECTORY, - INVOCATION_IS_FILE, - INVOCATION_LIST_FILES, - INVOCATION_LIST_LOCATED_STATUS, - INVOCATION_LIST_STATUS, - INVOCATION_MKDIRS, - INVOCATION_OPEN, - INVOCATION_RENAME, - OBJECT_COPY_REQUESTS, - OBJECT_DELETE_REQUESTS, - OBJECT_DELETE_OBJECTS, - OBJECT_LIST_REQUESTS, - OBJECT_CONTINUE_LIST_REQUESTS, - OBJECT_METADATA_REQUESTS, - OBJECT_MULTIPART_UPLOAD_ABORTED, - OBJECT_PUT_BYTES, - OBJECT_PUT_REQUESTS, - OBJECT_PUT_REQUESTS_COMPLETED, - OBJECT_SELECT_REQUESTS, - STREAM_READ_VERSION_MISMATCHES, - STREAM_WRITE_FAILURES, - STREAM_WRITE_BLOCK_UPLOADS, - STREAM_WRITE_BLOCK_UPLOADS_COMMITTED, - STREAM_WRITE_BLOCK_UPLOADS_ABORTED, - STREAM_WRITE_TOTAL_TIME, - STREAM_WRITE_TOTAL_DATA, - COMMITTER_COMMITS_CREATED, - COMMITTER_COMMITS_COMPLETED, - COMMITTER_JOBS_SUCCEEDED, - COMMITTER_JOBS_FAILED, - COMMITTER_TASKS_SUCCEEDED, - COMMITTER_TASKS_FAILED, - COMMITTER_BYTES_COMMITTED, - COMMITTER_BYTES_UPLOADED, - COMMITTER_COMMITS_FAILED, - COMMITTER_COMMITS_ABORTED, - COMMITTER_COMMITS_REVERTED, - COMMITTER_MAGIC_FILES_CREATED, - S3GUARD_METADATASTORE_PUT_PATH_REQUEST, - S3GUARD_METADATASTORE_INITIALIZATION, - S3GUARD_METADATASTORE_RECORD_DELETES, - S3GUARD_METADATASTORE_RECORD_READS, - S3GUARD_METADATASTORE_RECORD_WRITES, - S3GUARD_METADATASTORE_RETRY, - S3GUARD_METADATASTORE_THROTTLED, - S3GUARD_METADATASTORE_AUTHORITATIVE_DIRECTORIES_UPDATED, - STORE_IO_THROTTLED, - DELEGATION_TOKENS_ISSUED, - FILES_DELETE_REJECTED, - MULTIPART_INSTANTIATED, - MULTIPART_PART_PUT, - MULTIPART_PART_PUT_BYTES, - MULTIPART_UPLOAD_ABORTED, - MULTIPART_UPLOAD_ABORT_UNDER_PATH_INVOKED, - MULTIPART_UPLOAD_COMPLETED, - MULTIPART_UPLOAD_STARTED - }; + /** + * This is the IOStatistics store for the S3AFileSystem + * instance. + * It is not kept in sync with the rest of the S3A instrumentation. + * Most inner statistics implementation classes only update this + * store when it is pushed back, such as as in close(). + */ + private final IOStatisticsStore instanceIOStatistics; + /** + * Gauges to create. + *

+ * All statistics which are not gauges or quantiles + * are registered as counters. + */ private static final Statistic[] GAUGES_TO_CREATE = { OBJECT_PUT_REQUESTS_ACTIVE, OBJECT_PUT_BYTES_PENDING, STREAM_WRITE_BLOCK_UPLOADS_ACTIVE, STREAM_WRITE_BLOCK_UPLOADS_PENDING, - STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING, + STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING, }; + /** + * Construct the instrumentation for a filesystem. + * @param name URI of filesystem. + */ public S3AInstrumentation(URI name) { UUID fileSystemInstanceId = UUID.randomUUID(); registry.tag(METRIC_TAG_FILESYSTEM_ID, "A unique identifier for the instance", fileSystemInstanceId.toString()); registry.tag(METRIC_TAG_BUCKET, "Hostname from the FS URL", name.getHost()); - streamOpenOperations = counter(STREAM_OPENED); - streamCloseOperations = counter(STREAM_CLOSE_OPERATIONS); - streamClosed = counter(STREAM_CLOSED); - streamAborted = counter(STREAM_ABORTED); - streamSeekOperations = counter(STREAM_SEEK_OPERATIONS); - streamReadExceptions = counter(STREAM_READ_EXCEPTIONS); - streamForwardSeekOperations = - counter(STREAM_FORWARD_SEEK_OPERATIONS); - streamBackwardSeekOperations = - counter(STREAM_BACKWARD_SEEK_OPERATIONS); - streamBytesSkippedOnSeek = counter(STREAM_SEEK_BYTES_SKIPPED); - streamBytesBackwardsOnSeek = - counter(STREAM_SEEK_BYTES_BACKWARDS); - streamBytesRead = counter(STREAM_SEEK_BYTES_READ); - streamReadOperations = counter(STREAM_READ_OPERATIONS); - streamReadFullyOperations = - counter(STREAM_READ_FULLY_OPERATIONS); - streamReadsIncomplete = - counter(STREAM_READ_OPERATIONS_INCOMPLETE); - streamBytesReadInClose = counter(STREAM_CLOSE_BYTES_READ); - streamBytesDiscardedInAbort = counter(STREAM_ABORT_BYTES_DISCARDED); - numberOfFilesCreated = counter(FILES_CREATED); - numberOfFilesCopied = counter(FILES_COPIED); - bytesOfFilesCopied = counter(FILES_COPIED_BYTES); - numberOfFilesDeleted = counter(FILES_DELETED); - numberOfFakeDirectoryDeletes = counter(FAKE_DIRECTORIES_DELETED); - numberOfDirectoriesCreated = counter(DIRECTORIES_CREATED); - numberOfDirectoriesDeleted = counter(DIRECTORIES_DELETED); - ignoredErrors = counter(IGNORED_ERRORS); - for (Statistic statistic : COUNTERS_TO_CREATE) { - counter(statistic); - } - for (Statistic statistic : GAUGES_TO_CREATE) { - gauge(statistic.getSymbol(), statistic.getDescription()); - } + + // now set up the instance IOStatistics. + // create the builder + IOStatisticsStoreBuilder storeBuilder = iostatisticsStore(); + + // add the gauges + List gauges = Arrays.asList(GAUGES_TO_CREATE); + gauges.forEach(this::gauge); + + // declare all counter statistics + EnumSet.allOf(Statistic.class).stream() + .filter(statistic -> + statistic.getType() == StatisticTypeEnum.TYPE_COUNTER) + .forEach(stat -> { + counter(stat); + storeBuilder.withCounters(stat.getSymbol()); + }); + + // and durations + EnumSet.allOf(Statistic.class).stream() + .filter(statistic -> + statistic.getType() == StatisticTypeEnum.TYPE_DURATION) + .forEach(stat -> { + duration(stat); + storeBuilder.withDurationTracking(stat.getSymbol()); + }); + //todo need a config for the quantiles interval? int interval = 1; putLatencyQuantile = quantiles(S3GUARD_METADATASTORE_PUT_PATH_LATENCY, @@ -262,12 +242,21 @@ public S3AInstrumentation(URI name) { throttleRateQuantile = quantiles(STORE_IO_THROTTLE_RATE, "events", "frequency (Hz)", interval); + // register with Hadoop metrics registerAsMetricsSource(name); + + // and build the IO Statistics + instanceIOStatistics = storeBuilder.build(); + + // duration track metrics (Success/failure) and IOStatistics. + durationTrackerFactory = IOStatisticsBinding.pairedTrackerFactory( + instanceIOStatistics, + new MetricDurationTrackerFactory()); } @VisibleForTesting public MetricsSystem getMetricsSystem() { - synchronized (metricsSystemLock) { + synchronized (METRICS_SYSTEM_LOCK) { if (metricsSystem == null) { metricsSystem = new MetricsSystemImpl(); metricsSystem.init(METRICS_SYSTEM_NAME); @@ -282,7 +271,7 @@ public MetricsSystem getMetricsSystem() { */ private void registerAsMetricsSource(URI name) { int number; - synchronized(metricsSystemLock) { + synchronized(METRICS_SYSTEM_LOCK) { getMetricsSystem(); metricsSourceActiveCounter++; @@ -312,6 +301,15 @@ protected final MutableCounterLong counter(Statistic op) { return counter(op.getSymbol(), op.getDescription()); } + /** + * Registering a duration adds the success and failure counters. + * @param op statistic to track + */ + protected final void duration(Statistic op) { + counter(op.getSymbol(), op.getDescription()); + counter(op.getSymbol() + SUFFIX_FAILURES, op.getDescription()); + } + /** * Create a gauge in the registry. * @param name name gauge name @@ -322,6 +320,15 @@ protected final MutableGaugeLong gauge(String name, String desc) { return registry.newGauge(name, desc, 0L); } + /** + * Create a gauge in the registry. + * @param op statistic to count + * @return the gauge + */ + protected final MutableGaugeLong gauge(Statistic op) { + return gauge(op.getSymbol(), op.getDescription()); + } + /** * Create a quantiles in the registry. * @param op statistic to collect @@ -398,7 +405,8 @@ private MutableCounterLong lookupCounter(String name) { } if (!(metric instanceof MutableCounterLong)) { throw new IllegalStateException("Metric " + name - + " is not a MutableCounterLong: " + metric); + + " is not a MutableCounterLong: " + metric + + " (type: " + metric.getClass() +")"); } return (MutableCounterLong) metric; } @@ -442,11 +450,57 @@ public MutableMetric lookupMetric(String name) { return metric; } + /** + * Get the instance IO Statistics. + * @return statistics. + */ + @Override + public IOStatisticsStore getIOStatistics() { + return instanceIOStatistics; + } + + /** + * Get the duration tracker factory. + * @return duration tracking for the instrumentation. + */ + public DurationTrackerFactory getDurationTrackerFactory() { + return durationTrackerFactory; + } + + /** + * The duration tracker updates the metrics with the count + * and IOStatistics will full duration information. + * @param key statistic key prefix + * @param count #of times to increment the matching counter in this + * operation. + * @return a duration tracker. + */ + @Override + public DurationTracker trackDuration(final String key, final long count) { + return durationTrackerFactory.trackDuration(key, count); + } + + /** + * String representation. Includes the IOStatistics + * when logging is at DEBUG. + * @return a string form. + */ + @Override + public String toString() { + final StringBuilder sb = new StringBuilder( + "S3AInstrumentation{"); + if (LOG.isDebugEnabled()) { + sb.append("instanceIOStatistics=").append(instanceIOStatistics); + } + sb.append('}'); + return sb.toString(); + } + /** * Indicate that S3A created a file. */ public void fileCreated() { - numberOfFilesCreated.incr(); + incrementCounter(FILES_CREATED, 1); } /** @@ -454,7 +508,7 @@ public void fileCreated() { * @param count number of files. */ public void fileDeleted(int count) { - numberOfFilesDeleted.incr(count); + incrementCounter(FILES_DELETED, count); } /** @@ -462,21 +516,21 @@ public void fileDeleted(int count) { * @param count number of directory entries included in the delete request. */ public void fakeDirsDeleted(int count) { - numberOfFakeDirectoryDeletes.incr(count); + incrementCounter(FAKE_DIRECTORIES_DELETED, count); } /** * Indicate that S3A created a directory. */ public void directoryCreated() { - numberOfDirectoriesCreated.incr(); + incrementCounter(DIRECTORIES_CREATED, 1); } /** * Indicate that S3A just deleted a directory. */ public void directoryDeleted() { - numberOfDirectoriesDeleted.incr(); + incrementCounter(DIRECTORIES_DELETED, 1); } /** @@ -486,27 +540,44 @@ public void directoryDeleted() { * @param size total size in bytes */ public void filesCopied(int files, long size) { - numberOfFilesCopied.incr(files); - bytesOfFilesCopied.incr(size); + incrementCounter(FILES_COPIED, files); + incrementCounter(FILES_COPIED_BYTES, size); } /** * Note that an error was ignored. */ public void errorIgnored() { - ignoredErrors.incr(); + incrementCounter(IGNORED_ERRORS, 1); } /** - * Increment a specific counter. - * No-op if not defined. + * Increments a mutable counter and the matching + * instance IOStatistics counter. + * No-op if the counter is not defined, or the count == 0. * @param op operation * @param count increment value */ public void incrementCounter(Statistic op, long count) { - MutableCounterLong counter = lookupCounter(op.getSymbol()); - if (counter != null) { - counter.incr(count); + String name = op.getSymbol(); + if (count != 0) { + incrementMutableCounter(name, count); + instanceIOStatistics.incrementCounter(name, count); + } + } + + /** + * Increments a Mutable counter. + * No-op if not a positive integer. + * @param name counter name. + * @param count increment value + */ + private void incrementMutableCounter(final String name, final long count) { + if (count > 0) { + MutableCounterLong counter = lookupCounter(name); + if (counter != null) { + counter.incr(count); + } } } @@ -525,8 +596,10 @@ public void addValueToQuantiles(Statistic op, long value) { } /** - * Increment a specific counter. - * No-op if not defined. + * Increments a mutable counter and the matching + * instance IOStatistics counter with the value of + * the atomic long. + * No-op if the counter is not defined, or the count == 0. * @param op operation * @param count atomic long containing value */ @@ -566,12 +639,30 @@ public void decrementGauge(Statistic op, long count) { } } + /** + * Add the duration as a timed statistic, deriving + * statistic name from the operation symbol and the outcome. + * @param op operation + * @param success was the operation a success? + * @param duration how long did it take + */ + @Override + public void recordDuration(final Statistic op, + final boolean success, + final Duration duration) { + String name = op.getSymbol() + + (success ? "" : SUFFIX_FAILURES); + instanceIOStatistics.addTimedOperation(name, duration); + } + /** * Create a stream input statistics instance. * @return the new instance + * @param filesystemStatistics FS Statistics to update in close(). */ - public InputStreamStatistics newInputStreamStatistics() { - return new InputStreamStatistics(); + public S3AInputStreamStatistics newInputStreamStatistics( + @Nullable final FileSystem.Statistics filesystemStatistics) { + return new InputStreamStatistics(filesystemStatistics); } /** @@ -587,34 +678,8 @@ public MetastoreInstrumentation getS3GuardInstrumentation() { * Create a new instance of the committer statistics. * @return a new committer statistics instance */ - CommitterStatistics newCommitterStatistics() { - return new CommitterStatistics(); - } - - /** - * Merge in the statistics of a single input stream into - * 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); - incrementCounter(STREAM_READ_VERSION_MISMATCHES, - statistics.versionMismatches.get()); + public CommitterStatistics newCommitterStatistics() { + return new CommitterStatisticsImpl(); } @Override @@ -623,14 +688,15 @@ public void getMetrics(MetricsCollector collector, boolean all) { } public void close() { - synchronized (metricsSystemLock) { + synchronized (METRICS_SYSTEM_LOCK) { // it is critical to close each quantile, as they start a scheduled // task in a shared thread pool. putLatencyQuantile.stop(); 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(); @@ -641,164 +707,364 @@ 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. + * A duration tracker which updates a mutable counter with a metric. + * The metric is updated with the count on start; after a failure + * the failures count is incremented by one. */ - @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 AtomicLong versionMismatches = new AtomicLong(0); - private InputStreamStatistics mergedStats; - - private InputStreamStatistics() { + private final class MetricUpdatingDurationTracker + implements DurationTracker { + + private final String symbol; + + private boolean failed; + + private MetricUpdatingDurationTracker( + final String symbol, + final long count) { + this.symbol = symbol; + incrementMutableCounter(symbol, count); + } + + @Override + public void failed() { + failed = true; } /** - * Seek backwards, incrementing the seek and backward seek counters. - * @param negativeOffset how far was the seek? - * This is expected to be negative. + * Close: on failure increment any mutable counter of + * failures. */ + @Override + public void close() { + if (failed) { + incrementMutableCounter(symbol + SUFFIX_FAILURES, 1); + } + } + } + + /** + * Duration Tracker Factory for updating metrics. + */ + private final class MetricDurationTrackerFactory + implements DurationTrackerFactory { + + @Override + public DurationTracker trackDuration(final String key, final long count) { + return new MetricUpdatingDurationTracker(key, count); + } + + } + + /** + * Statistics updated by an S3AInputStream during its actual operation. + *

+ * When {@code unbuffer()} is called, the changed numbers are propagated + * to the S3AFileSystem metrics. + *

+ *

+ * When {@code close()} is called, the final set of numbers are propagated + * to the S3AFileSystem metrics. + *

+ * The {@link FileSystem.Statistics} statistics passed in are also + * updated. This ensures that whichever thread calls close() gets the + * total count of bytes read, even if any work is done in other + * threads. + * + */ + private final class InputStreamStatistics + extends AbstractS3AStatisticsSource + implements S3AInputStreamStatistics { + + /** + * Distance used when incrementing FS stats. + */ + private static final int DISTANCE = 5; + + /** + * FS statistics for the thread creating the stream. + */ + private final FileSystem.Statistics filesystemStatistics; + + /** + * The statistics from the last merge. + */ + private IOStatisticsSnapshot mergedStats; + + /* + The core counters are extracted to atomic longs for slightly + faster resolution on the critical paths, especially single byte + reads and the like. + */ + private final AtomicLong aborted; + private final AtomicLong backwardSeekOperations; + private final AtomicLong bytesBackwardsOnSeek; + private final AtomicLong bytesDiscardedInAbort; + /** Bytes read by the application. */ + private final AtomicLong bytesRead; + private final AtomicLong bytesDiscardedInClose; + private final AtomicLong bytesDiscardedOnSeek; + private final AtomicLong bytesSkippedOnSeek; + private final AtomicLong closed; + private final AtomicLong forwardSeekOperations; + private final AtomicLong openOperations; + private final AtomicLong readExceptions; + private final AtomicLong readsIncomplete; + private final AtomicLong readOperations; + private final AtomicLong readFullyOperations; + private final AtomicLong seekOperations; + + /** Bytes read by the application and any when draining streams . */ + private final AtomicLong totalBytesRead; + + /** + * Instantiate. + * @param filesystemStatistics FS Statistics to update in close(). + */ + private InputStreamStatistics( + @Nullable FileSystem.Statistics filesystemStatistics) { + this.filesystemStatistics = filesystemStatistics; + IOStatisticsStore st = iostatisticsStore() + .withCounters( + StreamStatisticNames.STREAM_READ_ABORTED, + StreamStatisticNames.STREAM_READ_BYTES_DISCARDED_ABORT, + StreamStatisticNames.STREAM_READ_CLOSED, + StreamStatisticNames.STREAM_READ_BYTES_DISCARDED_CLOSE, + 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_SEEK_OPERATIONS, + StreamStatisticNames.STREAM_READ_SEEK_POLICY_CHANGED, + StreamStatisticNames.STREAM_READ_SEEK_BACKWARD_OPERATIONS, + StreamStatisticNames.STREAM_READ_SEEK_FORWARD_OPERATIONS, + StreamStatisticNames.STREAM_READ_SEEK_BYTES_BACKWARDS, + StreamStatisticNames.STREAM_READ_SEEK_BYTES_DISCARDED, + StreamStatisticNames.STREAM_READ_SEEK_BYTES_SKIPPED, + StreamStatisticNames.STREAM_READ_TOTAL_BYTES, + StreamStatisticNames.STREAM_READ_UNBUFFERED, + StreamStatisticNames.STREAM_READ_VERSION_MISMATCHES) + .withGauges(STREAM_READ_GAUGE_INPUT_POLICY) + .withDurationTracking(ACTION_HTTP_GET_REQUEST) + .build(); + setIOStatistics(st); + aborted = st.getCounterReference( + StreamStatisticNames.STREAM_READ_ABORTED); + backwardSeekOperations = st.getCounterReference( + StreamStatisticNames.STREAM_READ_SEEK_BACKWARD_OPERATIONS); + bytesBackwardsOnSeek = st.getCounterReference( + StreamStatisticNames.STREAM_READ_SEEK_BYTES_BACKWARDS); + bytesDiscardedInAbort = st.getCounterReference( + StreamStatisticNames.STREAM_READ_BYTES_DISCARDED_ABORT); + bytesRead = st.getCounterReference( + StreamStatisticNames.STREAM_READ_BYTES); + bytesDiscardedInClose = st.getCounterReference( + StreamStatisticNames.STREAM_READ_BYTES_DISCARDED_CLOSE); + bytesDiscardedOnSeek = st.getCounterReference( + StreamStatisticNames.STREAM_READ_SEEK_BYTES_DISCARDED); + bytesSkippedOnSeek = st.getCounterReference( + StreamStatisticNames.STREAM_READ_SEEK_BYTES_SKIPPED); + closed = st.getCounterReference( + StreamStatisticNames.STREAM_READ_CLOSED); + forwardSeekOperations = st.getCounterReference( + StreamStatisticNames.STREAM_READ_SEEK_FORWARD_OPERATIONS); + openOperations = st.getCounterReference( + StreamStatisticNames.STREAM_READ_OPENED); + readExceptions = st.getCounterReference( + StreamStatisticNames.STREAM_READ_EXCEPTIONS); + readsIncomplete = st.getCounterReference( + StreamStatisticNames.STREAM_READ_OPERATIONS_INCOMPLETE); + readOperations = st.getCounterReference( + StreamStatisticNames.STREAM_READ_OPERATIONS); + readFullyOperations = st.getCounterReference( + StreamStatisticNames.STREAM_READ_FULLY_OPERATIONS); + seekOperations = st.getCounterReference( + StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS); + totalBytesRead = st.getCounterReference( + StreamStatisticNames.STREAM_READ_TOTAL_BYTES); + setIOStatistics(st); + // create initial snapshot of merged statistics + mergedStats = snapshotIOStatistics(st); + } + + /** + * Increment a named counter by one. + * @param name counter name + * @return the new value + */ + private long increment(String name) { + return increment(name, 1); + } + + /** + * Increment a named counter by a given value. + * @param name counter name + * @param value value to increment by. + * @return the new value + */ + private long increment(String name, long value) { + return incCounter(name, value); + } + + /** + * {@inheritDoc}. + * Increments the number of seek operations, + * and backward seek operations. + * The offset is inverted and used as the increment + * of {@link #bytesBackwardsOnSeek}. + */ + @Override public void seekBackwards(long negativeOffset) { - seekOperations++; - backwardSeekOperations++; - bytesBackwardsOnSeek -= negativeOffset; + seekOperations.incrementAndGet(); + backwardSeekOperations.incrementAndGet(); + bytesBackwardsOnSeek.addAndGet(-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. + * {@inheritDoc}. + * Increment the number of seek and forward seek + * operations, as well as counters of bytes skipped + * and bytes read in seek, where appropriate. + * Bytes read in seek are also added to the totalBytesRead + * counter. */ - public void seekForwards(long skipped) { - seekOperations++; - forwardSeekOperations++; + @Override + public void seekForwards(final long skipped, + long bytesReadInSeek) { + seekOperations.incrementAndGet(); + forwardSeekOperations.incrementAndGet(); if (skipped > 0) { - bytesSkippedOnSeek += skipped; + bytesSkippedOnSeek.addAndGet(skipped); + } + if (bytesReadInSeek > 0) { + bytesDiscardedOnSeek.addAndGet(bytesReadInSeek); + totalBytesRead.addAndGet(bytesReadInSeek); } } /** - * The inner stream was opened. - * @return the previous count + * {@inheritDoc}. + * Use {@code getAnIncrement()} on {@link #openOperations} + * so that on invocation 1 it returns 0. + * The caller will know that this is the first invocation. */ + @Override public long streamOpened() { - long count = openOperations; - openOperations++; - return count; + return openOperations.getAndIncrement(); } /** - * 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. + * {@inheritDoc}. + * If the connection was aborted, increment {@link #aborted} + * and add the byte's remaining count to {@link #bytesDiscardedInAbort}. + * If not aborted, increment {@link #closed} and + * then {@link #bytesDiscardedInClose} and {@link #totalBytesRead} + * with the bytes remaining value. */ + @Override public void streamClose(boolean abortedConnection, long remainingInCurrentRequest) { - closeOperations++; if (abortedConnection) { - this.aborted++; - bytesDiscardedInAbort += remainingInCurrentRequest; + // the connection was aborted. + // update the counter of abort() calls and bytes discarded + aborted.incrementAndGet(); + bytesDiscardedInAbort.addAndGet(remainingInCurrentRequest); } else { - closed++; - bytesReadInClose += remainingInCurrentRequest; + // connection closed, possibly draining the stream of surplus + // bytes. + closed.incrementAndGet(); + bytesDiscardedInClose.addAndGet(remainingInCurrentRequest); + totalBytesRead.addAndGet(remainingInCurrentRequest); } } /** - * An ignored stream read exception was received. + * {@inheritDoc}. */ + @Override public void readException() { - readExceptions++; + readExceptions.incrementAndGet(); } /** - * Increment the bytes read counter by the number of bytes; - * no-op if the argument is negative. - * @param bytes number of bytes read + * {@inheritDoc}. + * If the byte counter is positive, increment bytesRead and totalBytesRead. */ + @Override public void bytesRead(long bytes) { if (bytes > 0) { - bytesRead += bytes; + bytesRead.addAndGet(bytes); + totalBytesRead.addAndGet(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 - */ + @Override public void readOperationStarted(long pos, long len) { - readOperations++; + readOperations.incrementAndGet(); } - /** - * 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 - */ + @Override public void readFullyOperationStarted(long pos, long len) { - readFullyOperations++; + readFullyOperations.incrementAndGet(); } /** - * A read operation has completed. - * @param requested number of requested bytes - * @param actual the actual number of bytes + * {@inheritDoc}. + * If more data was requested than was actually returned, this + * was an incomplete read. Increment {@link #readsIncomplete}. */ + @Override public void readOperationCompleted(int requested, int actual) { if (requested > actual) { - readsIncomplete++; + readsIncomplete.incrementAndGet(); } } /** - * Close triggers the merge of statistics into the filesystem's + * {@code close()} merges the stream statistics into the filesystem's * instrumentation instance. */ @Override public void close() { + increment(StreamStatisticNames.STREAM_READ_CLOSE_OPERATIONS); merge(true); } /** - * The input policy has been switched. - * @param updatedPolicy enum value of new policy. + * {@inheritDoc}. + * As well as incrementing the {@code STREAM_READ_SEEK_POLICY_CHANGED} + * counter, the + * {@code STREAM_READ_GAUGE_INPUT_POLICY} gauge is set to the new value. + * */ + @Override public void inputPolicySet(int updatedPolicy) { - policySetCount++; - inputPolicy = updatedPolicy; + increment(StreamStatisticNames.STREAM_READ_SEEK_POLICY_CHANGED); + localIOStatistics().setGauge(STREAM_READ_GAUGE_INPUT_POLICY, + updatedPolicy); } /** - * Get a reference to the version mismatch counter. - * @return a counter which can be incremented. + * Get the inner class's IO Statistics. This is + * needed to avoid findbugs warnings about ambiguity. + * @return the Input Stream's statistics. */ - public AtomicLong getVersionMismatchCounter() { - return versionMismatches; + private IOStatisticsStore localIOStatistics() { + return InputStreamStatistics.super.getIOStatistics(); + } + + /** + * The change tracker increments {@code versionMismatches} on any + * mismatch. + * @return change tracking. + */ + @Override + public ChangeTrackerStatistics getChangeTrackerStatistics() { + return new CountingChangeTracker( + localIOStatistics().getCounterReference( + StreamStatisticNames.STREAM_READ_VERSION_MISMATCHES)); } /** @@ -812,323 +1078,510 @@ 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(", InputPolicy=").append(inputPolicy); - sb.append(", InputPolicySetCount=").append(policySetCount); - sb.append(", versionMismatches=").append(versionMismatches.get()); + sb.append(IOStatisticsLogging.ioStatisticsToString( + localIOStatistics())); sb.append('}'); return sb.toString(); } + /** + * {@inheritDoc} + * Increment the counter {@code STREAM_READ_UNBUFFERED} + * and then merge the current set of statistics into the + * FileSystem's statistics through {@link #merge(boolean)}. + */ + @Override + public void unbuffered() { + increment(STREAM_READ_UNBUFFERED); + merge(false); + } + /** * 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. + *

+ * If the merge is invoked because the stream has been closed, + * then all statistics are merged, and the filesystem + * statistics of {@link #filesystemStatistics} updated + * with the bytes read values. + *

+ *

+ * Whichever thread close()d the stream will have its counters + * updated. + *

+ *

+ * If the merge is due to an unbuffer() call, the change in all + * counters since the last merge will be pushed to the Instrumentation's + * counters. + *

+ * + * @param isClosed is this merge invoked because the stream is closed? */ - void merge(boolean isClosed) { - if (mergedStats != null) { - mergeInputStreamStatistics(diff(mergedStats)); - } else { - mergeInputStreamStatistics(this); - } - // If stats are closed, no need to create another copy - if (!isClosed) { - mergedStats = copy(); + private void merge(boolean isClosed) { + + IOStatisticsStore ioStatistics = localIOStatistics(); + LOG.debug("Merging statistics into FS statistics in {}: {}", + (isClosed ? "close()" : "unbuffer()"), + demandStringifyIOStatistics(ioStatistics)); + promoteInputStreamCountersToMetrics(); + mergedStats = snapshotIOStatistics(localIOStatistics()); + + if (isClosed) { + // stream is being closed. + // merge in all the IOStatistics + S3AInstrumentation.this.getIOStatistics().aggregate(ioStatistics); + + // increment the filesystem statistics for this thread. + if (filesystemStatistics != null) { + long t = getTotalBytesRead(); + filesystemStatistics.incrementBytesRead(t); + filesystemStatistics.incrementBytesReadByDistance(DISTANCE, t); + } } } /** - * Returns a diff between this {@link InputStreamStatistics} instance and - * the given {@link InputStreamStatistics} instance. + * Propagate a counter from the instance-level statistics + * to the S3A instrumentation, subtracting the previous merged value. + * @param name statistic to promote */ - 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; - diff.inputPolicy = inputPolicy - inputStats.inputPolicy; - diff.versionMismatches.set(versionMismatches.longValue() - - inputStats.versionMismatches.longValue()); - return diff; + void promoteIOCounter(String name) { + incrementMutableCounter(name, + lookupCounterValue(name) + - mergedStats.counters().get(name)); } /** - * Returns a new {@link InputStreamStatistics} instance with all the same - * values as this {@link InputStreamStatistics}. + * Merge in the statistics of a single input stream into + * the filesystem-wide metrics counters. + * This does not update the FS IOStatistics values. */ - 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; - copy.inputPolicy = inputPolicy; - return copy; + private void promoteInputStreamCountersToMetrics() { + // iterate through all the counters + localIOStatistics().counters() + .keySet().stream() + .forEach(e -> promoteIOCounter(e)); + } + + @Override + public long getCloseOperations() { + return lookupCounterValue( + StreamStatisticNames.STREAM_READ_CLOSE_OPERATIONS); + } + + @Override + public long getClosed() { + return lookupCounterValue(StreamStatisticNames.STREAM_READ_CLOSED); } + + @Override + public long getAborted() { + return lookupCounterValue(StreamStatisticNames.STREAM_READ_ABORTED); + } + + @Override + public long getForwardSeekOperations() { + return lookupCounterValue( + StreamStatisticNames.STREAM_READ_SEEK_FORWARD_OPERATIONS); + } + + @Override + public long getBackwardSeekOperations() { + return lookupCounterValue( + StreamStatisticNames.STREAM_READ_SEEK_BACKWARD_OPERATIONS); + } + + @Override + public long getBytesRead() { + return lookupCounterValue(StreamStatisticNames.STREAM_READ_BYTES); + } + + @Override + public long getTotalBytesRead() { + return lookupCounterValue(StreamStatisticNames.STREAM_READ_TOTAL_BYTES); + } + + @Override + public long getBytesSkippedOnSeek() { + return lookupCounterValue( + StreamStatisticNames.STREAM_READ_SEEK_BYTES_SKIPPED); + } + + @Override + public long getBytesBackwardsOnSeek() { + return lookupCounterValue( + StreamStatisticNames.STREAM_READ_SEEK_BYTES_BACKWARDS); + } + + @Override + public long getBytesReadInClose() { + return lookupCounterValue( + StreamStatisticNames.STREAM_READ_BYTES_DISCARDED_CLOSE); + } + + @Override + public long getBytesDiscardedInAbort() { + return lookupCounterValue( + StreamStatisticNames.STREAM_READ_BYTES_DISCARDED_ABORT); + } + + @Override + public long getOpenOperations() { + return lookupCounterValue(StreamStatisticNames.STREAM_READ_OPENED); + } + + @Override + public long getSeekOperations() { + return lookupCounterValue( + StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS); + } + + @Override + public long getReadExceptions() { + return lookupCounterValue( + StreamStatisticNames.STREAM_READ_EXCEPTIONS); + } + + @Override + public long getReadOperations() { + return lookupCounterValue(StreamStatisticNames.STREAM_READ_OPERATIONS); + } + + @Override + public long getReadFullyOperations() { + return lookupCounterValue( + StreamStatisticNames.STREAM_READ_FULLY_OPERATIONS); + } + + @Override + public long getReadsIncomplete() { + return lookupCounterValue( + StreamStatisticNames.STREAM_READ_OPERATIONS_INCOMPLETE); + } + + @Override + public long getPolicySetCount() { + return lookupCounterValue( + StreamStatisticNames.STREAM_READ_SEEK_POLICY_CHANGED); + } + + @Override + public long getVersionMismatches() { + return lookupCounterValue( + StreamStatisticNames.STREAM_READ_VERSION_MISMATCHES); + } + + @Override + public long getInputPolicy() { + return localIOStatistics().gauges() + .get(STREAM_READ_GAUGE_INPUT_POLICY); + } + + @Override + public DurationTracker initiateGetRequest() { + return trackDuration(ACTION_HTTP_GET_REQUEST); + } + } /** * Create a stream output statistics instance. + * @param filesystemStatistics thread-local FS statistics. * @return the new instance */ - OutputStreamStatistics newOutputStreamStatistics(Statistics statistics) { - return new OutputStreamStatistics(statistics); + public BlockOutputStreamStatistics newOutputStreamStatistics( + FileSystem.Statistics filesystemStatistics) { + return new OutputStreamStatistics(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( + OutputStreamStatistics 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_EXCEPTIONS, + source.lookupCounterValue( + StreamStatisticNames.STREAM_WRITE_EXCEPTIONS)); + // merge in all the IOStatistics + this.getIOStatistics().aggregate(source.getIOStatistics()); } /** * Statistics updated by an output stream during its actual operation. - * Some of these stats may be relayed. However, as block upload is - * spans multiple + *

+ * Some of these stats are propagated to any passed in + * {@link FileSystem.Statistics} instance; this is done + * in close() for better cross-thread accounting. + *

+ *

+ * Some of the collected statistics are not directly served via + * IOStatistics. + * They are added to the instrumentation IOStatistics and metric counters + * during the {@link #mergeOutputStreamStatistics(OutputStreamStatistics)} + * operation. + *

*/ - @InterfaceAudience.Private - @InterfaceStability.Unstable - public final class OutputStreamStatistics implements Closeable { - private final AtomicLong blocksSubmitted = new AtomicLong(0); - private final AtomicLong blocksInQueue = new AtomicLong(0); + private final class OutputStreamStatistics + extends AbstractS3AStatisticsSource + implements BlockOutputStreamStatistics { + private final AtomicLong blocksActive = new AtomicLong(0); private final AtomicLong blockUploadsCompleted = new AtomicLong(0); - private final AtomicLong blockUploadsFailed = new AtomicLong(0); - private final AtomicLong bytesPendingUpload = new AtomicLong(0); - private final AtomicLong bytesUploaded = new AtomicLong(0); + private final AtomicLong bytesWritten; + private final AtomicLong bytesUploaded; private final AtomicLong transferDuration = new AtomicLong(0); private final AtomicLong queueDuration = new AtomicLong(0); - private final AtomicLong exceptionsInMultipartFinalize = new AtomicLong(0); 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; + /** + * Instantiate. + * @param filesystemStatistics FS Statistics to update in close(). + */ + private OutputStreamStatistics( + @Nullable FileSystem.Statistics filesystemStatistics) { + this.filesystemStatistics = filesystemStatistics; + IOStatisticsStore st = iostatisticsStore() + .withCounters( + StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS, + STREAM_WRITE_BYTES.getSymbol(), + STREAM_WRITE_EXCEPTIONS.getSymbol(), + StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING, + STREAM_WRITE_TOTAL_TIME.getSymbol(), + STREAM_WRITE_QUEUE_DURATION.getSymbol(), + STREAM_WRITE_TOTAL_DATA.getSymbol(), + STREAM_WRITE_EXCEPTIONS.getSymbol(), + STREAM_WRITE_EXCEPTIONS_COMPLETING_UPLOADS.getSymbol()) + .withGauges( + STREAM_WRITE_BLOCK_UPLOADS_PENDING.getSymbol(), + STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING.getSymbol()) + .withDurationTracking(ACTION_EXECUTOR_ACQUIRED) + .build(); + setIOStatistics(st); + // these are extracted to avoid lookups on heavily used counters. + bytesUploaded = st.getCounterReference( + STREAM_WRITE_TOTAL_DATA.getSymbol()); + bytesWritten = st.getCounterReference( + StreamStatisticNames.STREAM_WRITE_BYTES); } /** - * A block has been allocated. + * Increment the Statistic gauge and the local IOStatistics + * equivalent. + * @param statistic statistic + * @param v value. + * @return local IOStatistic value */ - void blockAllocated() { + private long incAllGauges(Statistic statistic, long v) { + incrementGauge(statistic, v); + return incGauge(statistic.getSymbol(), v); + } + + @Override + public void blockAllocated() { blocksAllocated.incrementAndGet(); } + @Override + public void blockReleased() { + blocksReleased.incrementAndGet(); + } + /** - * A block has been released. + * {@inheritDoc} + * Increments the counter of block uplaods, and the gauges + * of block uploads pending (1) and the bytes pending (blockSize). */ - void blockReleased() { - blocksReleased.incrementAndGet(); + @Override + public void blockUploadQueued(int blockSize) { + incCounter(StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS); + incAllGauges(STREAM_WRITE_BLOCK_UPLOADS_PENDING, 1); + incAllGauges(STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING, blockSize); } /** - * Block is queued for upload. + * {@inheritDoc} + * Update {@link #queueDuration} with queue duration, decrement + * {@code STREAM_WRITE_BLOCK_UPLOADS_PENDING} gauge and increment + * {@code STREAM_WRITE_BLOCK_UPLOADS_ACTIVE}. */ - void blockUploadQueued(int blockSize) { - blocksSubmitted.incrementAndGet(); - blocksInQueue.incrementAndGet(); - bytesPendingUpload.addAndGet(blockSize); - incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_PENDING, 1); - incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING, blockSize); - } - - /** Queued block has been scheduled for upload. */ - void blockUploadStarted(long duration, int blockSize) { - queueDuration.addAndGet(duration); - blocksInQueue.decrementAndGet(); - blocksActive.incrementAndGet(); - incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_PENDING, -1); - incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_ACTIVE, 1); - } - - /** A block upload has completed. */ - void blockUploadCompleted(long duration, int blockSize) { - this.transferDuration.addAndGet(duration); - incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_ACTIVE, -1); - blocksActive.decrementAndGet(); + @Override + public void blockUploadStarted(Duration timeInQueue, int blockSize) { + // the local counter is used in toString reporting. + queueDuration.addAndGet(timeInQueue.toMillis()); + // update the duration fields in the IOStatistics. + localIOStatistics().addTimedOperation( + ACTION_EXECUTOR_ACQUIRED, + timeInQueue); + incAllGauges(STREAM_WRITE_BLOCK_UPLOADS_PENDING, -1); + incAllGauges(STREAM_WRITE_BLOCK_UPLOADS_ACTIVE, 1); + } + + /** + * Get the inner class's IO Statistics. This is + * needed to avoid findbugs warnings about ambiguity. + * @return the Input Stream's statistics. + */ + private IOStatisticsStore localIOStatistics() { + return OutputStreamStatistics.super.getIOStatistics(); + } + + /** + * {@inheritDoc} + * Increment the transfer duration; decrement the + * {@code STREAM_WRITE_BLOCK_UPLOADS_ACTIVE} gauge. + */ + @Override + public void blockUploadCompleted( + Duration timeSinceUploadStarted, + int blockSize) { + transferDuration.addAndGet(timeSinceUploadStarted.toMillis()); + incAllGauges(STREAM_WRITE_BLOCK_UPLOADS_ACTIVE, -1); blockUploadsCompleted.incrementAndGet(); } /** * A block upload has failed. * A final transfer completed event is still expected, so this - * does not decrement the active block counter. + * does not decrement any gauges. */ - void blockUploadFailed(long duration, int blockSize) { - blockUploadsFailed.incrementAndGet(); + @Override + public void blockUploadFailed( + Duration timeSinceUploadStarted, + int blockSize) { + incCounter(StreamStatisticNames.STREAM_WRITE_EXCEPTIONS); } - /** Intermediate report of bytes uploaded. */ - void bytesTransferred(long byteCount) { + /** + * Intermediate report of bytes uploaded. + * Increment counters of bytes upload, reduce the counter and + * gauge of pending bytes.; + * @param byteCount bytes uploaded + */ + @Override + public void bytesTransferred(long byteCount) { bytesUploaded.addAndGet(byteCount); - statistics.incrementBytesWritten(byteCount); - bytesPendingUpload.addAndGet(-byteCount); - incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING, -byteCount); + incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING, -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); + incCounter( + STREAM_WRITE_EXCEPTIONS_COMPLETING_UPLOADS.getSymbol(), + count); } } - /** - * Note an exception in a multipart abort. - */ - void exceptionInMultipartAbort() { - exceptionsInMultipartFinalize.incrementAndGet(); + @Override + public void exceptionInMultipartAbort() { + incCounter( + STREAM_WRITE_EXCEPTIONS_COMPLETING_UPLOADS.getSymbol()); } - /** - * Get the number of bytes pending upload. - * @return the number of bytes in the pending upload state. - */ + @Override public long getBytesPendingUpload() { - return bytesPendingUpload.get(); + return lookupGaugeValue( + STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING.getSymbol()); } - /** - * 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 - */ + @Override public void commitUploaded(long size) { incrementCounter(COMMITTER_BYTES_UPLOADED, size); } - /** - * Output stream has closed. - * Trigger merge in of all statistics not updated during operation. - */ @Override public void close() { - if (bytesPendingUpload.get() > 0) { + if (getBytesPendingUpload() > 0) { LOG.warn("Closing output stream statistics while data is still marked" + " 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() { - return blocksSubmitted.get() > 0 ? - (queueDuration.get() / blocksSubmitted.get()) : 0; - } - - double effectiveBandwidth() { + /** + * What is the effective bandwidth of this stream's write. + * @return the bytes uploaded divided by the total duration. + */ + private double effectiveBandwidth() { double duration = totalUploadDuration() / 1000.0; return duration > 0 ? (bytesUploaded.get() / duration) : 0; } - long totalUploadDuration() { + /** + * Total of time spend uploading bytes. + * @return the transfer duration plus queue duration. + */ + private 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() { final StringBuilder sb = new StringBuilder( "OutputStreamStatistics{"); - sb.append("blocksSubmitted=").append(blocksSubmitted); - sb.append(", blocksInQueue=").append(blocksInQueue); + sb.append(localIOStatistics().toString()); sb.append(", blocksActive=").append(blocksActive); sb.append(", blockUploadsCompleted=").append(blockUploadsCompleted); - sb.append(", blockUploadsFailed=").append(blockUploadsFailed); - sb.append(", bytesPendingUpload=").append(bytesPendingUpload); - sb.append(", bytesUploaded=").append(bytesUploaded); 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(", transferDuration=").append(transferDuration).append(" ms"); - sb.append(", queueDuration=").append(queueDuration).append(" ms"); - sb.append(", averageQueueTime=").append(averageQueueTime()).append(" ms"); sb.append(", totalUploadDuration=").append(totalUploadDuration()) .append(" ms"); sb.append(", effectiveBandwidth=").append(effectiveBandwidth()) @@ -1174,10 +1627,6 @@ public void recordsRead(int count) { incrementCounter(S3GUARD_METADATASTORE_RECORD_READS, count); } - /** - * records have been written (including deleted). - * @param count number of records written. - */ @Override public void recordsWritten(int count) { incrementCounter(S3GUARD_METADATASTORE_RECORD_WRITES, count); @@ -1185,7 +1634,8 @@ public void recordsWritten(int count) { @Override public void directoryMarkedAuthoritative() { - incrementCounter(S3GUARD_METADATASTORE_AUTHORITATIVE_DIRECTORIES_UPDATED, + incrementCounter( + S3GUARD_METADATASTORE_AUTHORITATIVE_DIRECTORIES_UPDATED, 1); } @@ -1200,60 +1650,95 @@ public void entryAdded(final long durationNanos) { } /** - * Instrumentation exported to S3Guard Committers. + * Instrumentation exported to S3A Committers. + * The S3AInstrumentation metrics and + * {@link #instanceIOStatistics} are updated continuously. */ - @InterfaceAudience.Private - @InterfaceStability.Unstable - public final class CommitterStatistics { + private final class CommitterStatisticsImpl + extends AbstractS3AStatisticsSource + implements CommitterStatistics { + + private CommitterStatisticsImpl() { + IOStatisticsStore st = iostatisticsStore() + .withCounters( + COMMITTER_BYTES_COMMITTED.getSymbol(), + COMMITTER_BYTES_UPLOADED.getSymbol(), + COMMITTER_COMMITS_CREATED.getSymbol(), + COMMITTER_COMMITS_ABORTED.getSymbol(), + COMMITTER_COMMITS_COMPLETED.getSymbol(), + COMMITTER_COMMITS_FAILED.getSymbol(), + COMMITTER_COMMITS_REVERTED.getSymbol(), + COMMITTER_JOBS_FAILED.getSymbol(), + COMMITTER_JOBS_SUCCEEDED.getSymbol(), + COMMITTER_TASKS_FAILED.getSymbol(), + COMMITTER_TASKS_SUCCEEDED.getSymbol()) + .withDurationTracking( + COMMITTER_COMMIT_JOB.getSymbol(), + COMMITTER_MATERIALIZE_FILE.getSymbol(), + COMMITTER_STAGE_FILE_UPLOAD.getSymbol()) + .build(); + setIOStatistics(st); + } + + /** + * Increment both the local counter and the S3AInstrumentation counters. + * @param stat statistic + * @param value value + * @return the new value + */ + private long increment(Statistic stat, long value) { + incrementCounter(stat, value); + return incCounter(stat.getSymbol(), value); + } /** A commit has been created. */ + @Override public void commitCreated() { - incrementCounter(COMMITTER_COMMITS_CREATED, 1); + increment(COMMITTER_COMMITS_CREATED, 1); } - /** - * 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); + increment(COMMITTER_BYTES_UPLOADED, 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); + increment(COMMITTER_COMMITS_COMPLETED, 1); + increment(COMMITTER_BYTES_COMMITTED, size); } - /** A commit has been aborted. */ + @Override public void commitAborted() { - incrementCounter(COMMITTER_COMMITS_ABORTED, 1); + increment(COMMITTER_COMMITS_ABORTED, 1); } + @Override public void commitReverted() { - incrementCounter(COMMITTER_COMMITS_REVERTED, 1); + increment(COMMITTER_COMMITS_REVERTED, 1); } + @Override public void commitFailed() { - incrementCounter(COMMITTER_COMMITS_FAILED, 1); + increment(COMMITTER_COMMITS_FAILED, 1); } + @Override public void taskCompleted(boolean success) { - incrementCounter( - success ? COMMITTER_TASKS_SUCCEEDED + increment(success + ? COMMITTER_TASKS_SUCCEEDED : COMMITTER_TASKS_FAILED, 1); } + @Override public void jobCompleted(boolean success) { - incrementCounter( - success ? COMMITTER_JOBS_SUCCEEDED + increment(success + ? COMMITTER_JOBS_SUCCEEDED : COMMITTER_JOBS_FAILED, 1); } + } /** @@ -1261,26 +1746,33 @@ 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. + * The {@link #tokenIssued()} call is a no-op; + * This statistics class doesn't collect any local statistics. + * Instead it directly updates the S3A Instrumentation. */ - @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); + } + + @Override + public DurationTracker trackDuration(final String key, final long count) { + return getDurationTrackerFactory() + .trackDuration(key, count); } } - /** + /** * Copy all the metrics to a map of (name, long-value). * @return a map of the metrics */ @@ -1296,7 +1788,7 @@ public Map toMap() { private static class MetricsToMap extends MetricsRecordBuilder { private final MetricsCollector parent; private final Map map = - new HashMap<>(COUNTERS_TO_CREATE.length * 2); + new HashMap<>(); MetricsToMap(MetricsCollector parent) { this.parent = parent; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOpContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOpContext.java index f09aa82281db8..aca1fa5e1408f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOpContext.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOpContext.java @@ -24,19 +24,23 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.s3a.impl.ActiveOperationContext; +import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext; /** - * Base class for operation context struct passed through codepaths for main + * Class for operation context struct passed through codepaths for main * S3AFileSystem operations. * Anything op-specific should be moved to a subclass of this. + * + * This was originally a base class, but {@link ActiveOperationContext} was + * created to be more minimal and cover many more operation type. */ -@SuppressWarnings("visibilitymodifier") // I want a struct of finals, for real. -public class S3AOpContext { +@SuppressWarnings("visibilitymodifier") +public class S3AOpContext extends ActiveOperationContext { final boolean isS3GuardEnabled; final Invoker invoker; @Nullable final FileSystem.Statistics stats; - final S3AInstrumentation instrumentation; @Nullable final Invoker s3guardInvoker; /** FileStatus for "destination" path being operated on. */ @@ -53,9 +57,14 @@ public class S3AOpContext { * @param dstFileStatus file status from existence check */ public S3AOpContext(boolean isS3GuardEnabled, Invoker invoker, - Invoker s3guardInvoker, @Nullable FileSystem.Statistics stats, - S3AInstrumentation instrumentation, FileStatus dstFileStatus) { + @Nullable Invoker s3guardInvoker, + @Nullable FileSystem.Statistics stats, + S3AStatisticsContext instrumentation, + FileStatus dstFileStatus) { + super(newOperationId(), + instrumentation, + null); Preconditions.checkNotNull(invoker, "Null invoker arg"); Preconditions.checkNotNull(instrumentation, "Null instrumentation arg"); Preconditions.checkNotNull(dstFileStatus, "Null dstFileStatus arg"); @@ -65,7 +74,6 @@ public S3AOpContext(boolean isS3GuardEnabled, Invoker invoker, this.invoker = invoker; this.s3guardInvoker = s3guardInvoker; this.stats = stats; - this.instrumentation = instrumentation; this.dstFileStatus = dstFileStatus; } @@ -77,8 +85,10 @@ public S3AOpContext(boolean isS3GuardEnabled, Invoker invoker, * @param instrumentation instrumentation to use * @param dstFileStatus file status from existence check */ - public S3AOpContext(boolean isS3GuardEnabled, Invoker invoker, - @Nullable FileSystem.Statistics stats, S3AInstrumentation instrumentation, + public S3AOpContext(boolean isS3GuardEnabled, + Invoker invoker, + @Nullable FileSystem.Statistics stats, + S3AStatisticsContext instrumentation, FileStatus dstFileStatus) { this(isS3GuardEnabled, invoker, null, stats, instrumentation, dstFileStatus); @@ -97,10 +107,6 @@ public FileSystem.Statistics getStats() { return stats; } - public S3AInstrumentation getInstrumentation() { - return instrumentation; - } - @Nullable public Invoker getS3guardInvoker() { return s3guardInvoker; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java index 12be3a51f8107..3729341dbfe27 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java @@ -22,6 +22,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; +import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext; import javax.annotation.Nullable; @@ -60,8 +61,8 @@ public class S3AReadOpContext extends S3AOpContext { * @param isS3GuardEnabled true iff S3Guard is enabled. * @param invoker invoker for normal retries. * @param s3guardInvoker S3Guard-specific retry invoker. - * @param stats statistics (may be null) - * @param instrumentation FS instrumentation + * @param stats Fileystem statistics (may be null) + * @param instrumentation statistics context * @param dstFileStatus target file status * @param inputPolicy the input policy * @param readahead readahead for GET operations/skip, etc. @@ -71,13 +72,14 @@ public S3AReadOpContext( final Path path, boolean isS3GuardEnabled, Invoker invoker, - Invoker s3guardInvoker, + @Nullable Invoker s3guardInvoker, @Nullable FileSystem.Statistics stats, - S3AInstrumentation instrumentation, + S3AStatisticsContext instrumentation, FileStatus dstFileStatus, S3AInputPolicy inputPolicy, ChangeDetectionPolicy changeDetectionPolicy, final long readahead) { + super(isS3GuardEnabled, invoker, s3guardInvoker, stats, instrumentation, dstFileStatus); this.path = checkNotNull(path); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStorageStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStorageStatistics.java index 4b126673b555f..33ae09119960e 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStorageStatistics.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStorageStatistics.java @@ -20,106 +20,27 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.fs.StorageStatistics; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.impl.StorageStatisticsFromIOStatistics; -import java.util.Collections; -import java.util.EnumMap; -import java.util.Iterator; -import java.util.Map; -import java.util.NoSuchElementException; -import java.util.concurrent.atomic.AtomicLong; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.emptyStatistics; /** - * Storage statistics for S3A. + * Storage statistics for S3A, dynamically generated from the IOStatistics. */ @InterfaceAudience.Private @InterfaceStability.Evolving -public class S3AStorageStatistics extends StorageStatistics - implements Iterable { - private static final Logger LOG = - LoggerFactory.getLogger(S3AStorageStatistics.class); +public class S3AStorageStatistics + extends StorageStatisticsFromIOStatistics { public static final String NAME = "S3AStorageStatistics"; - private final Map opsCount = - new EnumMap<>(Statistic.class); - public S3AStorageStatistics() { - super(NAME); - for (Statistic opType : Statistic.values()) { - opsCount.put(opType, new AtomicLong(0)); - } - } - - /** - * Increment a specific counter. - * @param op operation - * @param count increment value - * @return the new value - */ - public long incrementCounter(Statistic op, long count) { - long updated = opsCount.get(op).addAndGet(count); - LOG.debug("{} += {} -> {}", op, count, updated); - return updated; - } - - private class LongIterator implements Iterator { - private Iterator> iterator = - Collections.unmodifiableSet(opsCount.entrySet()).iterator(); - - @Override - public boolean hasNext() { - return iterator.hasNext(); - } - - @Override - public LongStatistic next() { - if (!iterator.hasNext()) { - throw new NoSuchElementException(); - } - final Map.Entry entry = iterator.next(); - return new LongStatistic(entry.getKey().getSymbol(), - entry.getValue().get()); - } - - @Override - public void remove() { - throw new UnsupportedOperationException(); - } - } - - @Override - public String getScheme() { - return "s3a"; + public S3AStorageStatistics(final IOStatistics ioStatistics) { + super(NAME, "s3a", ioStatistics); } - @Override - public Iterator getLongStatistics() { - return new LongIterator(); - } - - @Override - public Iterator iterator() { - return getLongStatistics(); - } - - @Override - public Long getLong(String key) { - final Statistic type = Statistic.fromSymbol(key); - return type == null ? null : opsCount.get(type).get(); - } - - @Override - public boolean isTracked(String key) { - return Statistic.fromSymbol(key) != null; - } - - @Override - public void reset() { - for (AtomicLong value : opsCount.values()) { - value.set(0); - } + public S3AStorageStatistics() { + super(NAME, "s3a", emptyStatistics()); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java index 65c0c2360784a..314f13f0352e9 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java @@ -48,6 +48,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.util.functional.RemoteIterators; import org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider; import org.apache.hadoop.fs.s3a.impl.NetworkBinding; import org.apache.hadoop.fs.s3native.S3xLoginHelper; @@ -1470,12 +1471,7 @@ public interface LocatedFileStatusMap { public static long applyLocatedFiles( RemoteIterator iterator, CallOnLocatedFileStatus eval) throws IOException { - long count = 0; - while (iterator.hasNext()) { - count++; - eval.call(iterator.next()); - } - return count; + return RemoteIterators.foreach(iterator, eval::call); } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java index e0a1d780ccf5f..e04d3b5cbd269 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java @@ -26,6 +26,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.s3a.statistics.StatisticsFromAwsSdk; /** * Factory for creation of {@link AmazonS3} client instances. @@ -41,12 +42,14 @@ public interface S3ClientFactory { * @param bucket Optional bucket to use to look up per-bucket proxy secrets * @param credentialSet credentials to use * @param userAgentSuffix optional suffix for the UA field. + * @param statisticsFromAwsSdk binding for AWS stats - may be null * @return S3 client * @throws IOException IO problem */ AmazonS3 createS3Client(URI name, String bucket, AWSCredentialsProvider credentialSet, - String userAgentSuffix) throws IOException; + String userAgentSuffix, + StatisticsFromAwsSdk statisticsFromAwsSdk) throws IOException; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java index 1addfbef78710..6709382baf54d 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java @@ -18,248 +18,504 @@ package org.apache.hadoop.fs.s3a; -import org.apache.hadoop.fs.StorageStatistics.CommonStatisticNames; - import java.util.HashMap; import java.util.Map; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.s3a.statistics.StatisticTypeEnum; +import org.apache.hadoop.fs.statistics.StoreStatisticNames; +import org.apache.hadoop.fs.statistics.StreamStatisticNames; + +import static org.apache.hadoop.fs.s3a.statistics.StatisticTypeEnum.TYPE_COUNTER; +import static org.apache.hadoop.fs.s3a.statistics.StatisticTypeEnum.TYPE_DURATION; +import static org.apache.hadoop.fs.s3a.statistics.StatisticTypeEnum.TYPE_GAUGE; +import static org.apache.hadoop.fs.s3a.statistics.StatisticTypeEnum.TYPE_QUANTILE; + /** * Statistic which are collected in S3A. - * These statistics are available at a low level in {@link S3AStorageStatistics} - * and as metrics in {@link S3AInstrumentation} + * Counter and duration statistics are published in + * {@link S3AFileSystem#getStorageStatistics()}. + * and as metrics in {@link S3AInstrumentation}. + *

+ * Where possible, stream names come from {@link StreamStatisticNames} + * and {@link StoreStatisticNames} + *

*/ +@InterfaceStability.Unstable public enum Statistic { + /* Low-level duration counters */ + ACTION_EXECUTOR_ACQUIRED( + StoreStatisticNames.ACTION_EXECUTOR_ACQUIRED, + "Executor acquired.", + TYPE_DURATION), + ACTION_HTTP_HEAD_REQUEST( + StoreStatisticNames.ACTION_HTTP_HEAD_REQUEST, + "HEAD request.", + TYPE_DURATION), + ACTION_HTTP_GET_REQUEST( + StoreStatisticNames.ACTION_HTTP_GET_REQUEST, + "GET request.", + TYPE_DURATION), + + /* FileSystem Level statistics */ DIRECTORIES_CREATED("directories_created", - "Total number of directories created through the object store."), + "Total number of directories created through the object store.", + TYPE_COUNTER), DIRECTORIES_DELETED("directories_deleted", - "Total number of directories deleted through the object store."), + "Total number of directories deleted through the object store.", + TYPE_COUNTER), FILES_COPIED("files_copied", - "Total number of files copied within the object store."), + "Total number of files copied within the object store.", + TYPE_COUNTER), FILES_COPIED_BYTES("files_copied_bytes", - "Total number of bytes copied within the object store."), + "Total number of bytes copied within the object store.", + TYPE_COUNTER), FILES_CREATED("files_created", - "Total number of files created through the object store."), + "Total number of files created through the object store.", + TYPE_COUNTER), FILES_DELETED("files_deleted", - "Total number of files deleted from the object store."), + "Total number of files deleted from the object store.", + TYPE_COUNTER), FILES_DELETE_REJECTED("files_delete_rejected", - "Total number of files whose delete request was rejected"), + "Total number of files whose delete request was rejected", + TYPE_COUNTER), FAKE_DIRECTORIES_CREATED("fake_directories_created", - "Total number of fake directory entries created in the object store."), + "Total number of fake directory entries created in the object store.", + TYPE_COUNTER), FAKE_DIRECTORIES_DELETED("fake_directories_deleted", - "Total number of fake directory deletes submitted to object store."), - IGNORED_ERRORS("ignored_errors", "Errors caught and ignored"), - INVOCATION_COPY_FROM_LOCAL_FILE(CommonStatisticNames.OP_COPY_FROM_LOCAL_FILE, - "Calls of copyFromLocalFile()"), - INVOCATION_CREATE(CommonStatisticNames.OP_CREATE, - "Calls of create()"), - INVOCATION_CREATE_NON_RECURSIVE(CommonStatisticNames.OP_CREATE_NON_RECURSIVE, - "Calls of createNonRecursive()"), - INVOCATION_DELETE(CommonStatisticNames.OP_DELETE, - "Calls of delete()"), - INVOCATION_EXISTS(CommonStatisticNames.OP_EXISTS, - "Calls of exists()"), - INVOCATION_GET_DELEGATION_TOKEN(CommonStatisticNames.OP_GET_DELEGATION_TOKEN, - "Calls of getDelegationToken()"), - INVOCATION_GET_FILE_CHECKSUM(CommonStatisticNames.OP_GET_FILE_CHECKSUM, - "Calls of getFileChecksum()"), - INVOCATION_GET_FILE_STATUS(CommonStatisticNames.OP_GET_FILE_STATUS, - "Calls of getFileStatus()"), - INVOCATION_GLOB_STATUS(CommonStatisticNames.OP_GLOB_STATUS, - "Calls of globStatus()"), - INVOCATION_IS_DIRECTORY(CommonStatisticNames.OP_IS_DIRECTORY, - "Calls of isDirectory()"), - INVOCATION_IS_FILE(CommonStatisticNames.OP_IS_FILE, - "Calls of isFile()"), - INVOCATION_LIST_FILES(CommonStatisticNames.OP_LIST_FILES, - "Calls of listFiles()"), - INVOCATION_LIST_LOCATED_STATUS(CommonStatisticNames.OP_LIST_LOCATED_STATUS, - "Calls of listLocatedStatus()"), - INVOCATION_LIST_STATUS(CommonStatisticNames.OP_LIST_STATUS, - "Calls of listStatus()"), - INVOCATION_MKDIRS(CommonStatisticNames.OP_MKDIRS, - "Calls of mkdirs()"), - INVOCATION_OPEN(CommonStatisticNames.OP_OPEN, - "Calls of open()"), - INVOCATION_RENAME(CommonStatisticNames.OP_RENAME, - "Calls of rename()"), - OBJECT_COPY_REQUESTS("object_copy_requests", "Object copy requests"), - OBJECT_DELETE_REQUESTS("object_delete_requests", "Object delete requests"), - OBJECT_DELETE_OBJECTS("object_delete_objects", - "Objects deleted in delete requests"), - OBJECT_LIST_REQUESTS("object_list_requests", - "Number of object listings made"), - OBJECT_CONTINUE_LIST_REQUESTS("object_continue_list_requests", - "Number of continued object listings made"), - OBJECT_METADATA_REQUESTS("object_metadata_requests", - "Number of requests for object metadata"), - OBJECT_MULTIPART_UPLOAD_INITIATED("object_multipart_initiated", - "Object multipart upload initiated"), - OBJECT_MULTIPART_UPLOAD_ABORTED("object_multipart_aborted", - "Object multipart upload aborted"), - OBJECT_PUT_REQUESTS("object_put_requests", - "Object put/multipart upload count"), - OBJECT_PUT_REQUESTS_COMPLETED("object_put_requests_completed", - "Object put/multipart upload completed count"), - OBJECT_PUT_REQUESTS_ACTIVE("object_put_requests_active", - "Current number of active put requests"), - OBJECT_PUT_BYTES("object_put_bytes", "number of bytes uploaded"), - OBJECT_PUT_BYTES_PENDING("object_put_bytes_pending", - "number of bytes queued for upload/being actively uploaded"), - OBJECT_SELECT_REQUESTS("object_select_requests", - "Count of S3 Select requests issued"), - STREAM_ABORTED("stream_aborted", - "Count of times the TCP stream was aborted"), - STREAM_BACKWARD_SEEK_OPERATIONS("stream_backward_seek_operations", - "Number of executed seek operations which went backwards in a stream"), - STREAM_CLOSED("stream_closed", "Count of times the TCP stream was closed"), - STREAM_CLOSE_OPERATIONS("stream_close_operations", - "Total count of times an attempt to close a data stream was made"), - STREAM_FORWARD_SEEK_OPERATIONS("stream_forward_seek_operations", - "Number of executed seek operations which went forward in a stream"), - STREAM_OPENED("stream_opened", - "Total count of times an input stream to object store was opened"), - STREAM_READ_EXCEPTIONS("stream_read_exceptions", - "Number of exceptions invoked on input streams"), - STREAM_READ_FULLY_OPERATIONS("stream_read_fully_operations", - "Count of readFully() operations in streams"), - STREAM_READ_OPERATIONS("stream_read_operations", - "Count of read() operations in streams"), - STREAM_READ_OPERATIONS_INCOMPLETE("stream_read_operations_incomplete", - "Count of incomplete read() operations in streams"), - STREAM_READ_VERSION_MISMATCHES("stream_read_version_mismatches", - "Count of version mismatches encountered while reading streams"), - STREAM_SEEK_BYTES_BACKWARDS("stream_bytes_backwards_on_seek", - "Count of bytes moved backwards during seek operations"), - STREAM_SEEK_BYTES_READ("stream_bytes_read", - "Count of bytes read during seek() in stream operations"), - STREAM_SEEK_BYTES_SKIPPED("stream_bytes_skipped_on_seek", - "Count of bytes skipped during forward seek operation"), - STREAM_SEEK_OPERATIONS("stream_seek_operations", - "Number of seek operations during stream IO."), - STREAM_CLOSE_BYTES_READ("stream_bytes_read_in_close", - "Count of bytes read when closing streams during seek operations."), - STREAM_ABORT_BYTES_DISCARDED("stream_bytes_discarded_in_abort", - "Count of bytes discarded by aborting the stream"), - STREAM_WRITE_FAILURES("stream_write_failures", - "Count of stream write failures reported"), - STREAM_WRITE_BLOCK_UPLOADS("stream_write_block_uploads", - "Count of block/partition uploads completed"), - STREAM_WRITE_BLOCK_UPLOADS_ACTIVE("stream_write_block_uploads_active", - "Count of block/partition uploads completed"), - STREAM_WRITE_BLOCK_UPLOADS_COMMITTED("stream_write_block_uploads_committed", - "Count of number of block uploads committed"), - STREAM_WRITE_BLOCK_UPLOADS_ABORTED("stream_write_block_uploads_aborted", - "Count of number of block uploads aborted"), - - STREAM_WRITE_BLOCK_UPLOADS_PENDING("stream_write_block_uploads_pending", - "Gauge of block/partitions uploads queued to be written"), - STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING( - "stream_write_block_uploads_data_pending", - "Gauge of block/partitions data uploads queued to be written"), - STREAM_WRITE_TOTAL_TIME("stream_write_total_time", - "Count of total time taken for uploads to complete"), - STREAM_WRITE_TOTAL_DATA("stream_write_total_data", - "Count of total data uploaded in block output"), - STREAM_WRITE_QUEUE_DURATION("stream_write_queue_duration", - "Total queue duration of all block uploads"), - - // S3guard committer stats + "Total number of fake directory deletes submitted to object store.", + TYPE_COUNTER), + IGNORED_ERRORS("ignored_errors", "Errors caught and ignored", + TYPE_COUNTER), + INVOCATION_COPY_FROM_LOCAL_FILE( + StoreStatisticNames.OP_COPY_FROM_LOCAL_FILE, + "Calls of copyFromLocalFile()", + TYPE_COUNTER), + INVOCATION_CREATE( + StoreStatisticNames.OP_CREATE, + "Calls of create()", + TYPE_COUNTER), + INVOCATION_CREATE_NON_RECURSIVE( + StoreStatisticNames.OP_CREATE_NON_RECURSIVE, + "Calls of createNonRecursive()", + TYPE_COUNTER), + INVOCATION_DELETE( + StoreStatisticNames.OP_DELETE, + "Calls of delete()", + TYPE_COUNTER), + INVOCATION_EXISTS( + StoreStatisticNames.OP_EXISTS, + "Calls of exists()", + TYPE_COUNTER), + INVOCATION_GET_DELEGATION_TOKEN( + StoreStatisticNames.OP_GET_DELEGATION_TOKEN, + "Calls of getDelegationToken()", + TYPE_COUNTER), + INVOCATION_GET_FILE_CHECKSUM( + StoreStatisticNames.OP_GET_FILE_CHECKSUM, + "Calls of getFileChecksum()", + TYPE_COUNTER), + INVOCATION_GET_FILE_STATUS( + StoreStatisticNames.OP_GET_FILE_STATUS, + "Calls of getFileStatus()", + TYPE_COUNTER), + INVOCATION_GLOB_STATUS( + StoreStatisticNames.OP_GLOB_STATUS, + "Calls of globStatus()", + TYPE_COUNTER), + INVOCATION_IS_DIRECTORY( + StoreStatisticNames.OP_IS_DIRECTORY, + "Calls of isDirectory()", + TYPE_COUNTER), + INVOCATION_IS_FILE( + StoreStatisticNames.OP_IS_FILE, + "Calls of isFile()", + TYPE_COUNTER), + INVOCATION_LIST_FILES( + StoreStatisticNames.OP_LIST_FILES, + "Calls of listFiles()", + TYPE_COUNTER), + INVOCATION_LIST_LOCATED_STATUS( + StoreStatisticNames.OP_LIST_LOCATED_STATUS, + "Calls of listLocatedStatus()", + TYPE_COUNTER), + INVOCATION_LIST_STATUS( + StoreStatisticNames.OP_LIST_STATUS, + "Calls of listStatus()", + TYPE_COUNTER), + INVOCATION_MKDIRS( + StoreStatisticNames.OP_MKDIRS, + "Calls of mkdirs()", + TYPE_COUNTER), + INVOCATION_OPEN( + StoreStatisticNames.OP_OPEN, + "Calls of open()", + TYPE_COUNTER), + INVOCATION_RENAME( + StoreStatisticNames.OP_RENAME, + "Calls of rename()", + TYPE_COUNTER), + + /* Object IO */ + OBJECT_COPY_REQUESTS(StoreStatisticNames.OBJECT_COPY_REQUESTS, + "Object copy requests", + TYPE_COUNTER), + OBJECT_DELETE_REQUEST(StoreStatisticNames.OBJECT_DELETE_REQUEST, + "Object delete requests", + TYPE_DURATION), + OBJECT_BULK_DELETE_REQUEST(StoreStatisticNames.OBJECT_BULK_DELETE_REQUEST, + "Object bulk delete requests", + TYPE_DURATION), + OBJECT_DELETE_OBJECTS(StoreStatisticNames.OBJECT_DELETE_OBJECTS, + "Objects deleted in delete requests", + TYPE_COUNTER), + OBJECT_LIST_REQUEST(StoreStatisticNames.OBJECT_LIST_REQUEST, + "Count of object listings made", + TYPE_DURATION), + OBJECT_CONTINUE_LIST_REQUESTS( + StoreStatisticNames.OBJECT_CONTINUE_LIST_REQUEST, + "Count of continued object listings made", + TYPE_DURATION), + OBJECT_METADATA_REQUESTS( + StoreStatisticNames.OBJECT_METADATA_REQUESTS, + "Count of requests for object metadata", + TYPE_COUNTER), + OBJECT_MULTIPART_UPLOAD_INITIATED( + StoreStatisticNames.OBJECT_MULTIPART_UPLOAD_INITIATED, + "Object multipart upload initiated", + TYPE_COUNTER), + OBJECT_MULTIPART_UPLOAD_ABORTED( + StoreStatisticNames.OBJECT_MULTIPART_UPLOAD_ABORTED, + "Object multipart upload aborted", + TYPE_COUNTER), + OBJECT_PUT_REQUESTS( + StoreStatisticNames.OBJECT_PUT_REQUEST, + "Object put/multipart upload count", + TYPE_COUNTER), + OBJECT_PUT_REQUESTS_COMPLETED( + StoreStatisticNames.OBJECT_PUT_REQUEST_COMPLETED, + "Object put/multipart upload completed count", + TYPE_COUNTER), + OBJECT_PUT_REQUESTS_ACTIVE( + StoreStatisticNames.OBJECT_PUT_REQUEST_ACTIVE, + "Current number of active put requests", + TYPE_GAUGE), + OBJECT_PUT_BYTES( + StoreStatisticNames.OBJECT_PUT_BYTES, + "number of bytes uploaded", + TYPE_COUNTER), + OBJECT_PUT_BYTES_PENDING( + StoreStatisticNames.OBJECT_PUT_BYTES_PENDING, + "number of bytes queued for upload/being actively uploaded", + TYPE_GAUGE), + OBJECT_SELECT_REQUESTS( + StoreStatisticNames.OBJECT_SELECT_REQUESTS, + "Count of S3 Select requests issued", + TYPE_COUNTER), + STREAM_READ_ABORTED( + StreamStatisticNames.STREAM_READ_ABORTED, + "Count of times the TCP stream was aborted", + TYPE_COUNTER), + + /* Stream Reads */ + STREAM_READ_BYTES( + StreamStatisticNames.STREAM_READ_BYTES, + "Bytes read from an input stream in read() calls", + TYPE_COUNTER), + STREAM_READ_BYTES_DISCARDED_ABORT( + StreamStatisticNames.STREAM_READ_BYTES_DISCARDED_ABORT, + "Count of bytes discarded by aborting an input stream", + TYPE_COUNTER), + STREAM_READ_BYTES_READ_CLOSE( + StreamStatisticNames.STREAM_READ_BYTES_DISCARDED_CLOSE, + "Count of bytes read and discarded when closing an input stream", + TYPE_COUNTER), + STREAM_READ_CLOSED( + StreamStatisticNames.STREAM_READ_CLOSED, + "Count of times the TCP stream was closed", + TYPE_COUNTER), + STREAM_READ_CLOSE_OPERATIONS( + StreamStatisticNames.STREAM_READ_CLOSE_OPERATIONS, + "Total count of times an attempt to close an input stream was made", + TYPE_COUNTER), + STREAM_READ_EXCEPTIONS( + StreamStatisticNames.STREAM_READ_EXCEPTIONS, + "Count of exceptions raised during input stream reads", + TYPE_COUNTER), + STREAM_READ_FULLY_OPERATIONS( + StreamStatisticNames.STREAM_READ_FULLY_OPERATIONS, + "Count of readFully() operations in an input stream", + TYPE_COUNTER), + STREAM_READ_OPENED( + StreamStatisticNames.STREAM_READ_OPENED, + "Total count of times an input stream to object store data was opened", + TYPE_COUNTER), + STREAM_READ_OPERATIONS( + StreamStatisticNames.STREAM_READ_OPERATIONS, + "Count of read() operations in an input stream", + TYPE_COUNTER), + STREAM_READ_OPERATIONS_INCOMPLETE( + StreamStatisticNames.STREAM_READ_OPERATIONS_INCOMPLETE, + "Count of incomplete read() operations in an input stream", + TYPE_COUNTER), + STREAM_READ_VERSION_MISMATCHES( + StreamStatisticNames.STREAM_READ_VERSION_MISMATCHES, + "Count of version mismatches encountered while reading an input stream", + TYPE_COUNTER), + STREAM_READ_SEEK_BACKWARD_OPERATIONS( + StreamStatisticNames.STREAM_READ_SEEK_BACKWARD_OPERATIONS, + "Count of executed seek operations which went backwards in a stream", + TYPE_COUNTER), + STREAM_READ_SEEK_BYTES_BACKWARDS( + StreamStatisticNames.STREAM_READ_SEEK_BYTES_BACKWARDS, + "Count of bytes moved backwards during seek operations" + + " in an input stream", + TYPE_COUNTER), + STREAM_READ_SEEK_BYTES_DISCARDED( + StreamStatisticNames.STREAM_READ_SEEK_BYTES_DISCARDED, + "Count of bytes read and discarded during seek() in an input stream", + TYPE_COUNTER), + STREAM_READ_SEEK_BYTES_SKIPPED( + StreamStatisticNames.STREAM_READ_SEEK_BYTES_SKIPPED, + "Count of bytes skipped during forward seek operations" + + " an input stream", + TYPE_COUNTER), + STREAM_READ_SEEK_FORWARD_OPERATIONS( + StreamStatisticNames.STREAM_READ_SEEK_FORWARD_OPERATIONS, + "Count of executed seek operations which went forward in" + + " an input stream", + TYPE_COUNTER), + STREAM_READ_SEEK_OPERATIONS( + StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS, + "Count of seek operations in an input stream", + TYPE_COUNTER), + STREAM_READ_SEEK_POLICY_CHANGED( + StreamStatisticNames.STREAM_READ_SEEK_POLICY_CHANGED, + "Count of times the seek policy was dynamically changed" + + " in an input stream", + TYPE_COUNTER), + STREAM_READ_TOTAL_BYTES( + StreamStatisticNames.STREAM_READ_TOTAL_BYTES, + "Total count of bytes read from an input stream", + TYPE_COUNTER), + + /* Stream Write statistics */ + + STREAM_WRITE_EXCEPTIONS( + StreamStatisticNames.STREAM_WRITE_EXCEPTIONS, + "Count of stream write failures reported", + TYPE_COUNTER), + STREAM_WRITE_EXCEPTIONS_COMPLETING_UPLOADS( + StreamStatisticNames.STREAM_WRITE_EXCEPTIONS_COMPLETING_UPLOADS, + "Count of failures when finalizing a multipart upload", + TYPE_COUNTER), + STREAM_WRITE_BLOCK_UPLOADS( + StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS, + "Count of block/partition uploads completed", + TYPE_COUNTER), + STREAM_WRITE_BLOCK_UPLOADS_ACTIVE( + StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS_ACTIVE, + "Count of block/partition uploads active", + TYPE_GAUGE), + STREAM_WRITE_BLOCK_UPLOADS_COMMITTED( + StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS_COMMITTED, + "Count of number of block uploads committed", + TYPE_COUNTER), + STREAM_WRITE_BLOCK_UPLOADS_ABORTED( + StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS_ABORTED, + "Count of number of block uploads aborted", + TYPE_COUNTER), + + STREAM_WRITE_BLOCK_UPLOADS_PENDING( + StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS_PENDING, + "Gauge of block/partitions uploads queued to be written", + TYPE_GAUGE), + STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING( + StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING, + "Gauge of data queued to be written", + TYPE_GAUGE), + STREAM_WRITE_TOTAL_TIME( + StreamStatisticNames.STREAM_WRITE_TOTAL_TIME, + "Count of total time taken for uploads to complete", + TYPE_COUNTER), + STREAM_WRITE_TOTAL_DATA(StreamStatisticNames.STREAM_WRITE_TOTAL_DATA, + "Count of total data uploaded", + TYPE_COUNTER), + STREAM_WRITE_BYTES( + StreamStatisticNames.STREAM_WRITE_BYTES, + "Count of bytes written to output stream" + + " (including all not yet uploaded)", + TYPE_COUNTER), + STREAM_WRITE_QUEUE_DURATION( + StreamStatisticNames.STREAM_WRITE_QUEUE_DURATION, + "Total queue duration of all block uploads", + TYPE_DURATION), + + /* committer stats */ COMMITTER_COMMITS_CREATED( "committer_commits_created", - "Number of files to commit created"), + "Count of files to commit created", + TYPE_COUNTER), COMMITTER_COMMITS_COMPLETED( "committer_commits_completed", - "Number of files committed"), + "Count of files committed", + TYPE_COUNTER), + COMMITTER_COMMIT_JOB( + "committer_commit_job", + "Duration Tracking of time to commit an entire job", + TYPE_DURATION), COMMITTER_JOBS_SUCCEEDED( "committer_jobs_completed", - "Number of successful jobs"), + "Count of successful jobs", + TYPE_COUNTER), COMMITTER_JOBS_FAILED( "committer_jobs_failed", - "Number of failed jobs"), + "Count of failed jobs", + TYPE_COUNTER), COMMITTER_TASKS_SUCCEEDED( "committer_tasks_completed", - "Number of successful tasks"), + "Count of successful tasks", + TYPE_COUNTER), COMMITTER_TASKS_FAILED( "committer_tasks_failed", - "Number of failed tasks"), + "Count of failed tasks", + TYPE_COUNTER), COMMITTER_BYTES_COMMITTED( "committer_bytes_committed", - "Amount of data committed"), + "Amount of data committed", + TYPE_COUNTER), COMMITTER_BYTES_UPLOADED( "committer_bytes_uploaded", - "Number of bytes uploaded duing commit operations"), + "Count of bytes uploaded duing commit operations", + TYPE_COUNTER), COMMITTER_COMMITS_FAILED( - "committer_commits_failed", - "Number of commits failed"), + "committer_commits"+ StoreStatisticNames.SUFFIX_FAILURES, + "Count of commits failed", + TYPE_COUNTER), COMMITTER_COMMITS_ABORTED( "committer_commits_aborted", - "Number of commits aborted"), + "Count of commits aborted", + TYPE_COUNTER), COMMITTER_COMMITS_REVERTED( "committer_commits_reverted", - "Number of commits reverted"), + "Count of commits reverted", + TYPE_COUNTER), COMMITTER_MAGIC_FILES_CREATED( "committer_magic_files_created", - "Number of files created under 'magic' paths"), + "Count of files created under 'magic' paths", + TYPE_COUNTER), + COMMITTER_MATERIALIZE_FILE( + "committer_materialize_file", + "Duration Tracking of time to materialize a file in job commit", + TYPE_DURATION), + COMMITTER_STAGE_FILE_UPLOAD( + "committer_stage_file_upload", + "Duration Tracking of files uploaded from a local staging path", + TYPE_DURATION), - // S3guard stats + /* S3guard stats */ S3GUARD_METADATASTORE_PUT_PATH_REQUEST( "s3guard_metadatastore_put_path_request", - "S3Guard metadata store put one metadata path request"), + "S3Guard metadata store put one metadata path request", + TYPE_COUNTER), S3GUARD_METADATASTORE_PUT_PATH_LATENCY( "s3guard_metadatastore_put_path_latency", - "S3Guard metadata store put one metadata path latency"), - S3GUARD_METADATASTORE_INITIALIZATION("s3guard_metadatastore_initialization", - "S3Guard metadata store initialization times"), + "S3Guard metadata store put one metadata path latency", + TYPE_QUANTILE), + S3GUARD_METADATASTORE_INITIALIZATION( + "s3guard_metadatastore_initialization", + "S3Guard metadata store initialization times", + TYPE_COUNTER), S3GUARD_METADATASTORE_RECORD_DELETES( "s3guard_metadatastore_record_deletes", - "S3Guard metadata store records deleted"), + "S3Guard metadata store records deleted", + TYPE_COUNTER), S3GUARD_METADATASTORE_RECORD_READS( "s3guard_metadatastore_record_reads", - "S3Guard metadata store records read"), + "S3Guard metadata store records read", + TYPE_COUNTER), S3GUARD_METADATASTORE_RECORD_WRITES( "s3guard_metadatastore_record_writes", - "S3Guard metadata store records written"), + "S3Guard metadata store records written", + TYPE_COUNTER), S3GUARD_METADATASTORE_RETRY("s3guard_metadatastore_retry", - "S3Guard metadata store retry events"), + "S3Guard metadata store retry events", + TYPE_COUNTER), S3GUARD_METADATASTORE_THROTTLED("s3guard_metadatastore_throttled", - "S3Guard metadata store throttled events"), + "S3Guard metadata store throttled events", + TYPE_COUNTER), S3GUARD_METADATASTORE_THROTTLE_RATE( "s3guard_metadatastore_throttle_rate", - "S3Guard metadata store throttle rate"), + "S3Guard metadata store throttle rate", + TYPE_QUANTILE), S3GUARD_METADATASTORE_AUTHORITATIVE_DIRECTORIES_UPDATED( "s3guard_metadatastore_authoritative_directories_updated", - "S3Guard metadata store authoritative directories updated from S3"), - - STORE_IO_THROTTLED("store_io_throttled", "Requests throttled and retried"), - STORE_IO_THROTTLE_RATE("store_io_throttle_rate", - "Rate of S3 request throttling"), - - DELEGATION_TOKENS_ISSUED("delegation_tokens_issued", - "Number of delegation tokens issued"), - - MULTIPART_INSTANTIATED( - "multipart_instantiated", - "Multipart Uploader Instantiated"), - MULTIPART_PART_PUT( - "multipart_part_put", - "Multipart Part Put Operation"), - MULTIPART_PART_PUT_BYTES( - "multipart_part_put_bytes", - "Multipart Part Put Bytes"), + "S3Guard metadata store authoritative directories updated from S3", + TYPE_COUNTER), + + + /* General Store operations */ + STORE_IO_REQUEST(StoreStatisticNames.STORE_IO_REQUEST, + "requests made of the remote store", + TYPE_COUNTER), + + STORE_IO_RETRY(StoreStatisticNames.STORE_IO_RETRY, + "retried requests made of the remote store", + TYPE_COUNTER), + + STORE_IO_THROTTLED( + StoreStatisticNames.STORE_IO_THROTTLED, + "Requests throttled and retried", + TYPE_COUNTER), + STORE_IO_THROTTLE_RATE( + StoreStatisticNames.STORE_IO_THROTTLE_RATE, + "Rate of S3 request throttling", + TYPE_QUANTILE), + + /* + * Delegation Token Operations. + */ + DELEGATION_TOKEN_ISSUED( + StoreStatisticNames.DELEGATION_TOKEN_ISSUED, + "Count of delegation tokens issued", + TYPE_DURATION), + + /* Multipart Upload API */ + + MULTIPART_UPLOAD_INSTANTIATED( + StoreStatisticNames.MULTIPART_UPLOAD_INSTANTIATED, + "Multipart Uploader Instantiated", + TYPE_COUNTER), + MULTIPART_UPLOAD_PART_PUT( + StoreStatisticNames.MULTIPART_UPLOAD_PART_PUT, + "Multipart Part Put Operation", + TYPE_COUNTER), + MULTIPART_UPLOAD_PART_PUT_BYTES( + StoreStatisticNames.MULTIPART_UPLOAD_PART_PUT_BYTES, + "Multipart Part Put Bytes", + TYPE_COUNTER), MULTIPART_UPLOAD_ABORTED( - "multipart_upload_aborted", - "Multipart Upload Aborted"), + StoreStatisticNames.MULTIPART_UPLOAD_ABORTED, + "Multipart Upload Aborted", + TYPE_COUNTER), MULTIPART_UPLOAD_ABORT_UNDER_PATH_INVOKED( - "multipart_upload_abort_under_path_invoked", - "Multipart Upload Abort Udner Path Invoked"), + StoreStatisticNames.MULTIPART_UPLOAD_ABORT_UNDER_PATH_INVOKED, + "Multipart Upload Abort Unner Path Invoked", + TYPE_COUNTER), MULTIPART_UPLOAD_COMPLETED( - "multipart_upload_completed", - "Multipart Upload Completed"), + StoreStatisticNames.MULTIPART_UPLOAD_COMPLETED, + "Multipart Upload Completed", + TYPE_COUNTER), MULTIPART_UPLOAD_STARTED( - "multipart_upload_started", - "Multipart Upload Started"); + StoreStatisticNames.MULTIPART_UPLOAD_STARTED, + "Multipart Upload Started", + TYPE_COUNTER); + + /** + * A map used to support the {@link #fromSymbol(String)} call. + */ private static final Map SYMBOL_MAP = new HashMap<>(Statistic.values().length); static { @@ -268,14 +524,28 @@ public enum Statistic { } } - Statistic(String symbol, String description) { + + /** + * Statistic definition. + * @param symbol name + * @param description description. + * @param type type + */ + Statistic(String symbol, String description, StatisticTypeEnum type) { this.symbol = symbol; this.description = description; + this.type = type; } + /** Statistic name. */ private final String symbol; + + /** Statistic description. */ private final String description; + /** Statistic type. */ + private final StatisticTypeEnum type; + public String getSymbol() { return symbol; } @@ -302,4 +572,12 @@ public String getDescription() { public String toString() { return symbol; } + + /** + * What type is this statistic? + * @return the type. + */ + public StatisticTypeEnum getType() { + return type; + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java index 6b7c5699a1760..e75c09c9f4c52 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java @@ -50,10 +50,12 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext; import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; import org.apache.hadoop.fs.s3a.s3guard.S3Guard; import org.apache.hadoop.fs.s3a.select.SelectBinding; import org.apache.hadoop.util.DurationInfo; +import org.apache.hadoop.util.functional.CallableRaisingIOE; import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkArgument; import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull; @@ -108,17 +110,26 @@ public class WriteOperationHelper implements WriteOperations { /** Bucket of the owner FS. */ private final String bucket; + /** + * statistics context. + */ + private final S3AStatisticsContext statisticsContext; + /** * Constructor. * @param owner owner FS creating the helper * @param conf Configuration object + * @param statisticsContext statistics context * */ - protected WriteOperationHelper(S3AFileSystem owner, Configuration conf) { + protected WriteOperationHelper(S3AFileSystem owner, + Configuration conf, + S3AStatisticsContext statisticsContext) { this.owner = owner; this.invoker = new Invoker(new S3ARetryPolicy(conf), this::operationRetried); this.conf = conf; + this.statisticsContext = statisticsContext; bucket = owner.getBucket(); } @@ -138,19 +149,19 @@ void operationRetried(String text, Exception ex, int retries, /** * Execute a function with retry processing. + * @param type of return value * @param action action to execute (used in error messages) * @param path path of work (used in error messages) * @param idempotent does the operation have semantics * which mean that it can be retried even if was already executed? * @param operation operation to execute - * @param type of return value * @return the result of the call * @throws IOException any IOE raised, or translated exception */ public T retry(String action, String path, boolean idempotent, - Invoker.Operation operation) + CallableRaisingIOE operation) throws IOException { return invoker.retry(action, path, idempotent, operation); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java index 95cbd7e70b73b..0b336142ccc9f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java @@ -42,6 +42,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; +import org.apache.hadoop.util.functional.CallableRaisingIOE; /** * Operations to update the store. @@ -56,19 +57,19 @@ public interface WriteOperations { /** * Execute a function with retry processing. + * @param type of return value * @param action action to execute (used in error messages) * @param path path of work (used in error messages) * @param idempotent does the operation have semantics * which mean that it can be retried even if was already executed? * @param operation operation to execute - * @param type of return value * @return the result of the call * @throws IOException any IOE raised, or translated exception */ T retry(String action, String path, boolean idempotent, - Invoker.Operation operation) + CallableRaisingIOE operation) throws IOException; /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/S3ADelegationTokens.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/S3ADelegationTokens.java index db149de4122d1..d8fc8b8396481 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/S3ADelegationTokens.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/S3ADelegationTokens.java @@ -34,9 +34,9 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.s3a.AWSCredentialProviderList; -import org.apache.hadoop.fs.s3a.S3AInstrumentation; import org.apache.hadoop.fs.s3a.auth.RoleModel; import org.apache.hadoop.fs.s3a.impl.StoreContext; +import org.apache.hadoop.fs.s3a.statistics.DelegationTokenStatistics; import org.apache.hadoop.io.Text; import org.apache.hadoop.security.Credentials; import org.apache.hadoop.security.UserGroupInformation; @@ -45,6 +45,8 @@ import org.apache.hadoop.service.ServiceOperations; import org.apache.hadoop.util.DurationInfo; +import static org.apache.hadoop.fs.s3a.Statistic.DELEGATION_TOKEN_ISSUED; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration; import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkArgument; import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkState; import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.DEFAULT_DELEGATION_TOKEN_BINDING; @@ -135,9 +137,9 @@ public class S3ADelegationTokens extends AbstractDTService { AWSPolicyProvider.AccessLevel.WRITE); /** - * Statistics for the owner FS. + * Statistics for the operations. */ - private S3AInstrumentation.DelegationTokenStatistics stats; + private DelegationTokenStatistics stats; /** * Name of the token binding as extracted from token kind; used for @@ -428,8 +430,10 @@ public Token createDelegationToken( try(DurationInfo ignored = new DurationInfo(LOG, DURATION_LOG_AT_INFO, "Creating New Delegation Token", tokenBinding.getKind())) { - Token token - = tokenBinding.createDelegationToken(rolePolicy, encryptionSecrets, renewer); + Token token = trackDuration(stats, + DELEGATION_TOKEN_ISSUED.getSymbol(), () -> + tokenBinding.createDelegationToken(rolePolicy, + encryptionSecrets, renewer)); if (token != null) { token.setService(service); noteTokenCreated(token); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java index 771d6064c0cfe..f73689979ec25 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java @@ -47,6 +47,10 @@ import org.apache.hadoop.fs.s3a.commit.files.PendingSet; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; import org.apache.hadoop.fs.s3a.commit.files.SuccessData; +import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.JobID; import org.apache.hadoop.mapreduce.JobStatus; @@ -60,6 +64,7 @@ import static org.apache.hadoop.fs.s3a.Constants.THREAD_POOL_SHUTDOWN_DELAY_SECONDS; import static org.apache.hadoop.fs.s3a.Invoker.ignoreIOExceptions; import static org.apache.hadoop.fs.s3a.S3AUtils.*; +import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_COMMIT_JOB; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; import static org.apache.hadoop.fs.s3a.commit.CommitUtils.*; import static org.apache.hadoop.fs.s3a.commit.CommitUtilsWithMR.*; @@ -67,6 +72,7 @@ import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.FS_S3A_COMMITTER_UUID; import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.FS_S3A_COMMITTER_UUID_SOURCE; import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.SPARK_WRITE_UUID; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfInvocation; /** * Abstract base class for S3A committers; allows for any commonality @@ -94,7 +100,8 @@ * created by a few tasks, and the HTTP connection pool in the S3A * committer was large enough for more all the parallel POST requests. */ -public abstract class AbstractS3ACommitter extends PathOutputCommitter { +public abstract class AbstractS3ACommitter extends PathOutputCommitter + implements IOStatisticsSource { private static final Logger LOG = LoggerFactory.getLogger(AbstractS3ACommitter.class); @@ -166,6 +173,8 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter { /** Should a job marker be created? */ private final boolean createJobMarker; + private final CommitterStatistics committerStatistics; + /** * Create a committer. * This constructor binds the destination directory and configuration, but @@ -197,7 +206,9 @@ protected AbstractS3ACommitter( this.createJobMarker = context.getConfiguration().getBoolean( CREATE_SUCCESSFUL_JOB_OUTPUT_DIR_MARKER, DEFAULT_CREATE_SUCCESSFUL_JOB_DIR_MARKER); - this.commitOperations = new CommitOperations(fs); + // the statistics are shared between this committer and its operations. + this.committerStatistics = fs.newCommitterStatistics(); + this.commitOperations = new CommitOperations(fs, committerStatistics); } /** @@ -437,7 +448,12 @@ protected void maybeCreateSuccessMarkerFromCommits(JobContext context, // The list of committed objects in pending is size limited in // ActiveCommit.uploadCommitted. filenames.addAll(pending.committedObjects); - maybeCreateSuccessMarker(context, filenames); + // load in all the pending statistics + IOStatisticsSnapshot snapshot = new IOStatisticsSnapshot( + pending.getIOStatistics()); + snapshot.aggregate(getIOStatistics()); + + maybeCreateSuccessMarker(context, filenames, snapshot); } /** @@ -448,10 +464,12 @@ protected void maybeCreateSuccessMarkerFromCommits(JobContext context, * PUT up a the contents of a {@link SuccessData} file. * @param context job context * @param filenames list of filenames. + * @param ioStatistics any IO Statistics to include * @throws IOException IO failure */ protected void maybeCreateSuccessMarker(JobContext context, - List filenames) + List filenames, + final IOStatisticsSnapshot ioStatistics) throws IOException { if (createJobMarker) { // create a success data structure and then save it @@ -465,6 +483,7 @@ protected void maybeCreateSuccessMarker(JobContext context, successData.setTimestamp(now.getTime()); successData.setDate(now.toString()); successData.setFilenames(filenames); + successData.getIOStatistics().aggregate(ioStatistics); commitOperations.createSuccessMarker(getOutputPath(), successData, true); } } @@ -644,6 +663,7 @@ private void loadAndCommit( activeCommit.uploadCommitted( commit.getDestinationKey(), commit.getLength()); }); + activeCommit.pendingsetCommitted(pendingSet.getIOStatistics()); } } @@ -728,8 +748,9 @@ protected CommitOperations.CommitContext initiateCommitOperation() protected void commitJobInternal(JobContext context, ActiveCommit pending) throws IOException { - - commitPendingUploads(context, pending); + trackDurationOfInvocation(committerStatistics, + COMMITTER_COMMIT_JOB.getSymbol(), + () -> commitPendingUploads(context, pending)); } @Override @@ -1175,6 +1196,11 @@ protected void abortPendingUploads( } } + @Override + public IOStatistics getIOStatistics() { + return committerStatistics.getIOStatistics(); + } + /** * Scan for active uploads and list them along with a warning message. * Errors are ignored. @@ -1385,6 +1411,13 @@ public static class ActiveCommit { */ private long committedBytes; + /** + * Aggregate statistics of all supplied by + * committed uploads. + */ + private final IOStatisticsSnapshot ioStatistics = + new IOStatisticsSnapshot(); + /** * Construct from a source FS and list of files. * @param sourceFS filesystem containing the list of pending files @@ -1433,7 +1466,8 @@ public FileSystem getSourceFS() { * @param key key of the committed object. * @param size size in bytes. */ - public synchronized void uploadCommitted(String key, long size) { + public synchronized void uploadCommitted(String key, + long size) { if (committedObjects.size() < SUCCESS_MARKER_FILE_LIMIT) { committedObjects.add( key.startsWith("/") ? key : ("/" + key)); @@ -1442,6 +1476,19 @@ public synchronized void uploadCommitted(String key, long size) { committedBytes += size; } + /** + * Callback when a pendingset has been committed, + * including any source statistics. + * @param sourceStatistics any source statistics + */ + public void pendingsetCommitted(final IOStatistics sourceStatistics) { + ioStatistics.aggregate(sourceStatistics); + } + + public IOStatisticsSnapshot getIOStatistics() { + return ioStatistics; + } + public synchronized List getCommittedObjects() { return committedObjects; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java index 11eeee9e94096..c9fb3806b2c5a 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java @@ -45,7 +45,6 @@ import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.s3a.S3AFileSystem; -import org.apache.hadoop.fs.s3a.S3AInstrumentation; import org.apache.hadoop.fs.s3a.S3AUtils; import org.apache.hadoop.fs.s3a.WriteOperationHelper; import org.apache.hadoop.fs.s3a.commit.files.PendingSet; @@ -53,13 +52,21 @@ import org.apache.hadoop.fs.s3a.commit.files.SuccessData; import org.apache.hadoop.fs.s3a.impl.InternalConstants; import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; +import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics; +import org.apache.hadoop.fs.statistics.DurationTracker; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.util.DurationInfo; import org.apache.hadoop.util.Progressable; +import static java.util.Objects.requireNonNull; import static org.apache.hadoop.fs.s3a.S3AUtils.*; +import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_MATERIALIZE_FILE; +import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_STAGE_FILE_UPLOAD; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; import static org.apache.hadoop.fs.s3a.Constants.*; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration; /** * The implementation of the various actions a committer needs. @@ -71,7 +78,7 @@ * duplicate that work. * */ -public class CommitOperations { +public class CommitOperations implements IOStatisticsSource { private static final Logger LOG = LoggerFactory.getLogger( CommitOperations.class); @@ -81,7 +88,7 @@ public class CommitOperations { private final S3AFileSystem fs; /** Statistics. */ - private final S3AInstrumentation.CommitterStatistics statistics; + private final CommitterStatistics statistics; /** * Write operations for the destination fs. @@ -105,9 +112,18 @@ public class CommitOperations { * @param fs FS to bind to */ public CommitOperations(S3AFileSystem fs) { - Preconditions.checkArgument(fs != null, "null fs"); - this.fs = fs; - statistics = fs.newCommitterStatistics(); + this(requireNonNull(fs), fs.newCommitterStatistics()); + } + + /** + * Instantiate. + * @param fs FS to bind to + * @param committerStatistics committer statistics + */ + public CommitOperations(S3AFileSystem fs, + CommitterStatistics committerStatistics) { + this.fs = requireNonNull(fs); + statistics = requireNonNull(committerStatistics); writeOperations = fs.getWriteOperationHelper(); } @@ -128,10 +144,15 @@ public String toString() { } /** @return statistics. */ - protected S3AInstrumentation.CommitterStatistics getStatistics() { + protected CommitterStatistics getStatistics() { return statistics; } + @Override + public IOStatistics getIOStatistics() { + return statistics.getIOStatistics(); + } + /** * Commit the operation, throwing an exception on any failure. * @param commit commit to execute @@ -166,7 +187,8 @@ private MaybeIOE commit( commit.validate(); destKey = commit.getDestinationKey(); - long l = innerCommit(commit, operationState); + long l = trackDuration(statistics, COMMITTER_MATERIALIZE_FILE.getSymbol(), + () -> innerCommit(commit, operationState)); LOG.debug("Successful commit of file length {}", l); outcome = MaybeIOE.NONE; statistics.commitCompleted(commit.getLength()); @@ -449,7 +471,7 @@ public void revertCommit(SinglePendingCommit commit, * @return a pending upload entry * @throws IOException failure */ - public SinglePendingCommit uploadFileToPendingCommit(File localFile, + public SinglePendingCommit uploadFileToPendingCommit(File localFile, Path destPath, String partition, long uploadPartSize, @@ -466,7 +488,11 @@ public SinglePendingCommit uploadFileToPendingCommit(File localFile, String destKey = fs.pathToKey(destPath); String uploadId = null; + // flag to indicate to the finally clause that the operation + // failed. it is cleared as the last action in the try block. boolean threw = true; + final DurationTracker tracker = statistics.trackDuration( + COMMITTER_STAGE_FILE_UPLOAD.getSymbol()); try (DurationInfo d = new DurationInfo(LOG, "Upload staged file from %s to %s", localFile.getAbsolutePath(), @@ -507,6 +533,7 @@ public SinglePendingCommit uploadFileToPendingCommit(File localFile, LOG.debug("File size is {}, number of parts to upload = {}", length, numParts); for (int partNumber = 1; partNumber <= numParts; partNumber += 1) { + progress.progress(); long size = Math.min(length - offset, uploadPartSize); UploadPartRequest part; part = writeOperations.newUploadPartRequest( @@ -525,7 +552,7 @@ public SinglePendingCommit uploadFileToPendingCommit(File localFile, commitData.bindCommitData(parts); statistics.commitUploaded(length); - progress.progress(); + // clear the threw flag. threw = false; return commitData; } finally { @@ -536,6 +563,11 @@ public SinglePendingCommit uploadFileToPendingCommit(File localFile, LOG.error("Failed to abort upload {} to {}", uploadId, destKey, e); } } + if (threw) { + tracker.failed(); + } + // close tracker and so report statistics of success/failure + tracker.close(); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java index 7f9dadf06f320..41f95c8620692 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java @@ -27,6 +27,7 @@ import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.Statistic; import org.apache.hadoop.fs.s3a.commit.magic.MagicCommitTracker; +import org.apache.hadoop.fs.s3a.impl.StoreContext; import static org.apache.hadoop.fs.s3a.commit.MagicCommitPaths.*; @@ -50,6 +51,8 @@ public class MagicCommitIntegration { private final S3AFileSystem owner; private final boolean magicCommitEnabled; + private final StoreContext storeContext; + /** * Instantiate. * @param owner owner class @@ -59,6 +62,7 @@ public MagicCommitIntegration(S3AFileSystem owner, boolean magicCommitEnabled) { this.owner = owner; this.magicCommitEnabled = magicCommitEnabled; + this.storeContext = owner.createStoreContext(); } /** @@ -94,10 +98,10 @@ public PutTracker createTracker(Path path, String key) { if (isMagicCommitPath(elements)) { final String destKey = keyOfFinalDestination(elements, key); String pendingsetPath = key + CommitConstants.PENDING_SUFFIX; - owner.getInstrumentation() - .incrementCounter(Statistic.COMMITTER_MAGIC_FILES_CREATED, 1); + storeContext.incrementStatistic( + Statistic.COMMITTER_MAGIC_FILES_CREATED); tracker = new MagicCommitTracker(path, - owner.getBucket(), + storeContext.getBucket(), key, destKey, pendingsetPath, diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/PutTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/PutTracker.java index bbffef3ad221c..10440f77e7277 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/PutTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/PutTracker.java @@ -24,6 +24,7 @@ import com.amazonaws.services.s3.model.PartETag; import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.fs.statistics.IOStatistics; /** * Multipart put tracker. @@ -68,6 +69,7 @@ public boolean outputImmediatelyVisible() { * @param uploadId Upload ID * @param parts list of parts * @param bytesWritten bytes written + * @param iostatistics nullable IO statistics * @return true if the commit is to be initiated immediately. * False implies the output stream does not need to worry about * what happens. @@ -75,7 +77,8 @@ public boolean outputImmediatelyVisible() { */ public boolean aboutToComplete(String uploadId, List parts, - long bytesWritten) + long bytesWritten, + final IOStatistics iostatistics) throws IOException { return true; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PendingSet.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PendingSet.java index 8ad0342105837..fd7341025667a 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PendingSet.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PendingSet.java @@ -27,6 +27,7 @@ import java.util.Map; import java.util.Set; +import com.fasterxml.jackson.annotation.JsonProperty; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -36,6 +37,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.commit.ValidationFailure; +import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; import org.apache.hadoop.util.JsonSerialization; import static org.apache.hadoop.fs.s3a.commit.CommitUtils.validateCollectionClass; @@ -45,11 +48,22 @@ * Persistent format for multiple pending commits. * Contains 0 or more {@link SinglePendingCommit} entries; validation logic * checks those values on load. + *

+ * The statistics published through the {@link IOStatisticsSource} + * interface are the static ones marshalled with the commit data; + * they may be empty. + *

+ *

+ * As single commits are added via {@link #add(SinglePendingCommit)}, + * any statistics from those commits are merged into the aggregate + * statistics, and those of the single commit cleared. + *

*/ @SuppressWarnings("unused") @InterfaceAudience.Private @InterfaceStability.Unstable -public class PendingSet extends PersistentCommitData { +public class PendingSet extends PersistentCommitData + implements IOStatisticsSource { private static final Logger LOG = LoggerFactory.getLogger(PendingSet.class); /** @@ -57,7 +71,7 @@ public class PendingSet extends PersistentCommitData { * If this is changed the value of {@link #serialVersionUID} will change, * to avoid deserialization problems. */ - public static final int VERSION = 2; + public static final int VERSION = 3; /** * Serialization ID: {@value}. @@ -81,6 +95,12 @@ public class PendingSet extends PersistentCommitData { */ private final Map extraData = new HashMap<>(0); + /** + * IOStatistics. + */ + @JsonProperty("iostatistics") + private IOStatisticsSnapshot iostats = new IOStatisticsSnapshot(); + public PendingSet() { this(0); } @@ -133,6 +153,12 @@ public static PendingSet load(FileSystem fs, FileStatus status) */ public void add(SinglePendingCommit commit) { commits.add(commit); + // add any statistics. + IOStatisticsSnapshot st = commit.getIOStatistics(); + if (st != null) { + iostats.aggregate(st); + st.clear(); + } } /** @@ -225,4 +251,12 @@ public void setJobId(String jobId) { this.jobId = jobId; } + @Override + public IOStatisticsSnapshot getIOStatistics() { + return iostats; + } + + public void setIOStatistics(final IOStatisticsSnapshot ioStatistics) { + this.iostats = ioStatistics; + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java index ee7c7b236c6cb..021cc7b34b1f5 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java @@ -31,6 +31,8 @@ import java.util.Map; import com.amazonaws.services.s3.model.PartETag; +import com.fasterxml.jackson.annotation.JsonProperty; + import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.apache.commons.lang3.StringUtils; @@ -39,6 +41,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.commit.ValidationFailure; +import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; import org.apache.hadoop.util.JsonSerialization; import static org.apache.hadoop.fs.s3a.commit.CommitUtils.validateCollectionClass; @@ -46,20 +50,25 @@ import static org.apache.hadoop.util.StringUtils.join; /** - * This is the serialization format for uploads yet to be committerd. - * + * This is the serialization format for uploads yet to be committed. + *

* It's marked as {@link Serializable} so that it can be passed in RPC * calls; for this to work it relies on the fact that java.io ArrayList * and LinkedList are serializable. If any other list type is used for etags, * it must also be serialized. Jackson expects lists, and it is used * to persist to disk. - * + *

+ *

+ * The statistics published through the {@link IOStatisticsSource} + * interface are the static ones marshalled with the commit data; + * they may be empty. + *

*/ @SuppressWarnings("unused") @InterfaceAudience.Private @InterfaceStability.Unstable public class SinglePendingCommit extends PersistentCommitData - implements Iterable { + implements Iterable, IOStatisticsSource { /** * Serialization ID: {@value}. @@ -113,6 +122,12 @@ public class SinglePendingCommit extends PersistentCommitData */ private Map extraData = new HashMap<>(0); + /** + * IOStatistics. + */ + @JsonProperty("iostatistics") + private IOStatisticsSnapshot iostats = new IOStatisticsSnapshot(); + /** Destination file size. */ private long length; @@ -439,4 +454,12 @@ public void setLength(long length) { this.length = length; } + @Override + public IOStatisticsSnapshot getIOStatistics() { + return iostats; + } + + public void setIOStatistics(final IOStatisticsSnapshot ioStatistics) { + this.iostats = ioStatistics; + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SuccessData.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SuccessData.java index b7509d6714bd3..4500e1429e447 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SuccessData.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SuccessData.java @@ -25,6 +25,7 @@ import java.util.List; import java.util.Map; +import com.fasterxml.jackson.annotation.JsonProperty; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -34,6 +35,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.commit.ValidationFailure; +import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; import org.apache.hadoop.util.JsonSerialization; /** @@ -62,9 +65,18 @@ @SuppressWarnings("unused") @InterfaceAudience.Private @InterfaceStability.Unstable -public class SuccessData extends PersistentCommitData { +public class SuccessData extends PersistentCommitData + implements IOStatisticsSource { + private static final Logger LOG = LoggerFactory.getLogger(SuccessData.class); + /** + * Supported version value: {@value}. + * If this is changed the value of {@link #serialVersionUID} will change, + * to avoid deserialization problems. + */ + public static final int VERSION = 1; + /** * Serialization ID: {@value}. */ @@ -75,7 +87,7 @@ public class SuccessData extends PersistentCommitData { * any other manifests: {@value}. */ public static final String NAME - = "org.apache.hadoop.fs.s3a.commit.files.SuccessData/1"; + = "org.apache.hadoop.fs.s3a.commit.files.SuccessData/" + VERSION; /** * Name of file; includes version marker. @@ -126,6 +138,12 @@ public class SuccessData extends PersistentCommitData { */ private List filenames = new ArrayList<>(0); + /** + * IOStatistics. + */ + @JsonProperty("iostatistics") + private IOStatisticsSnapshot iostats = new IOStatisticsSnapshot(); + @Override public void validate() throws ValidationFailure { ValidationFailure.verify(name != null, @@ -350,4 +368,13 @@ public String getJobIdSource() { public void setJobIdSource(final String jobIdSource) { this.jobIdSource = jobIdSource; } + + @Override + public IOStatisticsSnapshot getIOStatistics() { + return iostats; + } + + public void setIOStatistics(final IOStatisticsSnapshot ioStatistics) { + this.iostats = ioStatistics; + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java index 13967645df646..0f1a0a6534df9 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java @@ -34,6 +34,8 @@ import org.apache.hadoop.fs.s3a.WriteOperationHelper; import org.apache.hadoop.fs.s3a.commit.PutTracker; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; /** * Put tracker for Magic commits. @@ -102,6 +104,7 @@ public boolean outputImmediatelyVisible() { * @param uploadId Upload ID * @param parts list of parts * @param bytesWritten bytes written + * @param iostatistics nullable IO statistics * @return false, indicating that the commit must fail. * @throws IOException any IO problem. * @throws IllegalArgumentException bad argument @@ -109,7 +112,8 @@ public boolean outputImmediatelyVisible() { @Override public boolean aboutToComplete(String uploadId, List parts, - long bytesWritten) + long bytesWritten, + final IOStatistics iostatistics) throws IOException { Preconditions.checkArgument(StringUtils.isNotEmpty(uploadId), "empty/null upload ID: "+ uploadId); @@ -117,6 +121,15 @@ public boolean aboutToComplete(String uploadId, "No uploaded parts list"); Preconditions.checkArgument(!parts.isEmpty(), "No uploaded parts to save"); + + // put a 0-byte file with the name of the original under-magic path + PutObjectRequest originalDestPut = writer.createPutObjectRequest( + originalDestKey, + new ByteArrayInputStream(EMPTY), + 0); + writer.uploadObject(originalDestPut); + + // build the commit summary SinglePendingCommit commitData = new SinglePendingCommit(); commitData.touch(System.currentTimeMillis()); commitData.setDestinationKey(getDestKey()); @@ -126,6 +139,8 @@ public boolean aboutToComplete(String uploadId, commitData.setText(""); commitData.setLength(bytesWritten); commitData.bindCommitData(parts); + commitData.setIOStatistics( + new IOStatisticsSnapshot(iostatistics)); byte[] bytes = commitData.toBytes(); LOG.info("Uncommitted data pending to file {};" + " commit metadata for {} parts in {}. sixe: {} byte(s)", @@ -138,12 +153,6 @@ public boolean aboutToComplete(String uploadId, bytes.length); writer.uploadObject(put); - // now put a 0-byte file with the name of the original under-magic path - PutObjectRequest originalDestPut = writer.createPutObjectRequest( - originalDestKey, - new ByteArrayInputStream(EMPTY), - 0); - writer.uploadObject(originalDestPut); return false; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java index b330cee4e6b83..34bbfd4ed76e2 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java @@ -37,6 +37,7 @@ import org.apache.hadoop.fs.s3a.commit.CommitUtilsWithMR; import org.apache.hadoop.fs.s3a.commit.files.PendingSet; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; +import org.apache.hadoop.fs.statistics.IOStatisticsLogging; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.TaskAttemptID; @@ -47,6 +48,7 @@ import static org.apache.hadoop.fs.s3a.commit.CommitUtils.*; import static org.apache.hadoop.fs.s3a.commit.MagicCommitPaths.*; import static org.apache.hadoop.fs.s3a.commit.CommitUtilsWithMR.*; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.demandStringifyIOStatistics; /** * This is a dedicated committer which requires the "magic" directory feature @@ -169,6 +171,8 @@ public void commitTask(TaskAttemptContext context) throws IOException { destroyThreadPool(); } getCommitOperations().taskCompleted(true); + LOG.debug("aggregate statistics\n{}", + demandStringifyIOStatistics(getIOStatistics())); } /** @@ -213,6 +217,8 @@ private PendingSet innerCommitTask( taskAttemptID.getTaskID().toString() + CommitConstants.PENDINGSET_SUFFIX); LOG.info("Saving work of {} to {}", taskAttemptID, taskOutcomePath); + LOG.debug("task statistics\n{}", + IOStatisticsLogging.demandStringifyIOStatisticsSource(pendingSet)); try { // We will overwrite if there exists a pendingSet file already pendingSet.save(getDestFS(), taskOutcomePath, true); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ActiveOperationContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ActiveOperationContext.java new file mode 100644 index 0000000000000..3ad2bbff3ba92 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ActiveOperationContext.java @@ -0,0 +1,92 @@ +/* + * 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 javax.annotation.Nullable; + +import java.util.Objects; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; +import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext; + +/** + * Context for any active operation. + */ +public class ActiveOperationContext { + + /** + * An operation ID; currently just for logging...proper tracing needs more. + */ + private final long operationId; + + /** + * Statistics context. + */ + private final S3AStatisticsContext statisticsContext; + + /** + * S3Guard bulk operation state, if (currently) set. + */ + @Nullable private BulkOperationState bulkOperationState; + + public ActiveOperationContext( + final long operationId, + final S3AStatisticsContext statisticsContext, + @Nullable final BulkOperationState bulkOperationState) { + this.operationId = operationId; + this.statisticsContext = Objects.requireNonNull(statisticsContext, + "null statistics context"); + this.bulkOperationState = bulkOperationState; + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder( + "ActiveOperation{"); + sb.append("operationId=").append(operationId); + sb.append(", bulkOperationState=").append(bulkOperationState); + sb.append('}'); + return sb.toString(); + } + + @Nullable + public BulkOperationState getBulkOperationState() { + return bulkOperationState; + } + + public long getOperationId() { + return operationId; + } + + public S3AStatisticsContext getS3AStatisticsContext() { + return statisticsContext; + } + + private static final AtomicLong NEXT_OPERATION_ID = new AtomicLong(0); + + /** + * Create an operation ID. The nature of it should be opaque. + * @return an ID for the constructor. + */ + protected static long newOperationId() { + return NEXT_OPERATION_ID.incrementAndGet(); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteRetryHandler.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteRetryHandler.java index b2c1cc6271ec7..4169a9899cb15 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteRetryHandler.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteRetryHandler.java @@ -26,9 +26,9 @@ import org.slf4j.LoggerFactory; import org.apache.hadoop.fs.s3a.AWSClientIOException; -import org.apache.hadoop.fs.s3a.S3AInstrumentation; import org.apache.hadoop.fs.s3a.S3AStorageStatistics; import org.apache.hadoop.fs.s3a.Statistic; +import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext; import static org.apache.hadoop.fs.s3a.S3AUtils.isThrottleException; import static org.apache.hadoop.fs.s3a.Statistic.IGNORED_ERRORS; @@ -53,7 +53,7 @@ public class BulkDeleteRetryHandler extends AbstractStoreOperation { */ public static final String XML_PARSE_BROKEN = "Failed to parse XML document"; - private final S3AInstrumentation instrumentation; + private final S3AStatisticsContext instrumentation; private final S3AStorageStatistics storageStatistics; @@ -84,7 +84,6 @@ protected void incrementStatistic(Statistic statistic) { */ protected void incrementStatistic(Statistic statistic, long count) { instrumentation.incrementCounter(statistic, count); - storageStatistics.incrementCounter(statistic, count); } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CallableSupplier.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CallableSupplier.java index 1c61d30b08a60..b788f507735a0 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CallableSupplier.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CallableSupplier.java @@ -20,6 +20,7 @@ import javax.annotation.Nullable; import java.io.IOException; +import java.io.UncheckedIOException; import java.util.List; import java.util.concurrent.Callable; import java.util.concurrent.CancellationException; @@ -31,7 +32,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.fs.impl.WrappedIOException; import org.apache.hadoop.util.DurationInfo; import static org.apache.hadoop.fs.impl.FutureIOSupport.raiseInnerCause; @@ -63,9 +63,9 @@ public Object get() { } catch (RuntimeException e) { throw e; } catch (IOException e) { - throw new WrappedIOException(e); + throw new UncheckedIOException(e); } catch (Exception e) { - throw new WrappedIOException(new IOException(e)); + throw new UncheckedIOException(new IOException(e)); } } @@ -73,7 +73,7 @@ public Object get() { * Submit a callable into a completable future. * RTEs are rethrown. * Non RTEs are caught and wrapped; IOExceptions to - * {@link WrappedIOException} instances. + * {@code RuntimeIOException} instances. * @param executor executor. * @param call call to invoke * @param type diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java index bb025c4ed0764..48050563cde2b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java @@ -18,8 +18,6 @@ package org.apache.hadoop.fs.s3a.impl; -import java.util.concurrent.atomic.AtomicLong; - import com.amazonaws.AmazonServiceException; import com.amazonaws.SdkBaseException; import com.amazonaws.services.s3.model.CopyObjectRequest; @@ -39,6 +37,7 @@ import org.apache.hadoop.fs.s3a.NoVersionAttributeException; import org.apache.hadoop.fs.s3a.RemoteFileChangedException; import org.apache.hadoop.fs.s3a.S3ObjectAttributes; +import org.apache.hadoop.fs.s3a.statistics.ChangeTrackerStatistics; import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull; @@ -72,7 +71,7 @@ public class ChangeTracker { * Mismatch counter; expected to be wired up to StreamStatistics except * during testing. */ - private final AtomicLong versionMismatches; + private final ChangeTrackerStatistics versionMismatches; /** * Revision identifier (e.g. eTag or versionId, depending on change @@ -90,7 +89,7 @@ public class ChangeTracker { */ public ChangeTracker(final String uri, final ChangeDetectionPolicy policy, - final AtomicLong versionMismatches, + final ChangeTrackerStatistics versionMismatches, final S3ObjectAttributes s3ObjectAttributes) { this.policy = checkNotNull(policy); this.uri = uri; @@ -111,8 +110,8 @@ public ChangeDetectionPolicy.Source getSource() { } @VisibleForTesting - public AtomicLong getVersionMismatches() { - return versionMismatches; + public long getVersionMismatches() { + return versionMismatches.getVersionMismatches(); } /** @@ -177,7 +176,7 @@ public void processResponse(final S3Object object, if (revisionId != null) { // the requirements of the change detection policy wasn't met: the // object was not returned. - versionMismatches.incrementAndGet(); + versionMismatches.versionMismatchError(); throw new RemoteFileChangedException(uri, operation, String.format(CHANGE_REPORTED_BY_S3 + " during %s" @@ -235,7 +234,7 @@ public void processException(SdkBaseException e, String operation) throws // This isn't really going to be hit due to // https://github.com/aws/aws-sdk-java/issues/1644 if (serviceException.getStatusCode() == SC_PRECONDITION_FAILED) { - versionMismatches.incrementAndGet(); + versionMismatches.versionMismatchError(); throw new RemoteFileChangedException(uri, operation, String.format( RemoteFileChangedException.PRECONDITIONS_FAILED + " on %s." @@ -292,10 +291,10 @@ private void processNewRevision(final String newRevisionId, uri, pos, operation, - versionMismatches.get()); + versionMismatches.getVersionMismatches()); if (pair.left) { // an mismatch has occurred: note it. - versionMismatches.incrementAndGet(); + versionMismatches.versionMismatchError(); } if (pair.right != null) { // there's an exception to raise: do it diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ErrorTranslation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ErrorTranslation.java index d1baf3c898735..f7e06413a3761 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ErrorTranslation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ErrorTranslation.java @@ -54,6 +54,18 @@ public static boolean isUnknownBucket(AmazonServiceException e) { && AwsErrorCodes.E_NO_SUCH_BUCKET.equals(e.getErrorCode()); } + /** + * Does this exception indicate that a reference to an object + * returned a 404. Unknown bucket errors do not match this + * predicate. + * @param e exception. + * @return true if the status code and error code mean that the + * HEAD request returned 404 but the bucket was there. + */ + public static boolean isObjectNotFound(AmazonServiceException e) { + return e.getStatusCode() == SC_404 && !isUnknownBucket(e); + } + /** * AWS error codes explicitly recognized and processes specially; * kept in their own class for isolation. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java index 9ecc1a93c60e2..0e8d864d4c40c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java @@ -110,4 +110,11 @@ private InternalConstants() { * Value: {@value}. */ public static final int DEFAULT_UPLOAD_PART_COUNT_LIMIT = 10000; + + /** + * Flag to enable AWS Statistics binding. As this is triggering + * problems related to region/endpoint setup, it is currently + * disabled. + */ + public static final boolean AWS_SDK_METRICS_ENABLED = false; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ListingOperationCallbacks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ListingOperationCallbacks.java index e5f9f7d9808ea..b0ebf1ca16975 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ListingOperationCallbacks.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ListingOperationCallbacks.java @@ -29,6 +29,7 @@ import org.apache.hadoop.fs.s3a.S3ListRequest; import org.apache.hadoop.fs.s3a.S3ListResult; import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider; +import org.apache.hadoop.fs.statistics.DurationTrackerFactory; /** * These are all the callbacks which @@ -44,12 +45,14 @@ public interface ListingOperationCallbacks { * * Retry policy: retry untranslated. * @param request request to initiate + * @param trackerFactory tracker with statistics to update * @return the results * @throws IOException if the retry invocation raises one (it shouldn't). */ @Retries.RetryRaw CompletableFuture listObjectsAsync( - S3ListRequest request) + S3ListRequest request, + DurationTrackerFactory trackerFactory) throws IOException; /** @@ -57,13 +60,15 @@ CompletableFuture listObjectsAsync( * Retry policy: retry untranslated. * @param request last list objects request to continue * @param prevResult last paged result to continue from + * @param trackerFactory tracker with statistics to update * @return the next result object * @throws IOException none, just there for retryUntranslated. */ @Retries.RetryRaw CompletableFuture continueListObjectsAsync( S3ListRequest request, - S3ListResult prevResult) + S3ListResult prevResult, + DurationTrackerFactory trackerFactory) throws IOException; /** @@ -117,4 +122,5 @@ S3ListRequest createListObjectsRequest( * @return true iff the path is authoritative on the client. */ boolean allowAuthoritative(Path p); + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/NetworkBinding.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/NetworkBinding.java index ca1b09e9bdc93..3eeb01f971bad 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/NetworkBinding.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/NetworkBinding.java @@ -51,22 +51,18 @@ public class NetworkBinding { ".thirdparty.apache.http.conn.ssl.SSLConnectionSocketFactory"; /** - * Configures the {@link com.amazonaws.thirdparty.apache.http.conn.ssl - * .SSLConnectionSocketFactory} used by the AWS SDK. A custom - * SSLConnectionSocketFactory can be set using the method - * {@link com.amazonaws.ApacheHttpClientConfig#setSslSocketFactory( - * com.amazonaws.thirdparty.apache.http.conn.socket.ConnectionSocketFactory)}. - * If {@link com.amazonaws.thirdparty.apache.http.conn.ssl - * .SSLConnectionSocketFactory} cannot be found on the classpath, the value + * Configures the {@code SSLConnectionSocketFactory} used by the AWS SDK. + * A custom Socket Factory can be set using the method + * {@code setSslSocketFactory()}. + * If {@code SSLConnectionSocketFactory} cannot be found on the classpath, the value * of {@link org.apache.hadoop.fs.s3a.Constants#SSL_CHANNEL_MODE} is ignored. * * @param conf the {@link Configuration} used to get the client specified - * value of {@link org.apache.hadoop.fs.s3a.Constants - * #SSL_CHANNEL_MODE} - * @param awsConf the {@link ClientConfiguration} to set the + * value of {@code SSL_CHANNEL_MODE} + * @param awsConf the {@code ClientConfiguration} to set the * SSLConnectionSocketFactory for. * @throws IOException if there is an error while initializing the - * {@link SSLSocketFactory} other than classloader problems. + * {@code SSLSocketFactory} other than classloader problems. */ public static void bindSSLChannelMode(Configuration conf, ClientConfiguration awsConf) throws IOException { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploader.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploader.java index e811af94c37f4..5ff116fe38ec7 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploader.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploader.java @@ -30,6 +30,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -53,8 +54,11 @@ import org.apache.hadoop.fs.UploadHandle; import org.apache.hadoop.fs.impl.AbstractMultipartUploader; import org.apache.hadoop.fs.s3a.WriteOperations; -import org.apache.hadoop.fs.s3a.impl.statistics.S3AMultipartUploaderStatistics; +import org.apache.hadoop.fs.s3a.statistics.S3AMultipartUploaderStatistics; import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; +import org.apache.hadoop.fs.statistics.IOStatistics; + +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToString; /** * MultipartUploader for S3AFileSystem. This uses the S3 multipart @@ -102,7 +106,7 @@ class S3AMultipartUploader extends AbstractMultipartUploader { this.builder = builder; this.writeOperations = writeOperations; this.context = context; - this.statistics = statistics; + this.statistics = Objects.requireNonNull(statistics); } @Override @@ -113,6 +117,22 @@ public void close() throws IOException { super.close(); } + @Override + public IOStatistics getIOStatistics() { + return statistics.getIOStatistics(); + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder( + "S3AMultipartUploader{"); + sb.append("base=").append(getBasePath()); + sb.append("; statistics=").append( + ioStatisticsToString(statistics.getIOStatistics())); + sb.append('}'); + return sb.toString(); + } + /** * Retrieve the operation state; create one on demand if needed * and there has been no unsuccessful attempt to create one. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploaderBuilder.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploaderBuilder.java index 3bf1a7ddd919a..cb38b82a765cb 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploaderBuilder.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploaderBuilder.java @@ -25,7 +25,7 @@ import org.apache.hadoop.fs.impl.MultipartUploaderBuilderImpl; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.WriteOperations; -import org.apache.hadoop.fs.s3a.impl.statistics.S3AMultipartUploaderStatistics; +import org.apache.hadoop.fs.s3a.statistics.S3AMultipartUploaderStatistics; /** * Builder for S3A multipart uploaders. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java index 17eb27f4264b8..88231d8af9c04 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java @@ -35,9 +35,9 @@ 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.Statistic; +import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext; import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider; import org.apache.hadoop.fs.s3a.s3guard.MetadataStore; import org.apache.hadoop.security.UserGroupInformation; @@ -90,7 +90,8 @@ public class StoreContext { private final Invoker invoker; /** Instrumentation and statistics. */ - private final S3AInstrumentation instrumentation; + private final S3AStatisticsContext instrumentation; + private final S3AStorageStatistics storageStatistics; /** Seek policy. */ @@ -131,7 +132,7 @@ public StoreContext( final ExecutorService executor, final int executorCapacity, final Invoker invoker, - final S3AInstrumentation instrumentation, + final S3AStatisticsContext instrumentation, final S3AStorageStatistics storageStatistics, final S3AInputPolicy inputPolicy, final ChangeDetectionPolicy changeDetectionPolicy, @@ -188,7 +189,12 @@ public Invoker getInvoker() { return invoker; } - public S3AInstrumentation getInstrumentation() { + /** + * Get the statistics context for this StoreContext. + * @return the statistics context this store context was created + * with. + */ + public S3AStatisticsContext getInstrumentation() { return instrumentation; } @@ -270,7 +276,6 @@ public void incrementStatistic(Statistic statistic) { */ public void incrementStatistic(Statistic statistic, long count) { instrumentation.incrementCounter(statistic, count); - storageStatistics.incrementCounter(statistic, count); } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContextBuilder.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContextBuilder.java index 4d789d417dffa..13953f9c985f3 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContextBuilder.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContextBuilder.java @@ -24,8 +24,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.s3a.Invoker; 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.statistics.S3AStatisticsContext; import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider; import org.apache.hadoop.fs.s3a.s3guard.MetadataStore; import org.apache.hadoop.security.UserGroupInformation; @@ -51,7 +51,7 @@ public class StoreContextBuilder { private Invoker invoker; - private S3AInstrumentation instrumentation; + private S3AStatisticsContext instrumentation; private S3AStorageStatistics storageStatistics; @@ -112,7 +112,7 @@ public StoreContextBuilder setInvoker(final Invoker invoke) { } public StoreContextBuilder setInstrumentation( - final S3AInstrumentation instr) { + final S3AStatisticsContext instr) { this.instrumentation = instr; return this; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java index 976c6614837ad..b963e7e2532e5 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java @@ -21,6 +21,7 @@ import javax.annotation.Nullable; import java.io.IOException; import java.io.InterruptedIOException; +import java.io.UncheckedIOException; import java.net.URI; import java.nio.file.AccessDeniedException; import java.util.ArrayList; @@ -80,8 +81,8 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.RemoteIterator; -import org.apache.hadoop.fs.impl.FunctionsRaisingIOE; -import org.apache.hadoop.fs.impl.WrappedIOException; +import org.apache.hadoop.util.functional.CallableRaisingIOE; +import org.apache.hadoop.util.functional.RemoteIterators; import org.apache.hadoop.fs.s3a.AWSCredentialProviderList; import org.apache.hadoop.fs.s3a.AWSServiceThrottledException; import org.apache.hadoop.fs.s3a.Constants; @@ -451,7 +452,8 @@ void bindToOwnerFilesystem(final S3AFileSystem fs) { owner = fs; conf = owner.getConf(); StoreContext context = owner.createStoreContext(); - instrumentation = context.getInstrumentation().getS3GuardInstrumentation(); + instrumentation = context.getInstrumentation() + .getS3GuardInstrumentation(); username = context.getUsername(); executor = MoreExecutors.listeningDecorator( context.createThrottledExecutor()); @@ -641,8 +643,9 @@ public void deleteSubtree(Path path, LOG.debug("Subtree path {} is deleted; this will be a no-op", path); return; } - deleteEntries(new InternalIterators.PathFromRemoteStatusIterator( - new DescendantsIterator(this, meta)), + deleteEntries(RemoteIterators.mappingRemoteIterator( + new DescendantsIterator(this, meta), + FileStatus::getPath), operationState); } @@ -651,8 +654,7 @@ public void deleteSubtree(Path path, public void deletePaths(Collection paths, final BulkOperationState operationState) throws IOException { - deleteEntries( - new InternalIterators.RemoteIteratorFromIterator<>(paths.iterator()), + deleteEntries(RemoteIterators.remoteIteratorFromIterable(paths), operationState); } @@ -829,7 +831,7 @@ public DirListingMetadata listChildren(final Path path) throws IOException { for (Item item : wrapWithRetries(items)) { metas.add(itemToPathMetadata(item, username)); } - } catch (WrappedIOException e) { + } catch (UncheckedIOException e) { // failure in the iterators; unwrap. throw e.getCause(); } @@ -1637,7 +1639,7 @@ private int innerPrune( Set clearedParentPathSet = new HashSet<>(); // declare the operation to delete a batch as a function so // as to keep the code consistent across multiple uses. - FunctionsRaisingIOE.CallableRaisingIOE deleteBatchOperation = + CallableRaisingIOE deleteBatchOperation = () -> { // lowest path entries get deleted first. deletionBatch.sort(PathOrderComparators.TOPMOST_PATH_LAST); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/InternalIterators.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/InternalIterators.java deleted file mode 100644 index dd6fb5ff11544..0000000000000 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/InternalIterators.java +++ /dev/null @@ -1,91 +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.s3guard; - -import java.io.IOException; -import java.util.Iterator; - -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.RemoteIterator; -import org.apache.hadoop.fs.s3a.S3AFileStatus; - -/** - * Internal iterators. - */ -final class InternalIterators { - - private InternalIterators() { - } - - /** - * From a remote status iterator, build a path iterator. - */ - static final class PathFromRemoteStatusIterator implements - RemoteIterator { - - private final RemoteIterator source; - - /** - * Construct. - * @param source source iterator. - */ - PathFromRemoteStatusIterator(final RemoteIterator source) { - this.source = source; - } - - @Override - public boolean hasNext() throws IOException { - return source.hasNext(); - } - - @Override - public Path next() throws IOException { - return source.next().getPath(); - } - } - - /** - * From a classic java.util.Iterator, build a Hadoop remote iterator. - * @param type of iterated value. - */ - static final class RemoteIteratorFromIterator implements - RemoteIterator { - - private final Iterator source; - - /** - * Construct. - * @param source source iterator. - */ - RemoteIteratorFromIterator(final Iterator source) { - this.source = source; - } - - @Override - public boolean hasNext() { - return source.hasNext(); - } - - @Override - public T next() { - return source.next(); - } - } - -} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RetryingCollection.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RetryingCollection.java index 394f393c57ec5..108d205f74b05 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RetryingCollection.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RetryingCollection.java @@ -19,9 +19,9 @@ package org.apache.hadoop.fs.s3a.s3guard; import java.io.IOException; +import java.io.UncheckedIOException; import java.util.Iterator; -import org.apache.hadoop.fs.impl.WrappedIOException; import org.apache.hadoop.fs.s3a.Invoker; import org.apache.hadoop.fs.s3a.Retries; @@ -88,7 +88,7 @@ private RetryingIterator(final Iterator iterator) { /** * {@inheritDoc}. - * @throws WrappedIOException for IO failure, including throttling. + * @throws UncheckedIOException for IO failure, including throttling. */ @Override @Retries.RetryTranslated @@ -100,13 +100,13 @@ public boolean hasNext() { true, iterator::hasNext); } catch (IOException e) { - throw new WrappedIOException(e); + throw new UncheckedIOException(e); } } /** * {@inheritDoc}. - * @throws WrappedIOException for IO failure, including throttling. + * @throws UncheckedIOException for IO failure, including throttling. */ @Override @Retries.RetryTranslated @@ -118,7 +118,7 @@ public T next() { true, iterator::next); } catch (IOException e) { - throw new WrappedIOException(e); + throw new UncheckedIOException(e); } } } 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 69b9aaf780761..02d1e53c7ba81 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.statistics.S3AInputStreamStatistics; import org.apache.hadoop.io.IOUtils; import static org.apache.hadoop.thirdparty.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", @@ -204,7 +204,7 @@ public synchronized long skip(final long n) throws IOException { long skipped = once("skip", uri, () -> wrappedStream.skip(n)); pos.addAndGet(skipped); // treat as a forward skip for stats - streamStatistics.seekForwards(skipped); + streamStatistics.seekForwards(skipped, skipped); return skipped; } @@ -331,7 +331,7 @@ public synchronized void seek(long newPos) throws IOException { bytesSkipped++; } // read has finished. - streamStatistics.seekForwards(bytesSkipped); + streamStatistics.seekForwards(bytesSkipped, bytesSkipped); } } @@ -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/main/java/org/apache/hadoop/fs/s3a/statistics/BlockOutputStreamStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/BlockOutputStreamStatistics.java new file mode 100644 index 0000000000000..b1cee718c206a --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/BlockOutputStreamStatistics.java @@ -0,0 +1,137 @@ +/* + * 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.statistics; + +import java.io.Closeable; +import java.time.Duration; + +/** + * Block output stream statistics. + */ +public interface BlockOutputStreamStatistics extends Closeable, + S3AStatisticInterface { + + /** + * Block is queued for upload. + * @param blockSize block size. + */ + void blockUploadQueued(int blockSize); + + /** + * Queued block has been scheduled for upload. + * @param timeInQueue time in the queue. + * @param blockSize block size. + */ + void blockUploadStarted(Duration timeInQueue, int blockSize); + + /** + * A block upload has completed. Duration excludes time in the queue. + * @param timeSinceUploadStarted time in since the transfer began. + * @param blockSize block size + */ + void blockUploadCompleted(Duration timeSinceUploadStarted, int blockSize); + + /** + * A block upload has failed. Duration excludes time in the queue. + *

+ * A final transfer completed event is still expected, so this + * does not decrement the active block counter. + *

+ * @param timeSinceUploadStarted time in since the transfer began. + * @param blockSize block size + */ + void blockUploadFailed(Duration timeSinceUploadStarted, int blockSize); + + /** + * Intermediate report of bytes uploaded. + * @param byteCount 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(); + + /** + * Get the value of a counter. + * @param name counter name + * @return the value or null if no matching counter was found. + */ + Long lookupCounterValue(String name); + + /** + * Get the value of a gauge. + * @param name gauge name + * @return the value or null if no matching gauge was found. + */ + Long lookupGaugeValue(String name); +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/ChangeTrackerStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/ChangeTrackerStatistics.java new file mode 100644 index 0000000000000..572581e214fbb --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/ChangeTrackerStatistics.java @@ -0,0 +1,36 @@ +/* + * 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.statistics; + +/** + * Interface for change tracking statistics. + */ +public interface ChangeTrackerStatistics { + + /** + * A version mismatch was detected. + */ + void versionMismatchError(); + + /** + * How many version mismatches have occurred. + * @return counter of mismatches. + */ + long getVersionMismatches(); +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/CommitterStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/CommitterStatistics.java new file mode 100644 index 0000000000000..fd232a058d0b8 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/CommitterStatistics.java @@ -0,0 +1,66 @@ +/* + * 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.statistics; + +/** + * Statistics for S3A committers. + */ +public interface CommitterStatistics + extends S3AStatisticInterface { + + /** 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/statistics/CountersAndGauges.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/CountersAndGauges.java new file mode 100644 index 0000000000000..61cc0330936d8 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/CountersAndGauges.java @@ -0,0 +1,72 @@ +/* + * 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.statistics; + +import java.time.Duration; + +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); + + /** + * Record a duration. + * @param op operation + * @param success was the operation a success? + * @param duration how long did it take + */ + void recordDuration(Statistic op, boolean success, Duration duration); +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/DelegationTokenStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/DelegationTokenStatistics.java new file mode 100644 index 0000000000000..3ab9224f3c568 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/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.statistics; + +/** + * Instrumentation exported to for S3A Delegation Token support. + */ +public interface DelegationTokenStatistics extends S3AStatisticInterface { + + /** A token has been issued. */ + void tokenIssued(); +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AInputStreamStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AInputStreamStatistics.java new file mode 100644 index 0000000000000..328d9f7c4ce4a --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AInputStreamStatistics.java @@ -0,0 +1,191 @@ +/* + * 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.statistics; + +import org.apache.hadoop.fs.statistics.DurationTracker; + +/** + * Statistics updated by a + * {@link org.apache.hadoop.fs.s3a.S3AInputStream} during its use. + * It also contains getters for tests. + */ +public interface S3AInputStreamStatistics extends AutoCloseable, + S3AStatisticInterface { + + /** + * 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 bytes moved forward in stream + * @param bytesReadInSeek 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, long bytesReadInSeek); + + /** + * The inner stream was opened. + * The return value is used in the input stream to decide whether it is + * the initial vs later count. + * @return the previous count or zero if this is the first opening. + */ + 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(); + + /** + * A stream {@code unbuffer()} call has been made. + */ + void unbuffered(); + + long getCloseOperations(); + + long getClosed(); + + long getAborted(); + + long getForwardSeekOperations(); + + long getBackwardSeekOperations(); + + /** + * The bytes read in read() operations. + * @return the number of bytes returned to the caller. + */ + long getBytesRead(); + + /** + * The total number of bytes read, including + * all read and discarded when closing streams + * or skipped during seek calls. + * @return the total number of bytes read from + * S3. + */ + long getTotalBytesRead(); + + 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(); + + /** + * Get the value of a counter. + * @param name counter name + * @return the value or null if no matching counter was found. + */ + Long lookupCounterValue(String name); + + /** + * Get the value of a gauge. + * @param name gauge name + * @return the value or null if no matching gauge was found. + */ + Long lookupGaugeValue(String name); + + /** + * Initiate a GET request. + * @return duration tracker; + */ + DurationTracker initiateGetRequest(); + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AMultipartUploaderStatistics.java similarity index 88% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatistics.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AMultipartUploaderStatistics.java index 2cd74ff9cf5da..2aa7b341af679 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatistics.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AMultipartUploaderStatistics.java @@ -16,14 +16,17 @@ * limitations under the License. */ -package org.apache.hadoop.fs.s3a.impl.statistics; +package org.apache.hadoop.fs.s3a.statistics; import java.io.Closeable; /** * Statistics for the S3A multipart uploader. + * It is expected to act as the statistics source for + * the uploader. */ -public interface S3AMultipartUploaderStatistics extends Closeable { +public interface S3AMultipartUploaderStatistics extends Closeable, + S3AStatisticInterface { void instantiated(); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AStatisticInterface.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AStatisticInterface.java new file mode 100644 index 0000000000000..836f2038d129b --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AStatisticInterface.java @@ -0,0 +1,32 @@ +/* + * 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.statistics; + +import org.apache.hadoop.fs.statistics.DurationTrackerFactory; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; + +/** + * Base Interface for S3A Statistics, as a way of managing + * them. + */ +public interface S3AStatisticInterface extends IOStatisticsSource, + DurationTrackerFactory { + + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AStatisticsContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AStatisticsContext.java new file mode 100644 index 0000000000000..27f1398d4ea51 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AStatisticsContext.java @@ -0,0 +1,69 @@ +/* + * 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.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(); + + /** + * Creaet a multipart statistics collector. + * @return an instance + */ + S3AMultipartUploaderStatistics createMultipartUploaderStatistics(); +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/StatisticTypeEnum.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/StatisticTypeEnum.java new file mode 100644 index 0000000000000..ac7bd9b5e7b49 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/StatisticTypeEnum.java @@ -0,0 +1,46 @@ +/* + * 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.statistics; + +/** + * Enum of statistic types. + */ +public enum StatisticTypeEnum { + + /** + * Counter. + */ + TYPE_COUNTER, + + /** + * Duration. + */ + TYPE_DURATION, + + /** + * Gauge. + */ + TYPE_GAUGE, + + /** + * Quantile. + */ + TYPE_QUANTILE, + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/StatisticsFromAwsSdk.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/StatisticsFromAwsSdk.java new file mode 100644 index 0000000000000..2fd7c685ccdd8 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/StatisticsFromAwsSdk.java @@ -0,0 +1,81 @@ +/* + * 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.statistics; + +import java.time.Duration; + +import org.apache.hadoop.fs.s3a.statistics.impl.AwsStatisticsCollector; + +/** + * 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/statistics/impl/AbstractS3AStatisticsSource.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/AbstractS3AStatisticsSource.java new file mode 100644 index 0000000000000..6b7eb245e49ff --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/AbstractS3AStatisticsSource.java @@ -0,0 +1,111 @@ +/* + * 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.statistics.impl; + +import org.apache.hadoop.fs.statistics.DurationTracker; +import org.apache.hadoop.fs.statistics.DurationTrackerFactory; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; + +/** + * Base class for implementing IOStatistics sources in the S3 module. + *

+ * A lot of the methods are very terse, because S3AInstrumentation has + * verbose methods of similar names; the short ones always + * refer to the inner class and not any superclass method. + *

+ */ +public abstract class AbstractS3AStatisticsSource implements + IOStatisticsSource, DurationTrackerFactory { + + private IOStatisticsStore ioStatistics; + + protected AbstractS3AStatisticsSource() { + } + + @Override + public IOStatisticsStore getIOStatistics() { + return ioStatistics; + } + + /** + * Setter. + * this must be called in the subclass constructor with + * whatever + * @param statistics statistics to set + */ + protected void setIOStatistics(final IOStatisticsStore statistics) { + this.ioStatistics = statistics; + } + + /** + * Increment a named counter by 1. + * @param name counter name + * @return the updated value or, if the counter is unknown: 0 + */ + public long incCounter(String name) { + return incCounter(name, 1); + } + + /**DefaultS3ClientFactoryDefaultS3ClientFactory + * Increment a named counter by 1. + * @param name counter name + * @param value value to increment by + * @return the updated value or, if the counter is unknown: 0 + */ + public long incCounter(String name, long value) { + return ioStatistics.incrementCounter(name, value); + } + + /** + * {@inheritDoc} + */ + public Long lookupCounterValue(final String name) { + return ioStatistics.counters().get(name); + } + + /** + * {@inheritDoc} + */ + public Long lookupGaugeValue(final String name) { + return ioStatistics.gauges().get(name); + } + + public long incGauge(String name, long v) { + return ioStatistics.incrementGauge(name, v); + } + + public long incGauge(String name) { + return incGauge(name, 1); + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder( + "AbstractS3AStatisticsSource{"); + sb.append(ioStatistics); + sb.append('}'); + return sb.toString(); + } + + @Override + public DurationTracker trackDuration(final String key, final long count) { + return getIOStatistics().trackDuration(key, count); + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/AwsStatisticsCollector.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/AwsStatisticsCollector.java new file mode 100644 index 0000000000000..c002a4a6dee1d --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/AwsStatisticsCollector.java @@ -0,0 +1,129 @@ +/* + * 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.statistics.impl; + +import java.time.Duration; +import java.util.function.Consumer; +import java.util.function.LongConsumer; + +import com.amazonaws.Request; +import com.amazonaws.Response; +import com.amazonaws.metrics.RequestMetricCollector; +import com.amazonaws.util.TimingInfo; + +import org.apache.hadoop.fs.s3a.statistics.StatisticsFromAwsSdk; + +import static com.amazonaws.util.AWSRequestMetrics.Field.ClientExecuteTime; +import static com.amazonaws.util.AWSRequestMetrics.Field.HttpClientRetryCount; +import static com.amazonaws.util.AWSRequestMetrics.Field.HttpRequestTime; +import static com.amazonaws.util.AWSRequestMetrics.Field.RequestCount; +import static com.amazonaws.util.AWSRequestMetrics.Field.RequestMarshallTime; +import static com.amazonaws.util.AWSRequestMetrics.Field.RequestSigningTime; +import static com.amazonaws.util.AWSRequestMetrics.Field.ResponseProcessingTime; +import static com.amazonaws.util.AWSRequestMetrics.Field.ThrottleException; + +/** + * Collect statistics from the AWS SDK and forward to an instance of + * {@link StatisticsFromAwsSdk} and thence into the S3A statistics. + *

+ * 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 durationConsumer) { + TimingInfo t1 = timingInfo.getSubMeasurement(subMeasurementName); + if (t1 != null && t1.getTimeTakenMillisIfKnown() != null) { + durationConsumer.accept(Duration.ofMillis( + t1.getTimeTakenMillisIfKnown().longValue())); + } + } + + /** + * Process a counter. + * @param timingInfo timing info + * @param subMeasurementName sub measurement + * @param consumer consumer + */ + private void counter( + TimingInfo timingInfo, + String subMeasurementName, + LongConsumer consumer) { + Number n = timingInfo.getCounter(subMeasurementName); + if (n != null) { + consumer.accept(n.longValue()); + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/BondedS3AStatisticsContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/BondedS3AStatisticsContext.java new file mode 100644 index 0000000000000..006eb24f72c85 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/BondedS3AStatisticsContext.java @@ -0,0 +1,234 @@ +/* + * 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.statistics.impl; + +import javax.annotation.Nullable; + +import java.time.Duration; + +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; +import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics; +import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics; +import org.apache.hadoop.fs.s3a.statistics.DelegationTokenStatistics; +import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; +import org.apache.hadoop.fs.s3a.statistics.S3AMultipartUploaderStatistics; +import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext; +import org.apache.hadoop.fs.s3a.statistics.StatisticsFromAwsSdk; + +/** + * 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. + * The FileSystem.Statistics instance of the {@link #statisticsSource} + * is used as the reference to FileSystem statistics to update + * @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); + } + + @Override + public void recordDuration(final Statistic op, + final boolean success, + final Duration duration) { + getInstrumentation().recordDuration(op, success, duration); + } + + /** + * 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 new StatisticsFromAwsSdkImpl(getInstrumentation()); + } + + @Override + public S3AMultipartUploaderStatistics createMultipartUploaderStatistics() { + return new S3AMultipartUploaderStatisticsImpl(this::incrementCounter); + } + + /** + * 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/statistics/impl/CountingChangeTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/CountingChangeTracker.java new file mode 100644 index 0000000000000..18157740c8ba0 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/CountingChangeTracker.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.s3a.statistics.impl; + +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.hadoop.fs.s3a.statistics.ChangeTrackerStatistics; + +/** + * 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/statistics/impl/EmptyS3AStatisticsContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/EmptyS3AStatisticsContext.java new file mode 100644 index 0000000000000..c8cd80592088b --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/EmptyS3AStatisticsContext.java @@ -0,0 +1,597 @@ +/* + * 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.statistics.impl; + +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.s3a.statistics.BlockOutputStreamStatistics; +import org.apache.hadoop.fs.s3a.statistics.ChangeTrackerStatistics; +import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics; +import org.apache.hadoop.fs.s3a.statistics.DelegationTokenStatistics; +import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; +import org.apache.hadoop.fs.s3a.statistics.S3AMultipartUploaderStatistics; +import org.apache.hadoop.fs.s3a.statistics.S3AStatisticInterface; +import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext; +import org.apache.hadoop.fs.s3a.statistics.StatisticsFromAwsSdk; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.DurationTracker; + +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.emptyStatistics; +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.stubDurationTracker; + +/** + * 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 { + + public static final MetastoreInstrumentation + METASTORE_INSTRUMENTATION = new MetastoreInstrumentationImpl(); + + public static final S3AInputStreamStatistics + EMPTY_INPUT_STREAM_STATISTICS = new EmptyInputStreamStatistics(); + + public static final CommitterStatistics + EMPTY_COMMITTER_STATISTICS = new EmptyCommitterStatistics(); + + @SuppressWarnings("IOResourceOpenedButNotSafelyClosed") + public static final BlockOutputStreamStatistics + EMPTY_BLOCK_OUTPUT_STREAM_STATISTICS + = new EmptyBlockOutputStreamStatistics(); + + public static final DelegationTokenStatistics + EMPTY_DELEGATION_TOKEN_STATISTICS = new EmptyDelegationTokenStatistics(); + + public static final StatisticsFromAwsSdk + EMPTY_STATISTICS_FROM_AWS_SDK = new EmptyStatisticsFromAwsSdk(); + + @Override + public MetastoreInstrumentation getS3GuardInstrumentation() { + return METASTORE_INSTRUMENTATION; + } + + @Override + public S3AInputStreamStatistics newInputStreamStatistics() { + return EMPTY_INPUT_STREAM_STATISTICS; + } + + @Override + public CommitterStatistics newCommitterStatistics() { + return EMPTY_COMMITTER_STATISTICS; + } + + @Override + public BlockOutputStreamStatistics newOutputStreamStatistics() { + return EMPTY_BLOCK_OUTPUT_STREAM_STATISTICS; + } + + @Override + public DelegationTokenStatistics newDelegationTokenStatistics() { + return EMPTY_DELEGATION_TOKEN_STATISTICS; + } + + @Override + public StatisticsFromAwsSdk newStatisticsFromAwsSdk() { + return EMPTY_STATISTICS_FROM_AWS_SDK; + } + + @Override + public S3AMultipartUploaderStatistics createMultipartUploaderStatistics() { + return new EmptyMultipartUploaderStatistics(); + } + + @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) { + + } + + @Override + public void recordDuration(final Statistic op, + final boolean success, + final Duration duration) { + + } + + /** + * Base class for all the empty implementations. + */ + private static class EmptyS3AStatisticImpl implements + S3AStatisticInterface { + + /** + * Always return the stub duration tracker. + * @param key statistic key prefix + * @param count #of times to increment the matching counter in this + * operation. + * @return stub tracker. + */ + public DurationTracker trackDuration(String key, long count) { + return stubDurationTracker(); + } + } + + /** + * Input Stream statistics callbacks. + */ + private static final class EmptyInputStreamStatistics + extends EmptyS3AStatisticImpl + implements S3AInputStreamStatistics { + + @Override + public void seekBackwards(final long negativeOffset) { + + } + + @Override + public void seekForwards(final long skipped, + final long bytesReadInSeek) { + + } + + @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 unbuffered() { + + } + + /** + * 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 getTotalBytesRead() { + 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 lookupCounterValue(final String name) { + return 0L; + } + + @Override + public Long lookupGaugeValue(final String name) { + return 0L; + } + + @Override + public ChangeTrackerStatistics getChangeTrackerStatistics() { + return new CountingChangeTracker(); + } + + @Override + public DurationTracker initiateGetRequest() { + return stubDurationTracker(); + } + + } + + /** + * Committer statistics. + */ + private static final class EmptyCommitterStatistics + extends EmptyS3AStatisticImpl + 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 + extends EmptyS3AStatisticImpl + implements BlockOutputStreamStatistics { + + @Override + public void blockUploadQueued(final int blockSize) { + } + + @Override + public void blockUploadStarted(final Duration timeInQueue, + final int blockSize) { + } + + @Override + public void blockUploadCompleted(final Duration timeSinceUploadStarted, + final int blockSize) { + } + + @Override + public void blockUploadFailed(final Duration timeSinceUploadStarted, + 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 Long lookupCounterValue(final String name) { + return 0L; + } + + @Override + public Long lookupGaugeValue(final String name) { + return 0L; + } + + @Override + public void close() throws IOException { + } + + } + + /** + * Delegation Token Statistics. + */ + private static final class EmptyDelegationTokenStatistics + extends EmptyS3AStatisticImpl + implements DelegationTokenStatistics { + + @Override + public void tokenIssued() { + + } + } + + /** + * AWS SDK Callbacks. + */ + 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) { + + } + } + + /** + * Multipart Uploader. + */ + public static final class EmptyMultipartUploaderStatistics + implements S3AMultipartUploaderStatistics { + + @Override + public void instantiated() { + + } + + @Override + public void uploadStarted() { + + } + + @Override + public void partPut(final long lengthInBytes) { + + } + + @Override + public void uploadCompleted() { + + } + + @Override + public void uploadAborted() { + + } + + @Override + public void abortUploadsUnderPathInvoked() { + + } + + @Override + public void close() throws IOException { + + } + + @Override + public DurationTracker trackDuration(final String key, final long count) { + return stubDurationTracker(); + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatisticsImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/S3AMultipartUploaderStatisticsImpl.java similarity index 60% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatisticsImpl.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/S3AMultipartUploaderStatisticsImpl.java index 70e4785f62e19..7b6d559cf2099 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatisticsImpl.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/S3AMultipartUploaderStatisticsImpl.java @@ -16,30 +16,42 @@ * limitations under the License. */ -package org.apache.hadoop.fs.s3a.impl.statistics; +package org.apache.hadoop.fs.s3a.statistics.impl; import java.io.IOException; +import java.util.Objects; import java.util.function.BiConsumer; import org.apache.hadoop.fs.s3a.Statistic; +import org.apache.hadoop.fs.s3a.statistics.S3AMultipartUploaderStatistics; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; -import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_ABORT_UNDER_PATH_INVOKED; -import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_INSTANTIATED; -import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_PART_PUT; -import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_PART_PUT_BYTES; import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_ABORTED; +import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_ABORT_UNDER_PATH_INVOKED; import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_COMPLETED; +import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_INSTANTIATED; +import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_PART_PUT; +import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_PART_PUT_BYTES; import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_STARTED; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore; /** * Implementation of the uploader statistics. + *

* This takes a function to update some counter and will update * this value when things change, so it can be bonded to arbitrary * statistic collectors. + *

+ *

+ * Internally it builds a map of the relevant multipart statistics, + * increments as appropriate and serves this data back through + * the {@code IOStatisticsSource} API. + *

*/ -public final class S3AMultipartUploaderStatisticsImpl implements - S3AMultipartUploaderStatistics { +public final class S3AMultipartUploaderStatisticsImpl + extends AbstractS3AStatisticsSource + implements S3AMultipartUploaderStatistics { /** * The operation to increment a counter/statistic by a value. @@ -53,16 +65,28 @@ public final class S3AMultipartUploaderStatisticsImpl implements */ public S3AMultipartUploaderStatisticsImpl( final BiConsumer incrementCallback) { - this.incrementCallback = incrementCallback; + this.incrementCallback = Objects.requireNonNull(incrementCallback); + IOStatisticsStore st = iostatisticsStore() + .withCounters( + MULTIPART_UPLOAD_INSTANTIATED.getSymbol(), + MULTIPART_UPLOAD_PART_PUT.getSymbol(), + MULTIPART_UPLOAD_PART_PUT_BYTES.getSymbol(), + MULTIPART_UPLOAD_ABORTED.getSymbol(), + MULTIPART_UPLOAD_ABORT_UNDER_PATH_INVOKED.getSymbol(), + MULTIPART_UPLOAD_COMPLETED.getSymbol(), + MULTIPART_UPLOAD_STARTED.getSymbol()) + .build(); + setIOStatistics(st); } private void inc(Statistic op, long count) { incrementCallback.accept(op, count); + incCounter(op.getSymbol(), count); } @Override public void instantiated() { - inc(MULTIPART_INSTANTIATED, 1); + inc(MULTIPART_UPLOAD_INSTANTIATED, 1); } @Override @@ -72,8 +96,8 @@ public void uploadStarted() { @Override public void partPut(final long lengthInBytes) { - inc(MULTIPART_PART_PUT, 1); - inc(MULTIPART_PART_PUT_BYTES, lengthInBytes); + inc(MULTIPART_UPLOAD_PART_PUT, 1); + inc(MULTIPART_UPLOAD_PART_PUT_BYTES, lengthInBytes); } @Override diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/StatisticsFromAwsSdkImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/StatisticsFromAwsSdkImpl.java new file mode 100644 index 0000000000000..48b0b2bf454d3 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/StatisticsFromAwsSdkImpl.java @@ -0,0 +1,88 @@ +/* + * 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.statistics.impl; + +import java.time.Duration; + +import org.apache.hadoop.fs.s3a.statistics.CountersAndGauges; +import org.apache.hadoop.fs.s3a.statistics.StatisticsFromAwsSdk; + +import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_REQUEST; +import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_RETRY; +import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_THROTTLED; +import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_THROTTLE_RATE; + +/** + * Hook up AWS SDK Statistics to the S3 counters. + *

+ * Durations are not currently being used; that could be + * changed in future once an effective strategy for reporting + * them is determined. + */ +public final class StatisticsFromAwsSdkImpl implements + StatisticsFromAwsSdk { + + private final CountersAndGauges countersAndGauges; + + public StatisticsFromAwsSdkImpl( + final CountersAndGauges countersAndGauges) { + this.countersAndGauges = countersAndGauges; + } + + @Override + public void updateAwsRequestCount(final long count) { + countersAndGauges.incrementCounter(STORE_IO_REQUEST, count); + } + + @Override + public void updateAwsRetryCount(final long count) { + countersAndGauges.incrementCounter(STORE_IO_RETRY, count); + } + + @Override + public void updateAwsThrottleExceptionsCount(final long count) { + countersAndGauges.incrementCounter(STORE_IO_THROTTLED, count); + countersAndGauges.addValueToQuantiles(STORE_IO_THROTTLE_RATE, count); + } + + @Override + public void noteAwsRequestTime(final Duration duration) { + + } + + @Override + public void noteAwsClientExecuteTime(final Duration duration) { + + } + + @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/statistics/impl/package-info.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/package-info.java new file mode 100644 index 0000000000000..6b94ac767521e --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/package-info.java @@ -0,0 +1,29 @@ +/* + * 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: implementation. + * Not for use by anything outside the hadoop-aws source tree. + */ + +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.fs.s3a.statistics.impl; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/package-info.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/package-info.java new file mode 100644 index 0000000000000..b74bc3e25286e --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/package-info.java @@ -0,0 +1,31 @@ +/* + * 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: interfaces. + * This is private, though there's a risk that some extension + * points (delegation tokens?) may need access to the internal + * API. Hence the split packaging...with a java 9 module, the + * implementation classes would be declared internal. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.fs.s3a.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/tools/MarkerTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java index afc034743386c..f1fde7b353515 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java @@ -65,6 +65,7 @@ import static org.apache.hadoop.fs.s3a.Constants.BULK_DELETE_PAGE_SIZE; import static org.apache.hadoop.fs.s3a.Constants.BULK_DELETE_PAGE_SIZE_DEFAULT; import static org.apache.hadoop.fs.s3a.Invoker.once; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsSourceToString; import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_INTERRUPTED; import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_NOT_ACCEPTABLE; import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_NOT_FOUND; @@ -672,6 +673,7 @@ private boolean scanDirectoryTree( final int limit) throws IOException { int count = 0; + boolean result = true; RemoteIterator listing = operations .listObjects(path, storeContext.pathToKey(path)); while (listing.hasNext()) { @@ -700,10 +702,16 @@ private boolean scanDirectoryTree( if (limit > 0 && count >= limit) { println(out, "Limit of scan reached - %,d object%s", limit, suffix(limit)); - return false; + result = false; + break; } } - return true; + LOG.debug("Listing summary {}", listing); + if (verbose) { + println(out, "%nListing statistics:%n %s%n", + ioStatisticsSourceToString(listing)); + } + return result; } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java index 740f256b62e2c..ae54dfee0004e 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java @@ -79,15 +79,15 @@ protected Path path(final String filepath) throws IOException { @Override public void testDirectWrite() throws Exception { - resetStorageStatistics(); + final long renames = getRenameOperationCount(); super.testDirectWrite(); assertEquals("Expected no renames for a direct write distcp", 0L, - getRenameOperationCount()); + getRenameOperationCount() - renames); } @Override public void testNonDirectWrite() throws Exception { - resetStorageStatistics(); + final long renames = getRenameOperationCount(); try { super.testNonDirectWrite(); } catch (FileNotFoundException e) { @@ -96,11 +96,7 @@ public void testNonDirectWrite() throws Exception { // S3Guard is not enabled } assertEquals("Expected 2 renames for a non-direct write distcp", 2L, - getRenameOperationCount()); - } - - private void resetStorageStatistics() { - getFileSystem().getStorageStatistics().reset(); + getRenameOperationCount() - renames); } private long getRenameOperationCount() { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java index 73a503ace49a5..b1ea45f686528 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java @@ -26,7 +26,10 @@ import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.contract.s3a.S3AContract; import org.apache.hadoop.fs.s3a.tools.MarkerTool; +import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; import org.apache.hadoop.io.IOUtils; + +import org.junit.AfterClass; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -39,6 +42,8 @@ import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestPropertyBool; import static org.apache.hadoop.fs.s3a.S3AUtils.E_FS_CLOSED; import static org.apache.hadoop.fs.s3a.tools.MarkerTool.UNLIMITED_LISTING; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString; +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.snapshotIOStatistics; /** * An extension of the contract test base set up for S3A tests. @@ -48,6 +53,12 @@ public abstract class AbstractS3ATestBase extends AbstractFSContractTestBase protected static final Logger LOG = LoggerFactory.getLogger(AbstractS3ATestBase.class); + /** + * FileSystem statistics are collected across every test case. + */ + protected static final IOStatisticsSnapshot FILESYSTEM_IOSTATS = + snapshotIOStatistics(); + @Override protected AbstractFSContract createContract(Configuration conf) { return new S3AContract(conf, false); @@ -73,10 +84,22 @@ public void teardown() throws Exception { maybeAuditTestPath(); super.teardown(); + if (getFileSystem() != null) { + FILESYSTEM_IOSTATS.aggregate(getFileSystem().getIOStatistics()); + } describe("closing file system"); IOUtils.closeStream(getFileSystem()); } + /** + * Dump the filesystem statistics after the class. + */ + @AfterClass + public static void dumpFileSystemIOStatistics() { + LOG.info("Aggregate FileSystem Statistics {}", + ioStatisticsToPrettyString(FILESYSTEM_IOSTATS)); + } + /** * Audit the FS under {@link #methodPath()} if * the test option {@link #DIRECTORY_MARKER_AUDIT} is diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestLocatedFileStatusFetcher.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestLocatedFileStatusFetcher.java index bd6bf2f6cdbc3..5b6e634a63168 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestLocatedFileStatusFetcher.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestLocatedFileStatusFetcher.java @@ -18,23 +18,287 @@ package org.apache.hadoop.fs.s3a; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.Collection; + +import org.assertj.core.api.Assertions; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; +import org.apache.hadoop.fs.s3a.auth.ITestRestrictedReadAccess; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.mapred.LocatedFileStatusFetcher; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile; +import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; +import static org.apache.hadoop.fs.s3a.Constants.AUTHORITATIVE_PATH; +import static org.apache.hadoop.fs.s3a.Constants.METADATASTORE_AUTHORITATIVE; +import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBucketOverrides; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticCounter; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.extractStatistics; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_LIST_REQUEST; +import static org.apache.hadoop.mapreduce.lib.input.FileInputFormat.LIST_STATUS_NUM_THREADS; + /** * Test the LocatedFileStatusFetcher can do. * This is related to HADOOP-16458. * There's basic tests in ITestS3AFSMainOperations; this * is see if we can create better corner cases. + *

+ * Much of the class is based on tests in {@link ITestRestrictedReadAccess}, + * but whereas that tests failure paths, this looks at the performance + * of successful invocations. */ +@RunWith(Parameterized.class) public class ITestLocatedFileStatusFetcher extends AbstractS3ATestBase { private static final Logger LOG = LoggerFactory.getLogger(ITestLocatedFileStatusFetcher.class); + + /** + * Parameterization. + */ + @Parameterized.Parameters(name = "{0}") + public static Collection params() { + return Arrays.asList(new Object[][]{ + {"raw", false}, + {"nonauth", true} + }); + } + + /** Filter to select everything. */ + private static final PathFilter EVERYTHING = t -> true; + + /** Filter to select .txt files. */ + private static final PathFilter TEXT_FILE = + path -> path.toUri().toString().endsWith(".txt"); + + /** The same path filter used in FileInputFormat. */ + private static final PathFilter HIDDEN_FILE_FILTER = + (p) -> { + String n = p.getName(); + return !n.startsWith("_") && !n.startsWith("."); + }; + + /** + * Text found in LocatedFileStatusFetcher exception when the glob + * returned "null". + */ + private static final String DOES_NOT_EXIST = "does not exist"; + + /** + * Text found in LocatedFileStatusFetcher exception when + * the glob returned an empty list. + */ + private static final String MATCHES_0_FILES = "matches 0 files"; + + /** + * Text used in files. + */ + public static final byte[] HELLO = "hello".getBytes(StandardCharsets.UTF_8); + + /** + * How many list calls are expected in a run which collects them: {@value}. + */ + private static final int EXPECTED_LIST_COUNT = 4; + + private final String name; + + private final boolean s3guard; + + private Path basePath; + + private Path emptyDir; + + private Path emptyFile; + + private Path subDir; + + private Path subdirFile; + + private Path subDir2; + + private Path subdir2File1; + + private Path subdir2File2; + + private Configuration listConfig; + + public ITestLocatedFileStatusFetcher(final String name, + final boolean s3guard) { + this.name = name; + this.s3guard = s3guard; + } + + @Override + public Configuration createConfiguration() { + Configuration conf = super.createConfiguration(); + String bucketName = getTestBucketName(conf); + + removeBaseAndBucketOverrides(bucketName, conf, + METADATASTORE_AUTHORITATIVE, + AUTHORITATIVE_PATH); + removeBucketOverrides(bucketName, conf, + S3_METADATA_STORE_IMPL); + if (!s3guard) { + removeBaseAndBucketOverrides(bucketName, conf, + S3_METADATA_STORE_IMPL); + } + conf.setBoolean(METADATASTORE_AUTHORITATIVE, false); + disableFilesystemCaching(conf); + return conf; + } + @Override + public void setup() throws Exception { + super.setup(); + S3AFileSystem fs + = getFileSystem(); + // avoiding the parameterization to steer clear of accidentally creating + // patterns; a timestamp is used to ensure tombstones from previous runs + // do not interfere + basePath = path("ITestLocatedFileStatusFetcher-" + name + + "-" + System.currentTimeMillis() / 1000); + + // define the paths and create them. + describe("Creating test directories and files"); + + // an empty directory directory under the noReadDir + emptyDir = new Path(basePath, "emptyDir"); + fs.mkdirs(emptyDir); + + // an empty file directory under the noReadDir + emptyFile = new Path(basePath, "emptyFile.txt"); + touch(fs, emptyFile); + + // a subdirectory + subDir = new Path(basePath, "subDir"); + + // and a file in that subdirectory + subdirFile = new Path(subDir, "subdirFile.txt"); + createFile(fs, subdirFile, true, HELLO); + subDir2 = new Path(subDir, "subDir2"); + subdir2File1 = new Path(subDir2, "subdir2File1.txt"); + subdir2File2 = new Path(subDir2, "subdir2File2.txt"); + createFile(fs, subdir2File1, true, HELLO); + createFile(fs, subdir2File2, true, HELLO); + listConfig = new Configuration(getConfiguration()); + } + + + /** + * Assert that the fetcher stats logs the expected number of calls. + * @param fetcher fetcher + * @param expectedListCount expected number of list calls + */ + private void assertListCount(final LocatedFileStatusFetcher fetcher, + final int expectedListCount) { + IOStatistics iostats = extractStatistics(fetcher); + LOG.info("Statistics of fetcher: {}", iostats); + assertThatStatisticCounter(iostats, + OBJECT_LIST_REQUEST) + .describedAs("stats of %s", iostats) + .isEqualTo(expectedListCount); + } + + /** + * Run a located file status fetcher against the directory tree. + */ @Test - public void testGlobScan() throws Throwable { + public void testSingleThreadedLocatedFileStatus() throws Throwable { + + describe("LocatedFileStatusFetcher operations"); + // use the same filter as FileInputFormat; single thread. + listConfig.setInt(LIST_STATUS_NUM_THREADS, 1); + LocatedFileStatusFetcher fetcher = + new LocatedFileStatusFetcher( + listConfig, + new Path[]{basePath}, + true, + HIDDEN_FILE_FILTER, + true); + Iterable stats = fetcher.getFileStatuses(); + Assertions.assertThat(stats) + .describedAs("result of located scan") + .flatExtracting(FileStatus::getPath) + .containsExactlyInAnyOrder( + emptyFile, + subdirFile, + subdir2File1, + subdir2File2); + assertListCount(fetcher, EXPECTED_LIST_COUNT); } + + /** + * Run a located file status fetcher against the directory tree. + */ + @Test + public void testLocatedFileStatusFourThreads() throws Throwable { + + // four threads and the text filter. + int threads = 4; + describe("LocatedFileStatusFetcher with %d", threads); + listConfig.setInt(LIST_STATUS_NUM_THREADS, threads); + LocatedFileStatusFetcher fetcher = + new LocatedFileStatusFetcher( + listConfig, + new Path[]{basePath}, + true, + EVERYTHING, + true); + Iterable stats = fetcher.getFileStatuses(); + IOStatistics iostats = extractStatistics(fetcher); + LOG.info("Statistics of fetcher: {}", iostats); + Assertions.assertThat(stats) + .describedAs("result of located scan") + .isNotNull() + .flatExtracting(FileStatus::getPath) + .containsExactlyInAnyOrder( + emptyFile, + subdirFile, + subdir2File1, + subdir2File2); + assertListCount(fetcher, EXPECTED_LIST_COUNT); + } + + /** + * Run a located file status fetcher against a file. + */ + @Test + public void testLocatedFileStatusScanFile() throws Throwable { + // pass in a file as the base of the scan. + describe("LocatedFileStatusFetcher with file %s", subdirFile); + listConfig.setInt(LIST_STATUS_NUM_THREADS, 16); + LocatedFileStatusFetcher fetcher + = new LocatedFileStatusFetcher( + listConfig, + new Path[]{subdirFile}, + true, + TEXT_FILE, + true); + Iterable stats = fetcher.getFileStatuses(); + Assertions.assertThat(stats) + .describedAs("result of located scan") + .isNotNull() + .flatExtracting(FileStatus::getPath) + .containsExactly(subdirFile); + IOStatistics ioStatistics = fetcher.getIOStatistics(); + Assertions.assertThat(ioStatistics) + .describedAs("IO statistics of %s", fetcher) + .isNull(); + } + } 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..88e0cef2a34aa 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.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/ITestS3AMetrics.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetrics.java index 972c665438e9a..3bfe69c2bca91 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetrics.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetrics.java @@ -21,11 +21,15 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.metrics2.lib.MutableCounterLong; + +import org.assertj.core.api.Assertions; import org.junit.Test; import java.io.IOException; import java.io.InputStream; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsSourceToString; + /** * Test s3a performance metrics register and output. */ @@ -51,17 +55,34 @@ public void testStreamStatistics() throws IOException { Path file = path("testStreamStatistics"); byte[] data = "abcdefghijklmnopqrstuvwxyz".getBytes(); ContractTestUtils.createFile(fs, file, false, data); - - try (InputStream inputStream = fs.open(file)) { + InputStream inputStream = fs.open(file); + try { while (inputStream.read(data) != -1) { LOG.debug("Read batch of data from input stream..."); } + LOG.info("Final stream statistics: {}", + ioStatisticsSourceToString(inputStream)); + } finally { + // this is not try-with-resources only to aid debugging + inputStream.close(); } + final String statName = Statistic.STREAM_READ_BYTES.getSymbol(); + + final S3AInstrumentation instrumentation = fs.getInstrumentation(); + + final long counterValue = instrumentation.getCounterValue(statName); + + final int expectedBytesRead = 26; + Assertions.assertThat(counterValue) + .describedAs("Counter %s from instrumentation %s", + statName, instrumentation) + .isEqualTo(expectedBytesRead); MutableCounterLong read = (MutableCounterLong) - fs.getInstrumentation().getRegistry() - .get(Statistic.STREAM_SEEK_BYTES_READ.getSymbol()); - assertEquals("Stream statistics were not merged", 26, read.value()); + instrumentation.getRegistry() + .get(statName); + assertEquals("Stream statistics were not merged", expectedBytesRead, + read.value()); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java index d0d42b89f1027..2f0599dc8c374 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java @@ -276,8 +276,14 @@ public void testS3AToStringUnitialized() throws Throwable { } } + @Test + public void testS3AIOStatisticsUninitialized() throws Throwable { + try (S3AFileSystem fs = new S3AFileSystem()) { + fs.getIOStatistics(); + } + + } /** -<<<<<<< ours * Verify that paths with a trailing "/" are fixed up. */ @Test 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..3d7ee0882efa4 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 @@ -21,13 +21,23 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; +import org.apache.hadoop.fs.statistics.StoreStatisticNames; +import org.apache.hadoop.fs.statistics.StreamStatisticNames; import org.apache.hadoop.io.IOUtils; +import org.assertj.core.api.Assertions; import org.junit.Test; import java.io.IOException; -import static org.apache.hadoop.fs.s3a.Statistic.STREAM_SEEK_BYTES_READ; +import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_BYTES; +import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_BYTES_READ_CLOSE; +import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_TOTAL_BYTES; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.demandStringifyIOStatisticsSource; /** * Integration test for calling @@ -40,6 +50,8 @@ */ public class ITestS3AUnbuffer extends AbstractS3ATestBase { + public static final int FILE_LENGTH = 16; + private Path dest; @Override @@ -48,7 +60,7 @@ public void setup() throws Exception { dest = path("ITestS3AUnbuffer"); describe("ITestS3AUnbuffer"); - byte[] data = ContractTestUtils.dataset(16, 'a', 26); + byte[] data = ContractTestUtils.dataset(FILE_LENGTH, 'a', 26); ContractTestUtils.writeDataset(getFileSystem(), dest, data, data.length, 16, true); } @@ -57,13 +69,41 @@ public void setup() throws Exception { public void testUnbuffer() throws IOException { describe("testUnbuffer"); + IOStatisticsSnapshot iostats = new IOStatisticsSnapshot(); // Open file, read half the data, and then call unbuffer try (FSDataInputStream inputStream = getFileSystem().open(dest)) { assertTrue(inputStream.getWrappedStream() instanceof S3AInputStream); - readAndAssertBytesRead(inputStream, 8); + int bytesToRead = 8; + readAndAssertBytesRead(inputStream, bytesToRead); assertTrue(isObjectStreamOpen(inputStream)); + assertTrue("No IOstatistics from " + inputStream, + iostats.aggregate(inputStream.getIOStatistics())); + verifyStatisticCounterValue(iostats, + StreamStatisticNames.STREAM_READ_BYTES, + bytesToRead); + verifyStatisticCounterValue(iostats, + StoreStatisticNames.ACTION_HTTP_GET_REQUEST, + 1); + + // do the unbuffering inputStream.unbuffer(); + // audit the updated statistics + IOStatistics st2 = inputStream.getIOStatistics(); + + // the unbuffered operation must be tracked + verifyStatisticCounterValue(st2, + StreamStatisticNames.STREAM_READ_UNBUFFERED, + 1); + + // all other counter values consistent. + verifyStatisticCounterValue(st2, + StreamStatisticNames.STREAM_READ_BYTES, + bytesToRead); + verifyStatisticCounterValue(st2, + StoreStatisticNames.ACTION_HTTP_GET_REQUEST, + 1); + // Check the the wrapped stream is closed assertFalse(isObjectStreamOpen(inputStream)); } @@ -71,7 +111,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 @@ -79,36 +119,73 @@ public void testUnbufferStreamStatistics() throws IOException { describe("testUnbufferStreamStatistics"); // Validate bytesRead is updated correctly + S3AFileSystem fs = getFileSystem(); S3ATestUtils.MetricDiff bytesRead = new S3ATestUtils.MetricDiff( - getFileSystem(), STREAM_SEEK_BYTES_READ); + fs, STREAM_READ_BYTES); + S3ATestUtils.MetricDiff totalBytesRead = new S3ATestUtils.MetricDiff( + fs, STREAM_READ_TOTAL_BYTES); + S3ATestUtils.MetricDiff bytesReadInClose = new S3ATestUtils.MetricDiff( + fs, STREAM_READ_BYTES_READ_CLOSE); // Open file, read half the data, and then call unbuffer FSDataInputStream inputStream = null; + int firstBytesToRead = 8; + + int secondBytesToRead = 1; + long expectedFinalBytesRead; + long expectedTotalBytesRead; + + Object streamStatsStr; try { - inputStream = getFileSystem().open(dest); + inputStream = fs.open(dest); + streamStatsStr = demandStringifyIOStatisticsSource(inputStream); - readAndAssertBytesRead(inputStream, 8); + LOG.info("initial stream statistics {}", streamStatsStr); + readAndAssertBytesRead(inputStream, firstBytesToRead); + LOG.info("stream statistics after read {}", streamStatsStr); inputStream.unbuffer(); // Validate that calling unbuffer updates the input stream statistics - bytesRead.assertDiffEquals(8); + bytesRead.assertDiffEquals(firstBytesToRead); + final long bytesInUnbuffer = bytesReadInClose.diff(); + totalBytesRead.assertDiffEquals(firstBytesToRead + bytesInUnbuffer); // Validate that calling unbuffer twice in a row updates the statistics // correctly - readAndAssertBytesRead(inputStream, 4); + bytesReadInClose.reset(); + bytesRead.reset(); + readAndAssertBytesRead(inputStream, secondBytesToRead); inputStream.unbuffer(); - bytesRead.assertDiffEquals(12); + LOG.info("stream statistics after second read {}", streamStatsStr); + bytesRead.assertDiffEquals(secondBytesToRead); + final long bytesInClose = bytesReadInClose.diff(); + expectedFinalBytesRead = firstBytesToRead + secondBytesToRead; + expectedTotalBytesRead = expectedFinalBytesRead + + bytesInUnbuffer + bytesInClose; + + totalBytesRead.assertDiffEquals(expectedTotalBytesRead); } finally { + LOG.info("Closing stream"); IOUtils.closeStream(inputStream); } + LOG.info("stream statistics after close {}", streamStatsStr); // Validate that closing the file does not further change the statistics - bytesRead.assertDiffEquals(12); + totalBytesRead.assertDiffEquals(expectedTotalBytesRead); // 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); + S3AInputStreamStatistics streamStatistics = ((S3AInputStream) inputStream + .getWrappedStream()) + .getS3AStreamStatistics(); + Assertions.assertThat(streamStatistics) + .describedAs("Stream statistics %s", streamStatistics) + .hasFieldOrPropertyWithValue("bytesRead", + expectedFinalBytesRead) + .hasFieldOrPropertyWithValue("totalBytesRead", expectedTotalBytesRead); + assertEquals("S3AInputStream statistics were not updated properly in " + + streamStatsStr, + expectedFinalBytesRead, + streamStatistics.getBytesRead()); } private boolean isObjectStreamOpen(FSDataInputStream inputStream) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardEmptyDirs.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardEmptyDirs.java index 41110b9616cc3..bd69ef28669bb 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardEmptyDirs.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardEmptyDirs.java @@ -225,7 +225,7 @@ public void testTombstonesAndEmptyDirectories() throws Throwable { // if DDB is the metastore, then we expect no FS requests to be made // at all. S3ATestUtils.MetricDiff listMetric = new S3ATestUtils.MetricDiff(fs, - Statistic.OBJECT_LIST_REQUESTS); + Statistic.OBJECT_LIST_REQUEST); S3ATestUtils.MetricDiff getMetric = new S3ATestUtils.MetricDiff(fs, Statistic.OBJECT_METADATA_REQUESTS); // do a getFile status with empty dir flag 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 2e7fdc77ed61c..1570e10dd05c4 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,14 @@ 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.statistics.CommitterStatistics; +import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext; import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; +import org.apache.hadoop.fs.statistics.DurationTrackerFactory; import org.apache.hadoop.util.Progressable; +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.stubDurationTrackerFactory; import static org.apache.hadoop.thirdparty.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 +86,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 +147,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 +360,17 @@ public String toString() { } @Override - public S3AInstrumentation.CommitterStatistics newCommitterStatistics() { - return instrumentation.newCommitterStatistics(); + public CommitterStatistics newCommitterStatistics() { + return EmptyS3AStatisticsContext.EMPTY_COMMITTER_STATISTICS; } @Override public void operationRetried(Exception ex) { /** no-op */ } + + @Override + protected DurationTrackerFactory getDurationTrackerFactory() { + return stubDurationTrackerFactory(); + } } 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..868ec2c36cd03 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.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 c52bf3127deb9..68a064a0111bd 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 @@ -40,6 +40,8 @@ import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum; import org.apache.hadoop.fs.s3a.impl.StoreContext; import org.apache.hadoop.fs.s3a.impl.StoreContextBuilder; +import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics; +import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext; import org.apache.hadoop.fs.s3a.s3guard.MetadataStore; import org.apache.hadoop.fs.s3a.s3guard.MetadataStoreCapabilities; import org.apache.hadoop.fs.s3a.s3guard.S3Guard; @@ -54,6 +56,7 @@ import org.apache.hadoop.service.ServiceOperations; import org.apache.hadoop.util.BlockingThreadPoolExecutorService; import org.apache.hadoop.util.ReflectionUtils; +import org.apache.hadoop.util.functional.CallableRaisingIOE; import com.amazonaws.auth.AWSCredentialsProvider; import org.hamcrest.core.Is; @@ -836,9 +839,9 @@ public static void removeBaseAndBucketOverrides( * @param type of operation. */ public static void callQuietly(final Logger log, - final Invoker.Operation operation) { + final CallableRaisingIOE operation) { try { - operation.execute(); + operation.apply(); } catch (Exception e) { log.info(e.toString(), e); } @@ -933,7 +936,7 @@ public static StoreContext createMockStoreContext( .setExecutorCapacity(DEFAULT_EXECUTOR_CAPACITY) .setInvoker( new Invoker(RetryPolicies.TRY_ONCE_THEN_FAIL, Invoker.LOG_EVENT)) - .setInstrumentation(new S3AInstrumentation(name)) + .setInstrumentation(new EmptyS3AStatisticsContext()) .setStorageStatistics(new S3AStorageStatistics()) .setInputPolicy(S3AInputPolicy.Normal) .setChangeDetectionPolicy( @@ -1230,7 +1233,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/TestListing.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestListing.java index 34726741835d1..77ba31c3ce8ea 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestListing.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestListing.java @@ -18,9 +18,13 @@ package org.apache.hadoop.fs.s3a; -import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; + +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Test; @@ -31,8 +35,11 @@ import java.util.NoSuchElementException; import java.util.Set; -import static org.apache.hadoop.fs.s3a.S3AUtils.ACCEPT_ALL; -import static org.apache.hadoop.fs.s3a.Listing.ProvidedFileStatusIterator; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.extractStatistics; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_LIST_REQUEST; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** * Place for the S3A listing classes; keeps all the small classes under control. @@ -40,11 +47,18 @@ public class TestListing extends AbstractS3AMockTest { private static class MockRemoteIterator implements - RemoteIterator { + RemoteIterator, IOStatisticsSource { + + private final IOStatisticsStore ioStatistics; + private Iterator iterator; MockRemoteIterator(Collection source) { iterator = source.iterator(); + this.ioStatistics = iostatisticsStore() + .withDurationTracking(OBJECT_LIST_REQUEST) + .build(); + ioStatistics.incrementCounter(OBJECT_LIST_REQUEST); } public boolean hasNext() { @@ -54,6 +68,11 @@ public boolean hasNext() { public S3AFileStatus next() { return iterator.next(); } + + @Override + public IOStatistics getIOStatistics() { + return ioStatistics; + } } private S3AFileStatus blankFileStatus(Path path) { @@ -65,11 +84,9 @@ public void testTombstoneReconcilingIterator() throws Exception { Path parent = new Path("/parent"); Path liveChild = new Path(parent, "/liveChild"); Path deletedChild = new Path(parent, "/deletedChild"); - Path[] allFiles = {parent, liveChild, deletedChild}; - Path[] liveFiles = {parent, liveChild}; Listing listing = fs.getListing(); - Collection statuses = new ArrayList<>(); + Collection statuses = new ArrayList<>(); statuses.add(blankFileStatus(parent)); statuses.add(blankFileStatus(liveChild)); statuses.add(blankFileStatus(deletedChild)); @@ -92,7 +109,13 @@ public void testTombstoneReconcilingIterator() throws Exception { while (reconcilingIterator.hasNext()) { actualPaths.add(reconcilingIterator.next().getPath()); } - Assert.assertTrue(actualPaths.equals(expectedPaths)); + Assertions.assertThat(actualPaths) + .describedAs("paths from iterator") + .isEqualTo(expectedPaths); + + // now verify the stats went all the way through. + IOStatistics iostats = extractStatistics(reconcilingIterator); + verifyStatisticCounterValue(iostats, OBJECT_LIST_REQUEST, 1); } @Test @@ -104,18 +127,13 @@ public void testProvidedFileStatusIteratorEnd() throws Exception { S3AFileStatus[] statuses = { s3aStatus }; - ProvidedFileStatusIterator it = new ProvidedFileStatusIterator(statuses, - ACCEPT_ALL, new Listing.AcceptAllButS3nDirs()); + RemoteIterator it = Listing.toProvidedFileStatusIterator( + statuses); Assert.assertTrue("hasNext() should return true first time", it.hasNext()); - Assert.assertNotNull("first element should not be null", it.next()); + Assert.assertEquals("first element from iterator", + s3aStatus, it.next()); Assert.assertFalse("hasNext() should now be false", it.hasNext()); - try { - it.next(); - Assert.fail("next() should have thrown exception"); - } catch (NoSuchElementException e) { - // Correct behavior. Any other exceptions are propagated as failure. - return; - } + intercept(NoSuchElementException.class, it::next); } } 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..284718bd75c12 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.statistics.impl.EmptyS3AStatisticsContext; import org.apache.hadoop.util.Progressable; import org.junit.Before; import org.junit.Test; @@ -30,7 +31,6 @@ import java.util.concurrent.ExecutorService; import static org.apache.hadoop.test.LambdaTestUtils.intercept; -import static org.junit.Assert.*; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; @@ -50,11 +50,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 +69,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..4d3930fbc3c1d 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.statistics.impl.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/ITestAssumeRole.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java index 4f6a1ff417873..6b2290b44dfcb 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java @@ -51,6 +51,7 @@ import org.apache.hadoop.fs.s3a.commit.files.PendingSet; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool; +import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; import static org.apache.hadoop.fs.s3a.Constants.*; @@ -62,6 +63,7 @@ import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.forbidden; import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.newAssumedRoleConfig; import static org.apache.hadoop.fs.s3a.s3guard.S3GuardToolTestHelper.exec; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsSourceToString; import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains; import static org.apache.hadoop.test.LambdaTestUtils.*; @@ -573,8 +575,11 @@ public void testRestrictedCommitActions() throws Throwable { .addResources(directory(writeableDir)) ); roleFS = (S3AFileSystem) writeableDir.getFileSystem(conf); - CommitOperations fullOperations = new CommitOperations(fs); - CommitOperations operations = new CommitOperations(roleFS); + CommitterStatistics committerStatistics = fs.newCommitterStatistics(); + CommitOperations fullOperations = new CommitOperations(fs, + committerStatistics); + CommitOperations operations = new CommitOperations(roleFS, + committerStatistics); File localSrc = File.createTempFile("source", ""); writeCSVData(localSrc); @@ -608,7 +613,7 @@ public void testRestrictedCommitActions() throws Throwable { name + CommitConstants.PENDING_SUFFIX), true); assertTrue(src.delete()); })); - progress.assertCount("Process counter is not expected", + progress.assertCount("progress counter is not expected", range); try { @@ -652,6 +657,8 @@ public void testRestrictedCommitActions() throws Throwable { } finally { LOG.info("Cleanup"); fullOperations.abortPendingUploadsUnderPath(readOnlyDir); + LOG.info("Committer statistics {}", + ioStatisticsSourceToString(committerStatistics)); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestRestrictedReadAccess.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestRestrictedReadAccess.java index 1555217842144..402469eb3b736 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestRestrictedReadAccess.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestRestrictedReadAccess.java @@ -724,7 +724,7 @@ protected FileStatus[] globFS( S3ATestUtils.MetricDiff getMetric = new S3ATestUtils.MetricDiff(fs, Statistic.OBJECT_METADATA_REQUESTS); S3ATestUtils.MetricDiff listMetric = new S3ATestUtils.MetricDiff(fs, - Statistic.OBJECT_LIST_REQUESTS); + Statistic.OBJECT_LIST_REQUEST); FileStatus[] st; try { st = filter == null diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ProgressCounter.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ProgressCounter.java index 15a57152091b7..362e674e13a54 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ProgressCounter.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ProgressCounter.java @@ -18,6 +18,8 @@ package org.apache.hadoop.fs.s3a.auth; +import java.util.concurrent.atomic.AtomicLong; + import org.apache.hadoop.util.Progressable; import static org.junit.Assert.assertEquals; @@ -27,14 +29,14 @@ */ public class ProgressCounter implements Progressable { - private long count; + private final AtomicLong count = new AtomicLong(); public void progress() { - count++; + count.incrementAndGet(); } public long getCount() { - return count; + return count.get(); } public void assertCount(String message, int expected) { 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..7be5b754f2067 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.statistics.StatisticsFromAwsSdk; +import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext; 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; @@ -211,7 +214,7 @@ public void testGetDTfromFileSystem() throws Throwable { S3ATestUtils.MetricDiff invocationDiff = new S3ATestUtils.MetricDiff(fs, Statistic.INVOCATION_GET_DELEGATION_TOKEN); S3ATestUtils.MetricDiff issueDiff = new S3ATestUtils.MetricDiff(fs, - Statistic.DELEGATION_TOKENS_ISSUED); + Statistic.DELEGATION_TOKEN_ISSUED); Token token = requireNonNull(fs.getDelegationToken(""), "no token from filesystem " + fs); @@ -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(); @@ -365,7 +371,7 @@ public void testDelegatedFileSystem() throws Throwable { S3ATestUtils.MetricDiff issueDiff = new S3ATestUtils.MetricDiff( delegatedFS, - Statistic.DELEGATION_TOKENS_ISSUED); + Statistic.DELEGATION_TOKEN_ISSUED); // verify that the FS returns the existing token when asked // so that chained deployments will work @@ -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/commit/AbstractCommitITest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java index 9947ece0b8b4a..7a3ad655d0b35 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java @@ -53,6 +53,7 @@ import static org.apache.hadoop.fs.s3a.MultipartTestUtils.listMultipartUploads; import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToString; /** * Base test suite for committer operations. @@ -460,6 +461,8 @@ public static SuccessData validateSuccessFile(final Path outputPath, commitDetails); LOG.info("Committer statistics: \n{}", successData.dumpMetrics(" ", " = ", "\n")); + LOG.info("Job IOStatistics: \n{}", + ioStatisticsToString(successData.getIOStatistics())); LOG.info("Diagnostics\n{}", successData.dumpDiagnostics(" ", " = ", "\n")); if (!committerName.isEmpty()) { @@ -486,7 +489,7 @@ public static SuccessData validateSuccessFile(final Path outputPath, * @throws IOException failure to find/load the file * @throws AssertionError file is 0-bytes long, */ - public static SuccessData loadSuccessFile(final S3AFileSystem fs, + public static SuccessData loadSuccessFile(final FileSystem fs, final Path outputPath, final String origin) throws IOException { ContractTestUtils.assertPathExists(fs, "Output directory " + outputPath @@ -505,7 +508,9 @@ public static SuccessData loadSuccessFile(final S3AFileSystem fs, + success + " from " + origin + "; an S3A committer was not used", status.getLen() > 0); - LOG.info("Loading committer success file {}", success); + String body = ContractTestUtils.readUTF8(fs, success, -1); + LOG.info("Loading committer success file {}. Actual contents=\n{}", success, + body); return SuccessData.load(fs, success); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java index 89d505f20af81..1108194fb5cb5 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java @@ -43,6 +43,7 @@ import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.commit.files.SuccessData; import org.apache.hadoop.fs.s3a.commit.magic.MagicS3GuardCommitter; +import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.MapFile; @@ -77,6 +78,9 @@ import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.FS_S3A_COMMITTER_UUID; import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.FS_S3A_COMMITTER_UUID_SOURCE; import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.SPARK_WRITE_UUID; +import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_TASKS_SUCCEEDED; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticCounter; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsSourceToString; import static org.apache.hadoop.test.LambdaTestUtils.*; /** @@ -1339,9 +1343,28 @@ public void testOutputFormatIntegration() throws Throwable { assertTrue("Committer does not have data to commit " + committer, committer.needsTaskCommit(tContext)); commitTask(committer, tContext); + // at this point the committer tasks stats should be current. + IOStatisticsSnapshot snapshot = new IOStatisticsSnapshot( + committer.getIOStatistics()); + String commitsCompleted = COMMITTER_TASKS_SUCCEEDED.getSymbol(); + assertThatStatisticCounter(snapshot, commitsCompleted) + .describedAs("task commit count") + .isEqualTo(1L); + + commitJob(committer, jContext); + LOG.info("committer iostatistics {}", + ioStatisticsSourceToString(committer)); + // validate output - verifySuccessMarker(outDir, committer.getUUID()); + SuccessData successData = verifySuccessMarker(outDir, committer.getUUID()); + + // the task commit count should get through the job commit + IOStatisticsSnapshot successStats = successData.getIOStatistics(); + LOG.info("loaded statistics {}", successStats); + assertThatStatisticCounter(successStats, commitsCompleted) + .describedAs("task commit count") + .isEqualTo(1L); } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/integration/ITestS3ACommitterMRJob.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/integration/ITestS3ACommitterMRJob.java index 77f8c48baba0e..79838d6f00b46 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/integration/ITestS3ACommitterMRJob.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/integration/ITestS3ACommitterMRJob.java @@ -80,7 +80,6 @@ import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.FS_S3A_COMMITTER_UUID; import static org.apache.hadoop.fs.s3a.commit.staging.Paths.getMultipartUploadCommitsDirectory; import static org.apache.hadoop.fs.s3a.commit.staging.StagingCommitterConstants.STAGING_UPLOADS; -import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** * Test an MR Job with all the different committers. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/MockedStagingCommitter.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/MockedStagingCommitter.java index 47383b78a4d3b..d3da8185c8d65 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/MockedStagingCommitter.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/MockedStagingCommitter.java @@ -28,6 +28,7 @@ import org.apache.hadoop.fs.s3a.MockS3AFileSystem; import org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase.ClientErrors; import org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase.ClientResults; +import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.TaskAttemptContext; @@ -76,7 +77,8 @@ public void commitJob(JobContext context) throws IOException { @Override protected void maybeCreateSuccessMarker(JobContext context, - List filenames) + List filenames, + final IOStatisticsSnapshot ioStatistics) throws IOException { //skipped } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestPartialRenamesDeletes.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestPartialRenamesDeletes.java index 3c67ddb4f1e37..1ea38af4f6c97 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestPartialRenamesDeletes.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestPartialRenamesDeletes.java @@ -58,8 +58,9 @@ import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; import static org.apache.hadoop.fs.s3a.S3AUtils.applyLocatedFiles; import static org.apache.hadoop.fs.s3a.Statistic.FILES_DELETE_REJECTED; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_BULK_DELETE_REQUEST; import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_DELETE_OBJECTS; -import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_DELETE_REQUESTS; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_DELETE_REQUEST; import static org.apache.hadoop.fs.s3a.auth.RoleModel.Effects; import static org.apache.hadoop.fs.s3a.auth.RoleModel.Statement; import static org.apache.hadoop.fs.s3a.auth.RoleModel.directory; @@ -666,7 +667,9 @@ public void testPartialDirDelete() throws Throwable { // this set can be deleted by the role FS MetricDiff rejectionCount = new MetricDiff(roleFS, FILES_DELETE_REJECTED); - MetricDiff deleteVerbCount = new MetricDiff(roleFS, OBJECT_DELETE_REQUESTS); + MetricDiff deleteVerbCount = new MetricDiff(roleFS, OBJECT_DELETE_REQUEST); + MetricDiff bulkDeleteVerbCount = new MetricDiff(roleFS, + OBJECT_BULK_DELETE_REQUEST); MetricDiff deleteObjectCount = new MetricDiff(roleFS, OBJECT_DELETE_OBJECTS); @@ -675,7 +678,9 @@ public void testPartialDirDelete() throws Throwable { if (multiDelete) { // multi-delete status checks extractCause(MultiObjectDeleteException.class, ex); - deleteVerbCount.assertDiffEquals("Wrong delete request count", 1); + deleteVerbCount.assertDiffEquals("Wrong delete request count", 0); + bulkDeleteVerbCount.assertDiffEquals("Wrong bulk delete request count", + 1); deleteObjectCount.assertDiffEquals("Number of keys in delete request", readOnlyFiles.size()); rejectionCount.assertDiffEquals("Wrong rejection count", 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/performance/AbstractS3ACostTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java index 2b2fb7c9caca0..4a2d695e6a227 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java @@ -20,6 +20,7 @@ import java.io.FileNotFoundException; import java.io.IOException; +import java.util.EnumSet; import java.util.Set; import java.util.concurrent.Callable; @@ -31,16 +32,19 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.s3a.AbstractS3ATestBase; +import org.apache.hadoop.fs.s3a.Constants; import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.Statistic; import org.apache.hadoop.fs.s3a.Tristate; import org.apache.hadoop.fs.s3a.impl.DirectoryPolicy; import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum; +import org.apache.hadoop.fs.s3a.statistics.StatisticTypeEnum; import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; -import static org.apache.hadoop.fs.s3a.Statistic.*; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_BULK_DELETE_REQUEST; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_DELETE_REQUEST; import static org.apache.hadoop.fs.s3a.performance.OperationCost.*; import static org.apache.hadoop.fs.s3a.performance.OperationCostValidator.expect; import static org.apache.hadoop.fs.s3a.performance.OperationCostValidator.probe; @@ -83,6 +87,18 @@ public class AbstractS3ACostTest extends AbstractS3ATestBase { private OperationCostValidator costValidator; + /** + * Is bulk deletion enabled? + */ + private boolean isBulkDelete; + + /** + * Which statistic measures marker deletion? + * this is the bulk delete statistic by default; + * if that is disabled it becomes the single delete counter. + */ + private Statistic deleteMarkerStatistic; + public AbstractS3ACostTest( final boolean s3guard, final boolean keepMarkers, @@ -149,23 +165,24 @@ public void setup() throws Exception { .isEqualTo(isKeepingMarkers() ? DirectoryPolicy.MarkerPolicy.Keep : DirectoryPolicy.MarkerPolicy.Delete); - - // insert new metrics so as to keep the list sorted - costValidator = OperationCostValidator.builder(getFileSystem()) - .withMetrics( - DIRECTORIES_CREATED, - DIRECTORIES_DELETED, - FAKE_DIRECTORIES_DELETED, - FILES_DELETED, - INVOCATION_COPY_FROM_LOCAL_FILE, - OBJECT_COPY_REQUESTS, - OBJECT_DELETE_REQUESTS, - OBJECT_DELETE_OBJECTS, - OBJECT_LIST_REQUESTS, - OBJECT_METADATA_REQUESTS, - OBJECT_PUT_BYTES, - OBJECT_PUT_REQUESTS) - .build(); + // All counter statistics of the filesystem are added as metrics. + // Durations too, as they have counters of success and failure. + OperationCostValidator.Builder builder = OperationCostValidator.builder( + getFileSystem()); + EnumSet.allOf(Statistic.class).stream() + .filter(s -> + s.getType() == StatisticTypeEnum.TYPE_COUNTER + || s.getType() == StatisticTypeEnum.TYPE_DURATION) + .forEach(s -> builder.withMetric(s)); + costValidator = builder.build(); + + // determine bulk delete settings + final Configuration fsConf = getFileSystem().getConf(); + isBulkDelete = fsConf.getBoolean(Constants.ENABLE_MULTI_DELETE, + true); + deleteMarkerStatistic = isBulkDelete() + ? OBJECT_BULK_DELETE_REQUEST + : OBJECT_DELETE_REQUEST; } public void assumeUnguarded() { @@ -369,8 +386,7 @@ protected E verifyMetricsIntercepting( * Execute a closure expecting an exception. * @param clazz type of exception * @param text text to look for in exception (optional) - * @param head expected head request count. - * @param list expected list request count. + * @param cost expected cost declaration. * @param eval closure to evaluate * @param return type of closure * @param exception type @@ -647,4 +663,20 @@ protected void assertEmptyDirStatus(final S3AFileStatus status, getFileSystem(), status.getPath()))) .isEqualTo(expected); } + + /** + * Is bulk deletion enabled? + */ + protected boolean isBulkDelete() { + return isBulkDelete; + } + + /** + * Which statistic measures marker deletion? + * this is the bulk delete statistic by default; + * if that is disabled it becomes the single delete counter. + */ + protected Statistic getDeleteMarkerStatistic() { + return deleteMarkerStatistic; + } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java index 0b5afc6178522..f5d223932d4c4 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java @@ -47,6 +47,10 @@ * Use metrics to assert about the cost of file API calls. *

* Parameterized on guarded vs raw. and directory marker keep vs delete. + * There's extra complexity related to bulk/non-bulk delete calls. + * If bulk deletes are disabled, many more requests are made to delete + * parent directories. The counters of objects deleted are constant + * irrespective of the delete mode. */ @RunWith(Parameterized.class) public class ITestS3ADeleteCost extends AbstractS3ACostTest { @@ -112,18 +116,19 @@ public void testDeleteSingleFileInDir() throws Throwable { // if deleting markers, look for the parent too probe(rawAndDeleting, OBJECT_METADATA_REQUESTS, FILESTATUS_FILE_PROBE_H + FILESTATUS_DIR_PROBE_H), - withWhenRaw(OBJECT_LIST_REQUESTS, + withWhenRaw(OBJECT_LIST_REQUEST, FILESTATUS_FILE_PROBE_L + FILESTATUS_DIR_PROBE_L), with(DIRECTORIES_DELETED, 0), with(FILES_DELETED, 1), // keeping: create no parent dirs or delete parents withWhenKeeping(DIRECTORIES_CREATED, 0), - withWhenKeeping(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), + withWhenKeeping(OBJECT_DELETE_OBJECTS, DELETE_OBJECT_REQUEST), // deleting: create a parent and delete any of its parents withWhenDeleting(DIRECTORIES_CREATED, 1), - withWhenDeleting(OBJECT_DELETE_REQUESTS, + // two objects will be deleted + withWhenDeleting(OBJECT_DELETE_OBJECTS, DELETE_OBJECT_REQUEST + DELETE_MARKER_REQUEST) ); @@ -160,7 +165,7 @@ public void testDeleteFileInDir() throws Throwable { // if deleting markers, look for the parent too probe(rawAndDeleting, OBJECT_METADATA_REQUESTS, FILESTATUS_FILE_PROBE_H + FILESTATUS_DIR_PROBE_H), - withWhenRaw(OBJECT_LIST_REQUESTS, + withWhenRaw(OBJECT_LIST_REQUEST, FILESTATUS_FILE_PROBE_L + FILESTATUS_DIR_PROBE_L), with(DIRECTORIES_DELETED, 0), with(FILES_DELETED, 1), @@ -169,10 +174,10 @@ public void testDeleteFileInDir() throws Throwable { with(DIRECTORIES_CREATED, 0), // keeping: create no parent dirs or delete parents - withWhenKeeping(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), + withWhenKeeping(OBJECT_DELETE_REQUEST, DELETE_OBJECT_REQUEST), // deleting: create a parent and delete any of its parents - withWhenDeleting(OBJECT_DELETE_REQUESTS, + withWhenDeleting(OBJECT_DELETE_REQUEST, DELETE_OBJECT_REQUEST)); } @@ -199,22 +204,20 @@ public void testDirMarkersSubdir() throws Throwable { LOG.info("creating sub directory {}", subDir); // one dir created, possibly a parent removed + final int fakeDirectoriesToDelete = directoriesInPath(subDir) - 1; verifyMetrics(() -> { mkdirs(subDir); return "after mkdir(subDir) " + getMetricSummary(); }, with(DIRECTORIES_CREATED, 1), with(DIRECTORIES_DELETED, 0), - withWhenKeeping(OBJECT_DELETE_REQUESTS, 0), + withWhenKeeping(getDeleteMarkerStatistic(), 0), withWhenKeeping(FAKE_DIRECTORIES_DELETED, 0), - withWhenDeleting(OBJECT_DELETE_REQUESTS, DELETE_MARKER_REQUEST), + withWhenDeleting(getDeleteMarkerStatistic(), + isBulkDelete() ? DELETE_MARKER_REQUEST : fakeDirectoriesToDelete), // delete all possible fake dirs above the subdirectory withWhenDeleting(FAKE_DIRECTORIES_DELETED, - directoriesInPath(subDir) - 1)); - - int dirDeleteRequests = 1; - int fileDeleteRequests = 0; - int totalDeleteRequests = dirDeleteRequests + fileDeleteRequests; + fakeDirectoriesToDelete)); LOG.info("About to delete {}", parent); // now delete the deep tree. @@ -223,8 +226,6 @@ public void testDirMarkersSubdir() throws Throwable { return "deleting parent dir " + parent + " " + getMetricSummary(); }, - // two directory markers will be deleted in a single request - with(OBJECT_DELETE_REQUESTS, totalDeleteRequests), // keeping: the parent dir marker needs deletion alongside // the subdir one. withWhenKeeping(OBJECT_DELETE_OBJECTS, dirsCreated), @@ -265,6 +266,7 @@ public void testDirMarkersFileCreation() throws Throwable { // creating a file should trigger demise of the src dir marker // unless markers are being kept + final int directories = directoriesInPath(srcDir); verifyMetrics(() -> { file(new Path(srcDir, "source.txt")); LOG.info("Metrics: {}\n{}", getMetricSummary(), getFileSystem()); @@ -273,12 +275,13 @@ public void testDirMarkersFileCreation() throws Throwable { with(DIRECTORIES_CREATED, 0), with(DIRECTORIES_DELETED, 0), // keeping: no delete operations. - withWhenKeeping(OBJECT_DELETE_REQUESTS, 0), + withWhenKeeping(getDeleteMarkerStatistic(), 0), withWhenKeeping(FAKE_DIRECTORIES_DELETED, 0), // delete all possible fake dirs above the file - withWhenDeleting(OBJECT_DELETE_REQUESTS, 1), + withWhenDeleting(getDeleteMarkerStatistic(), + isBulkDelete() ? 1: directories), withWhenDeleting(FAKE_DIRECTORIES_DELETED, - directoriesInPath(srcDir))); + directories)); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ARenameCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ARenameCost.java index 85c70768356e6..0077503e87c1a 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ARenameCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ARenameCost.java @@ -35,6 +35,7 @@ import static org.apache.hadoop.fs.s3a.Statistic.*; import static org.apache.hadoop.fs.s3a.performance.OperationCost.*; +import static org.apache.hadoop.fs.s3a.performance.OperationCostValidator.probe; /** * Use metrics to assert about the cost of file API calls. @@ -97,19 +98,34 @@ public void testRenameFileToDifferentDirectory() throws Throwable { // rename the source file to the destination file. // this tests file rename, not dir rename // as srcFile2 exists, the parent dir of srcFilePath must not be created. + final int directoriesInPath = directoriesInPath(destDir); verifyMetrics(() -> execRename(srcFilePath, destFilePath), whenRaw(RENAME_SINGLE_FILE_DIFFERENT_DIR), with(DIRECTORIES_CREATED, 0), with(DIRECTORIES_DELETED, 0), // keeping: only the core delete operation is issued. - withWhenKeeping(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), + withWhenKeeping(OBJECT_DELETE_REQUEST, DELETE_OBJECT_REQUEST), withWhenKeeping(FAKE_DIRECTORIES_DELETED, 0), + withWhenKeeping(OBJECT_DELETE_OBJECTS, 1), + // deleting: delete any fake marker above the destination. - withWhenDeleting(OBJECT_DELETE_REQUESTS, - DELETE_OBJECT_REQUEST + DELETE_MARKER_REQUEST), + // the actual request count depends on whether bulk delete is + // enabled or not + + // no bulk delete: multiple marker calls + probe(isDeleting() && !isBulkDelete(), OBJECT_DELETE_REQUEST, + DELETE_OBJECT_REQUEST + directoriesInPath), + + // bulk delete: split up + probe(isDeleting() && isBulkDelete(), OBJECT_DELETE_REQUEST, + DELETE_OBJECT_REQUEST), + probe(isDeleting() && isBulkDelete(), OBJECT_BULK_DELETE_REQUEST, + DELETE_MARKER_REQUEST), withWhenDeleting(FAKE_DIRECTORIES_DELETED, - directoriesInPath(destDir))); + directoriesInPath), + withWhenDeleting(OBJECT_DELETE_OBJECTS, + directoriesInPath + 1)); assertIsFile(destFilePath); assertIsDirectory(srcDir); @@ -139,7 +155,7 @@ public void testRenameSameDirectory() throws Throwable { whenRaw(RENAME_SINGLE_FILE_SAME_DIR), with(OBJECT_COPY_REQUESTS, 1), with(DIRECTORIES_CREATED, 0), - with(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), + with(OBJECT_DELETE_REQUEST, DELETE_OBJECT_REQUEST), with(FAKE_DIRECTORIES_DELETED, 0)); } @@ -164,7 +180,7 @@ public void testCostOfRootFileRename() throws Throwable { // here we expect there to be no fake directories with(DIRECTORIES_CREATED, 0), // one for the renamed file only - with(OBJECT_DELETE_REQUESTS, + with(OBJECT_DELETE_REQUEST, DELETE_OBJECT_REQUEST), // no directories are deleted: This is root with(DIRECTORIES_DELETED, 0), @@ -196,7 +212,7 @@ public void testCostOfRootFileDelete() throws Throwable { with(DIRECTORIES_DELETED, 0), with(FAKE_DIRECTORIES_DELETED, 0), with(FILES_DELETED, 1), - with(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), + with(OBJECT_DELETE_REQUEST, DELETE_OBJECT_REQUEST), whenRaw(FILE_STATUS_FILE_PROBE)); /* no need to look at parent. */ } finally { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCost.java index 8605d7f2a2b9a..c2e0c04fbe63a 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCost.java @@ -51,6 +51,8 @@ public final class OperationCost { /** * Delete cost when deleting a marker. + * Note: if bulk delete is disabled, this changes to being + * the number of directories deleted. */ public static final int DELETE_MARKER_REQUEST = DELETE_OBJECT_REQUEST; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCostValidator.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCostValidator.java index c351d1b185a32..3f03abbabc3cc 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCostValidator.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCostValidator.java @@ -31,11 +31,15 @@ import org.slf4j.LoggerFactory; import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.fs.s3a.S3AInstrumentation; import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.Statistic; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; +import org.apache.hadoop.metrics2.lib.MutableCounter; +import org.apache.hadoop.metrics2.lib.MutableMetric; import static java.util.Objects.requireNonNull; -import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_LIST_REQUESTS; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_LIST_REQUEST; import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_METADATA_REQUESTS; import static org.apache.hadoop.test.LambdaTestUtils.intercept; @@ -91,15 +95,30 @@ public final class OperationCostValidator { private final Map metricDiffs = new TreeMap<>(); + /** + * Instrumentation's IO Statistics. + */ + private final IOStatisticsStore ioStatistics; + /** * Build the instance. * @param builder builder containing all options. */ private OperationCostValidator(Builder builder) { - builder.metrics.forEach(stat -> - metricDiffs.put(stat.getSymbol(), - new S3ATestUtils.MetricDiff(builder.filesystem, stat))); + S3AFileSystem fs = builder.filesystem; + S3AInstrumentation instrumentation = fs.getInstrumentation(); + for (Statistic stat : builder.metrics) { + String symbol = stat.getSymbol(); + MutableMetric metric = instrumentation.lookupMetric(symbol); + if (metric instanceof MutableCounter) { + // only counters are used in the cost tracking; + // other statistics are ignored. + metricDiffs.put(symbol, + new S3ATestUtils.MetricDiff(fs, stat)); + } + } builder.metrics.clear(); + ioStatistics = instrumentation.getIOStatistics(); } /** @@ -137,6 +156,7 @@ public T exec( ExpectedProbe... expectedA) throws Exception { List expected = Arrays.asList(expectedA); resetMetricDiffs(); + // verify that 1+ probe is enabled assumeProbesEnabled(expected); // if we get here, then yes. @@ -147,8 +167,9 @@ public T exec( "operation returning " + (r != null ? r.toString() : "null"); LOG.info("{}", text); - LOG.info("state {}", this); + LOG.info("state {}", this.toString()); LOG.info("probes {}", expected); + LOG.info("IOStatistics {}", ioStatistics); for (ExpectedProbe ed : expected) { ed.verify(this, text); } @@ -246,7 +267,8 @@ public Builder(final S3AFileSystem filesystem) { * @return this */ public Builder withMetric(Statistic statistic) { - return withMetric(statistic); + metrics.add(statistic); + return this; } /** @@ -330,7 +352,7 @@ public static ExpectedProbe expect( boolean enabled, OperationCost cost) { return probes(enabled, probe(OBJECT_METADATA_REQUESTS, cost.head()), - probe(OBJECT_LIST_REQUESTS, cost.list())); + probe(OBJECT_LIST_REQUEST, cost.list())); } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreAuthoritativeMode.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreAuthoritativeMode.java index aa2b4e77bd4bb..3d9715ceb3ebb 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreAuthoritativeMode.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreAuthoritativeMode.java @@ -54,7 +54,7 @@ import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; import static org.apache.hadoop.fs.s3a.S3AUtils.applyLocatedFiles; -import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_LIST_REQUESTS; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_LIST_REQUEST; import static org.apache.hadoop.fs.s3a.Statistic.S3GUARD_METADATASTORE_AUTHORITATIVE_DIRECTORIES_UPDATED; import static org.apache.hadoop.fs.s3a.Statistic.S3GUARD_METADATASTORE_RECORD_WRITES; import static org.apache.hadoop.fs.s3a.s3guard.AuthoritativeAuditOperation.ERROR_PATH_NOT_AUTH_IN_FS; @@ -330,7 +330,7 @@ public void testListFilesRecursiveWhenAllListingsAreAuthoritative() touchFile(nestedFile2); S3ATestUtils.MetricDiff objListRequests = - new S3ATestUtils.MetricDiff(authFS, OBJECT_LIST_REQUESTS); + new S3ATestUtils.MetricDiff(authFS, OBJECT_LIST_REQUEST); RemoteIterator statusIterator = authFS.listFiles(dir, true); @@ -372,7 +372,7 @@ public void testListFilesRecursiveWhenSomePathsAreNotAuthoritative() touchFile(nestedFile2); S3ATestUtils.MetricDiff objListRequests = - new S3ATestUtils.MetricDiff(authFS, OBJECT_LIST_REQUESTS); + new S3ATestUtils.MetricDiff(authFS, OBJECT_LIST_REQUEST); RemoteIterator statusIterator = authFS.listFiles(dir, true); @@ -766,7 +766,7 @@ private T expectAuthoritativeUpdate( S3ATestUtils.MetricDiff authDirsMarked = new S3ATestUtils.MetricDiff(authFS, S3GUARD_METADATASTORE_AUTHORITATIVE_DIRECTORIES_UPDATED); S3ATestUtils.MetricDiff listRequests = new S3ATestUtils.MetricDiff(authFS, - OBJECT_LIST_REQUESTS); + OBJECT_LIST_REQUEST); final T call = fn.call(); authDirsMarked.assertDiffEquals(updates); listRequests.assertDiffEquals(lists); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java index 0a0cb05828fdc..0f6800f473517 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java @@ -21,6 +21,7 @@ import javax.annotation.Nullable; import java.io.FileNotFoundException; import java.io.IOException; +import java.io.UncheckedIOException; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -49,7 +50,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.StorageStatistics; import org.apache.hadoop.fs.contract.ContractTestUtils; -import org.apache.hadoop.fs.impl.WrappedIOException; import org.apache.hadoop.fs.s3a.AWSServiceThrottledException; import org.apache.hadoop.fs.s3a.Invoker; import org.apache.hadoop.fs.s3a.S3AFileStatus; @@ -532,7 +532,7 @@ public void test_999_delete_all_entries() throws Throwable { LOG.info("Deleting {}", p); list.add(p); }); - } catch (WrappedIOException e) { + } catch (UncheckedIOException e) { // the iterator may have overloaded; swallow if so. if (!(e.getCause() instanceof AWSServiceThrottledException)) { throw e; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java index 8d29c29763fb5..851b1b16eeb75 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java @@ -25,8 +25,7 @@ import com.amazonaws.event.ProgressEvent; import com.amazonaws.event.ProgressEventType; import com.amazonaws.event.ProgressListener; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.s3a.S3ATestUtils; +import org.assertj.core.api.Assertions; import org.junit.FixMethodOrder; import org.junit.Test; import org.junit.runners.MethodSorters; @@ -36,17 +35,24 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.StorageStatistics; import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.s3a.S3AFileSystem; -import org.apache.hadoop.fs.s3a.S3AInstrumentation; +import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.Statistic; +import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics; +import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.util.Progressable; import static org.apache.hadoop.fs.contract.ContractTestUtils.*; import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.demandStringifyIOStatistics; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsSourceToString; +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics; +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.snapshotIOStatistics; /** * Scale test which creates a huge file. @@ -170,7 +176,7 @@ public void test_010_CreateHugeFile() throws IOException { Statistic putBytesPending = Statistic.OBJECT_PUT_BYTES_PENDING; ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer(); - S3AInstrumentation.OutputStreamStatistics streamStatistics; + BlockOutputStreamStatistics streamStatistics; long blocksPer10MB = blocksPerMB * 10; ProgressCallback progress = new ProgressCallback(timer); try (FSDataOutputStream out = fs.create(fileToCreate, @@ -221,8 +227,20 @@ public void test_010_CreateHugeFile() throws IOException { logFSState(); bandwidth(timer, filesize); LOG.info("Statistics after stream closed: {}", streamStatistics); + IOStatistics iostats = snapshotIOStatistics( + retrieveIOStatistics(getFileSystem())); + LOG.info("IOStatistics after upload: {}", + demandStringifyIOStatistics(iostats)); long putRequestCount = storageStatistics.getLong(putRequests); Long putByteCount = storageStatistics.getLong(putBytes); + Assertions.assertThat(putRequestCount) + .describedAs("Put request count from filesystem stats %s", + iostats) + .isGreaterThan(0); + Assertions.assertThat(putByteCount) + .describedAs("putByteCount count from filesystem stats %s", + iostats) + .isGreaterThan(0); LOG.info("PUT {} bytes in {} operations; {} MB/operation", putByteCount, putRequestCount, putByteCount / (putRequestCount * _1MB)); @@ -234,7 +252,7 @@ public void test_010_CreateHugeFile() throws IOException { "Put file " + fileToCreate + " of size " + filesize); if (streamStatistics != null) { assertEquals("actively allocated blocks in " + streamStatistics, - 0, streamStatistics.blocksActivelyAllocated()); + 0, streamStatistics.getBlocksActivelyAllocated()); } } @@ -541,12 +559,7 @@ public void test_800_DeleteHugeFiles() throws IOException { */ @Test public void test_900_dumpStats() { - StringBuilder sb = new StringBuilder(); - - getFileSystem().getStorageStatistics() - .forEach(kv -> sb.append(kv.toString()).append("\n")); - - LOG.info("Statistics\n{}", sb); + LOG.info("Statistics\n{}", ioStatisticsSourceToString(getFileSystem())); } protected void deleteHugeFile() throws IOException { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ILoadTestS3ABulkDeleteThrottling.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ILoadTestS3ABulkDeleteThrottling.java index aeb8ecaad553c..3fa288417c4d3 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ILoadTestS3ABulkDeleteThrottling.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ILoadTestS3ABulkDeleteThrottling.java @@ -45,8 +45,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.ContractTestUtils; -import org.apache.hadoop.fs.impl.FunctionsRaisingIOE; -import org.apache.hadoop.fs.impl.WrappedIOException; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.auth.delegation.Csvout; @@ -316,15 +314,6 @@ private List buildDeleteRequest( return request; } - - private R wrap(FunctionsRaisingIOE.CallableRaisingIOE callable) { - try { - return callable.apply(); - } catch (IOException e) { - throw new WrappedIOException(e); - } - } - /** * Outcome of one of the load operations. */ diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java index 9706597a09834..0751959bbdc74 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java @@ -28,6 +28,7 @@ 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.statistics.IOStatistics; import org.junit.Test; import org.assertj.core.api.Assertions; @@ -46,9 +47,16 @@ import com.amazonaws.services.s3.model.PutObjectRequest; import com.amazonaws.services.s3.model.PutObjectResult; +import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL; import static org.apache.hadoop.fs.s3a.Statistic.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; import static org.apache.hadoop.fs.contract.ContractTestUtils.*; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.lookupCounterStatistic; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToString; +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_CONTINUE_LIST_REQUEST; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_LIST_REQUEST; /** * Test the performance of listing files/directories. @@ -70,7 +78,7 @@ public void testListOperations() throws Throwable { int depth = scale; int files = scale; MetricDiff metadataRequests = new MetricDiff(fs, OBJECT_METADATA_REQUESTS); - MetricDiff listRequests = new MetricDiff(fs, OBJECT_LIST_REQUESTS); + MetricDiff listRequests = new MetricDiff(fs, Statistic.OBJECT_LIST_REQUEST); MetricDiff listContinueRequests = new MetricDiff(fs, OBJECT_CONTINUE_LIST_REQUESTS); MetricDiff listStatusCalls = new MetricDiff(fs, INVOCATION_LIST_FILES); @@ -166,8 +174,13 @@ public void testMultiPagesListingPerformanceAndCorrectness() final int numOfPutRequests = 1000; final int eachFileProcessingTime = 10; final int numOfPutThreads = 50; + Assertions.assertThat(numOfPutRequests % batchSize) + .describedAs("Files put %d must be a multiple of list batch size %d", + numOfPutRequests, batchSize) + .isEqualTo(0); final Configuration conf = getConfigurationWithConfiguredBatchSize(batchSize); + removeBaseAndBucketOverrides(conf, S3_METADATA_STORE_IMPL); final InputStream im = new InputStream() { @Override public int read() throws IOException { @@ -180,9 +193,10 @@ public int read() throws IOException { .newFixedThreadPool(numOfPutThreads); NanoTimer uploadTimer = new NanoTimer(); - try(S3AFileSystem fs = (S3AFileSystem) FileSystem.get(dir.toUri(), conf)) { - fs.create(dir); + S3AFileSystem fs = (S3AFileSystem) FileSystem.get(dir.toUri(), conf); + try { assume("Test is only for raw fs", !fs.hasMetadataStore()); + fs.create(dir); for (int i=0; i 0) { @@ -359,6 +397,9 @@ private void executeDecompression(long readahead, readahead); logTimePerIOP("line read", timer, lines); logStreamStatistics(); + assertNotNull("No IOStatistics through line reader", readerStatistics); + LOG.info("statistics from reader {}", + ioStatisticsToString(readerStatistics)); } private void logStreamStatistics() { @@ -391,8 +432,8 @@ protected void executeSeekReadSequence(long blockSize, readahead); logTimePerIOP("seek(pos + " + blockCount+"); read()", timer, blockCount); LOG.info("Effective bandwidth {} MB/S", - timer.bandwidthDescription(streamStatistics.bytesRead - - streamStatistics.bytesSkippedOnSeek)); + timer.bandwidthDescription(streamStatistics.getBytesRead() - + streamStatistics.getBytesSkippedOnSeek())); logStreamStatistics(); } @@ -419,7 +460,7 @@ protected void executeSeekReadSequence(long blockSize, public void testRandomIORandomPolicy() throws Throwable { executeRandomIO(S3AInputPolicy.Random, (long) RANDOM_IO_SEQUENCE.length); assertEquals("streams aborted in " + streamStatistics, - 0, streamStatistics.aborted); + 0, streamStatistics.getAborted()); } @Test @@ -427,11 +468,21 @@ public void testRandomIONormalPolicy() throws Throwable { long expectedOpenCount = RANDOM_IO_SEQUENCE.length; executeRandomIO(S3AInputPolicy.Normal, expectedOpenCount); assertEquals("streams aborted in " + streamStatistics, - 1, streamStatistics.aborted); + 1, streamStatistics.getAborted()); assertEquals("policy changes in " + streamStatistics, - 2, streamStatistics.policySetCount); + 2, streamStatistics.getPolicySetCount()); assertEquals("input policy in " + streamStatistics, - S3AInputPolicy.Random.ordinal(), streamStatistics.inputPolicy); + S3AInputPolicy.Random.ordinal(), + streamStatistics.getInputPolicy()); + IOStatistics ioStatistics = streamStatistics.getIOStatistics(); + verifyStatisticCounterValue( + ioStatistics, + StreamStatisticNames.STREAM_READ_ABORTED, + 1); + verifyStatisticCounterValue( + ioStatistics, + StreamStatisticNames.STREAM_READ_SEEK_POLICY_CHANGED, + 2); } /** @@ -466,9 +517,22 @@ private ContractTestUtils.NanoTimer executeRandomIO(S3AInputPolicy policy, assertOpenOperationCount(expectedOpenCount); logTimePerIOP("byte read", timer, totalBytesRead); LOG.info("Effective bandwidth {} MB/S", - timer.bandwidthDescription(streamStatistics.bytesRead - - streamStatistics.bytesSkippedOnSeek)); + timer.bandwidthDescription(streamStatistics.getBytesRead() - + streamStatistics.getBytesSkippedOnSeek())); logStreamStatistics(); + IOStatistics iostats = in.getIOStatistics(); + long maxHttpGet = lookupMaximumStatistic(iostats, + ACTION_HTTP_GET_REQUEST + SUFFIX_MAX); + assertThatStatisticMinimum(iostats, + ACTION_HTTP_GET_REQUEST + SUFFIX_MIN) + .isGreaterThan(0) + .isLessThan(maxHttpGet); + MeanStatistic getMeanStat = lookupMeanStatistic(iostats, + ACTION_HTTP_GET_REQUEST + SUFFIX_MEAN); + Assertions.assertThat(getMeanStat.getSamples()) + .describedAs("sample count of %s", getMeanStat) + .isEqualTo(expectedOpenCount); + return timer; } @@ -525,7 +589,7 @@ public void testRandomReadOverBuffer() throws Throwable { + " current position in stream " + currentPos + " in\n" + fs + "\n " + in, - 1, streamStatistics.openOperations); + 1, streamStatistics.getOpenOperations()); for (int i = currentPos; i < currentPos + read; i++) { assertEquals("Wrong value from byte " + i, sourceData[i], buffer[i]); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java index 42b73d3d88c09..33f69aff6c29c 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java @@ -26,6 +26,7 @@ import org.apache.hadoop.fs.s3a.S3AInstrumentation; import org.apache.hadoop.fs.s3a.S3ATestConstants; import org.apache.hadoop.fs.s3a.Statistic; +import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; import org.apache.hadoop.metrics2.lib.MutableGaugeLong; import org.slf4j.Logger; @@ -160,7 +161,7 @@ protected int getTestTimeoutMillis() { * @param in wrapper * @return the statistics for the inner stream */ - protected S3AInstrumentation.InputStreamStatistics getInputStreamStatistics( + protected S3AInputStreamStatistics getInputStreamStatistics( FSDataInputStream in) { return getS3AInputStream(in).getS3AStreamStatistics(); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3Select.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3Select.java index 64974db5a466c..6918941295699 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3Select.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3Select.java @@ -47,9 +47,9 @@ import org.apache.hadoop.fs.s3a.Constants; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3AInputStream; -import org.apache.hadoop.fs.s3a.S3AInstrumentation; import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.Statistic; +import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapred.JobConf; @@ -302,8 +302,8 @@ public void testSelectSeek() throws Throwable { SELECT_EVERYTHING)) { SelectInputStream sis = (SelectInputStream) seekStream.getWrappedStream(); - S3AInstrumentation.InputStreamStatistics streamStats - = sis.getS3AStreamStatistics(); + S3AInputStreamStatistics streamStats = + sis.getS3AStreamStatistics(); // lazy seek doesn't raise a problem here seekStream.seek(0); assertEquals("first byte read", fullData[0], seekStream.read()); @@ -344,7 +344,7 @@ public void testSelectSeek() throws Throwable { assertEquals("byte at seek position", fullData[(int)seekStream.getPos()], seekStream.read()); assertEquals("Seek bytes skipped in " + streamStats, - seekRange, streamStats.bytesSkippedOnSeek); + seekRange, streamStats.getBytesSkippedOnSeek()); // try an invalid readahead range intercept(IllegalArgumentException.class, @@ -395,8 +395,9 @@ public void testSelectOddLinesNoHeader() throws Throwable { "SELECT * FROM S3OBJECT s WHERE s._5 = `TRUE`"); // and do a quick check on the instrumentation long bytesRead = getFileSystem().getInstrumentation() - .getCounterValue(Statistic.STREAM_SEEK_BYTES_READ); - assertNotEquals("No bytes read count", 0, bytesRead); + .getCounterValue(Statistic.STREAM_READ_BYTES); + assertNotEquals("No bytes read count in filesystem instrumentation counter", + 0, bytesRead); } @Test @@ -588,13 +589,14 @@ public void testCloseWithAbort() throws Throwable { stream.setReadahead(1L); assertEquals("Readahead on " + sis, 1, sis.getReadahead()); stream.read(); - S3AInstrumentation.InputStreamStatistics stats - = sis.getS3AStreamStatistics(); + S3AInputStreamStatistics stats + = (S3AInputStreamStatistics) + sis.getS3AStreamStatistics(); assertEquals("Read count in " + sis, - 1, stats.bytesRead); + 1, stats.getBytesRead()); stream.close(); assertEquals("Abort count in " + sis, - 1, stats.aborted); + 1, stats.getAborted()); readOps.assertDiffEquals("Read operations are still considered active", 0); intercept(PathIOException.class, FSExceptionMessages.STREAM_IS_CLOSED, @@ -608,12 +610,14 @@ public void testCloseWithNoAbort() throws Throwable { "SELECT * FROM S3OBJECT s"); stream.setReadahead(0x1000L); SelectInputStream sis = (SelectInputStream) stream.getWrappedStream(); - S3AInstrumentation.InputStreamStatistics stats - = sis.getS3AStreamStatistics(); + S3AInputStreamStatistics stats + = (S3AInputStreamStatistics) + sis.getS3AStreamStatistics(); stream.close(); - assertEquals("Close count in " + sis, 1, stats.closed); - assertEquals("Abort count in " + sis, 0, stats.aborted); - assertTrue("No bytes read in close of " + sis, stats.bytesReadInClose > 0); + assertEquals("Close count in " + sis, 1, stats.getClosed()); + assertEquals("Abort count in " + sis, 0, stats.getAborted()); + assertTrue("No bytes read in close of " + sis, + stats.getBytesReadInClose() > 0); } @Test diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectLandsat.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectLandsat.java index 2099edd248b63..8b0578df11c01 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectLandsat.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectLandsat.java @@ -35,8 +35,8 @@ import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy.Source; +import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; import org.apache.hadoop.fs.s3a.S3AFileSystem; -import org.apache.hadoop.fs.s3a.S3AInstrumentation; import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.Statistic; import org.apache.hadoop.mapred.JobConf; @@ -381,7 +381,7 @@ public void testSelectSeekFullLandsat() throws Throwable { SELECT_EVERYTHING)) { SelectInputStream sis = (SelectInputStream) seekStream.getWrappedStream(); - S3AInstrumentation.InputStreamStatistics streamStats + S3AInputStreamStatistics streamStats = sis.getS3AStreamStatistics(); // lazy seek doesn't raise a problem here seekStream.seek(0); @@ -410,7 +410,7 @@ public void testSelectSeekFullLandsat() throws Throwable { assertEquals("byte at seek position", dataset[(int) seekStream.getPos()], seekStream.read()); assertEquals("Seek bytes skipped in " + streamStats, - seekRange, streamStats.bytesSkippedOnSeek); + seekRange, streamStats.getBytesSkippedOnSeek()); long offset; long increment = 64 * _1KB; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectMRJob.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectMRJob.java index 181d797767397..52a591384772a 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectMRJob.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectMRJob.java @@ -19,6 +19,7 @@ package org.apache.hadoop.fs.s3a.select; import java.io.IOException; +import java.io.UncheckedIOException; import java.util.concurrent.atomic.AtomicLong; import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; @@ -30,7 +31,6 @@ import org.apache.hadoop.examples.WordCount; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.impl.FutureIOSupport; -import org.apache.hadoop.fs.impl.WrappedIOException; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.S3AUtils; @@ -209,7 +209,7 @@ private String readStringFromFile(Path path) throws IOException { IOUtils.readFully(in, buffer, 0, bytesLen); return new String(buffer); } catch (IOException ex) { - throw new WrappedIOException(ex); + throw new UncheckedIOException(ex); } })); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestAggregateIOStatistics.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestAggregateIOStatistics.java new file mode 100644 index 0000000000000..c85651d8ab6c9 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestAggregateIOStatistics.java @@ -0,0 +1,100 @@ +/* + * 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.statistics; + +import java.io.File; +import java.time.LocalDateTime; +import java.time.format.DateTimeFormatter; +import java.time.format.DateTimeFormatterBuilder; + +import org.junit.Test; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.AbstractS3ATestBase; +import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; +import org.apache.hadoop.util.JsonSerialization; + +import static java.time.format.DateTimeFormatter.ISO_LOCAL_DATE; +import static java.time.temporal.ChronoField.HOUR_OF_DAY; +import static java.time.temporal.ChronoField.MINUTE_OF_HOUR; +import static java.time.temporal.ChronoField.NANO_OF_SECOND; +import static java.time.temporal.ChronoField.SECOND_OF_MINUTE; + +/** + * Saves, loads and logs the aggregate IOStatistics as collected in this + * process. + */ +public class ITestAggregateIOStatistics extends AbstractS3ATestBase { + + @Test + public void testSaveStatisticsLocal() throws Throwable { + IOStatisticsSnapshot iostats = FILESYSTEM_IOSTATS; + iostats.aggregate(getFileSystem().getIOStatistics()); + JsonSerialization serializer + = IOStatisticsSnapshot.serializer(); + File outputDir = createOutputDir(); + File file = new File(outputDir, outputFilename()); + serializer.save(file, iostats); + IOStatisticsSnapshot loaded = serializer.load(file); + String s = serializer.toString(loaded); + LOG.info("Deserialized statistics in {}\n{}", + file, s); + } + + @Test + public void testSaveStatisticsS3() throws Throwable { + IOStatisticsSnapshot iostats = FILESYSTEM_IOSTATS; + JsonSerialization serializer + = IOStatisticsSnapshot.serializer(); + Path path = methodPath(); + serializer.save(getFileSystem(), path, iostats, true); + IOStatisticsSnapshot loaded = serializer.load(getFileSystem(), path); + String s = serializer.toString(loaded); + LOG.info("Deserialized statistics in {}\n{}", + path, s); + } + + protected File createOutputDir() { + String target = System.getProperty("test.build.dir", "target"); + File buildDir = new File(target, + this.getClass().getSimpleName()).getAbsoluteFile(); + buildDir.mkdirs(); + return buildDir; + } + + protected String outputFilename() { + LocalDateTime now = LocalDateTime.now(); + DateTimeFormatter formatter = new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(ISO_LOCAL_DATE) + .appendLiteral('-') + .appendValue(HOUR_OF_DAY, 2) + .appendLiteral('.') + .appendValue(MINUTE_OF_HOUR, 2) + .optionalStart() + .appendLiteral('.') + .appendValue(SECOND_OF_MINUTE, 2) + .optionalStart() + .appendFraction(NANO_OF_SECOND, 0, 9, true) + .toFormatter(); + return String.format("iostats-%s.json", + now.format(formatter)); + + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestS3AContractStreamIOStatistics.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestS3AContractStreamIOStatistics.java new file mode 100644 index 0000000000000..8bed174fd31e1 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/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.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 inputStreamStatisticKeys() { + return Arrays.asList( + StreamStatisticNames.STREAM_READ_ABORTED, + StreamStatisticNames.STREAM_READ_BYTES_DISCARDED_ABORT, + StreamStatisticNames.STREAM_READ_CLOSED, + StreamStatisticNames.STREAM_READ_BYTES_DISCARDED_CLOSE, + 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_DISCARDED, + StreamStatisticNames.STREAM_READ_SEEK_BYTES_SKIPPED + ); + } + + /** + * Keys which the output stream must support. + * @return a list of keys + */ + @Override + public List outputStreamStatisticKeys() { + return Arrays.asList(STREAM_WRITE_BYTES, + STREAM_WRITE_BLOCK_UPLOADS, + STREAM_WRITE_EXCEPTIONS); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/MinimalListingOperationCallbacks.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/MinimalListingOperationCallbacks.java index bff9e91a49d0a..f683a0a008caf 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/MinimalListingOperationCallbacks.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/MinimalListingOperationCallbacks.java @@ -28,6 +28,7 @@ import org.apache.hadoop.fs.s3a.S3ListResult; import org.apache.hadoop.fs.s3a.impl.ListingOperationCallbacks; import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider; +import org.apache.hadoop.fs.statistics.DurationTrackerFactory; /** * Stub implementation of {@link ListingOperationCallbacks}. @@ -37,16 +38,16 @@ public class MinimalListingOperationCallbacks @Override public CompletableFuture listObjectsAsync( - S3ListRequest request) - throws IOException { + final S3ListRequest request, + final DurationTrackerFactory trackerFactory) throws IOException { return null; } @Override public CompletableFuture continueListObjectsAsync( - S3ListRequest request, - S3ListResult prevResult) - throws IOException { + final S3ListRequest request, + final S3ListResult prevResult, + final DurationTrackerFactory trackerFactory) throws IOException { return null; } @@ -82,4 +83,5 @@ public ITtlTimeProvider getUpdatedTtlTimeProvider() { public boolean allowAuthoritative(Path p) { return false; } + } diff --git a/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties index 6e20fbcda7efd..e8f3691caa6f0 100644 --- a/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties +++ b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties @@ -75,3 +75,11 @@ log4j.logger.org.apache.hadoop.fs.s3a.s3guard.Operations=DEBUG log4j.logger.org.apache.hadoop.mapreduce.lib.output=DEBUG log4j.logger.org.apache.hadoop.fs.s3a.S3AStorageStatistics=INFO + +# Set to debug if you need to debug S3A endpoint problems. +#log4j.logger.org.apache.hadoop.fs.s3a.DefaultS3ClientFactory=DEBUG + +# This causes all remote iterator stats +# to be logged when the RemoteIterators.foreach() method is +# invoked +#log4j.logger.org.apache.hadoop.util.functional.RemoteIterators=DEBUG From fd83274e5a25f4e976db104ccf77ce6ccf19a532 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 12 Jan 2021 17:25:14 +0000 Subject: [PATCH 4/4] HADOOP-17451. IOStatistics test failures in S3A code. (#2594) Caused by HADOOP-16830 and HADOOP-17271. Fixes tests which fail intermittently based on configs and in the case of the HugeFile tests, bulk runs with existing FS instances meant statistic probes sometimes ended up probing those of a previous FS. Contributed by Steve Loughran. Change-Id: I65ba3f44444e59d298df25ac5c8dc5a8781dfb7d --- .../StorageStatisticsFromIOStatistics.java | 33 +++++++++++++--- .../hadoop/fs/s3a/S3AInstrumentation.java | 38 ++++++------------- .../s3a/impl/ITestPartialRenamesDeletes.java | 10 ++++- .../s3a/performance/AbstractS3ACostTest.java | 5 ++- .../s3a/performance/ITestS3ADeleteCost.java | 14 ++++--- .../s3a/scale/AbstractSTestS3AHugeFiles.java | 29 +++++++------- .../hadoop/fs/s3a/scale/S3AScaleTestBase.java | 13 +++---- 7 files changed, 80 insertions(+), 62 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StorageStatisticsFromIOStatistics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StorageStatisticsFromIOStatistics.java index a55f04cae8c4c..f586cd8d9bdd4 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StorageStatisticsFromIOStatistics.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StorageStatisticsFromIOStatistics.java @@ -67,23 +67,46 @@ public Iterator iterator() { public Iterator getLongStatistics() { final Set> counters = counters() .entrySet(); - return counters.stream().map(e -> - new StorageStatistics.LongStatistic(e.getKey(), e.getValue())) - .collect(Collectors.toSet()).iterator(); + final Set statisticSet = counters.stream().map( + this::toLongStatistic) + .collect(Collectors.toSet()); + + // add the gauges + gauges().entrySet().forEach(entry -> + statisticSet.add(toLongStatistic(entry))); + return statisticSet.iterator(); + } + + /** + * Convert a counter/gauge entry to a long statistics. + * @param e entry + * @return statistic + */ + private LongStatistic toLongStatistic(final Map.Entry e) { + return new LongStatistic(e.getKey(), e.getValue()); } private Map counters() { return ioStatistics.counters(); } + private Map gauges() { + return ioStatistics.gauges(); + } + @Override public Long getLong(final String key) { - return counters().get(key); + Long l = counters().get(key); + if (l == null) { + l = gauges().get(key); + } + return l; } @Override public boolean isTracked(final String key) { - return counters().containsKey(key); + return counters().containsKey(key) + || gauges().containsKey(key); } @Override 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 982611a098401..c25e3b3c0efb5 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 @@ -64,10 +64,8 @@ import java.io.Closeable; import java.net.URI; import java.time.Duration; -import java.util.Arrays; import java.util.EnumSet; import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; @@ -182,20 +180,6 @@ public class S3AInstrumentation implements Closeable, MetricsSource, */ private final IOStatisticsStore instanceIOStatistics; - /** - * Gauges to create. - *

- * All statistics which are not gauges or quantiles - * are registered as counters. - */ - private static final Statistic[] GAUGES_TO_CREATE = { - OBJECT_PUT_REQUESTS_ACTIVE, - OBJECT_PUT_BYTES_PENDING, - STREAM_WRITE_BLOCK_UPLOADS_ACTIVE, - STREAM_WRITE_BLOCK_UPLOADS_PENDING, - STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING, - }; - /** * Construct the instrumentation for a filesystem. * @param name URI of filesystem. @@ -211,10 +195,6 @@ public S3AInstrumentation(URI name) { // create the builder IOStatisticsStoreBuilder storeBuilder = iostatisticsStore(); - // add the gauges - List gauges = Arrays.asList(GAUGES_TO_CREATE); - gauges.forEach(this::gauge); - // declare all counter statistics EnumSet.allOf(Statistic.class).stream() .filter(statistic -> @@ -223,6 +203,14 @@ public S3AInstrumentation(URI name) { counter(stat); storeBuilder.withCounters(stat.getSymbol()); }); + // declare all gauge statistics + EnumSet.allOf(Statistic.class).stream() + .filter(statistic -> + statistic.getType() == StatisticTypeEnum.TYPE_GAUGE) + .forEach(stat -> { + gauge(stat); + storeBuilder.withGauges(stat.getSymbol()); + }); // and durations EnumSet.allOf(Statistic.class).stream() @@ -1352,15 +1340,13 @@ private OutputStreamStatistics( this.filesystemStatistics = filesystemStatistics; IOStatisticsStore st = iostatisticsStore() .withCounters( - StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS, + STREAM_WRITE_BLOCK_UPLOADS.getSymbol(), STREAM_WRITE_BYTES.getSymbol(), STREAM_WRITE_EXCEPTIONS.getSymbol(), - StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING, - STREAM_WRITE_TOTAL_TIME.getSymbol(), + STREAM_WRITE_EXCEPTIONS_COMPLETING_UPLOADS.getSymbol(), STREAM_WRITE_QUEUE_DURATION.getSymbol(), STREAM_WRITE_TOTAL_DATA.getSymbol(), - STREAM_WRITE_EXCEPTIONS.getSymbol(), - STREAM_WRITE_EXCEPTIONS_COMPLETING_UPLOADS.getSymbol()) + STREAM_WRITE_TOTAL_TIME.getSymbol()) .withGauges( STREAM_WRITE_BLOCK_UPLOADS_PENDING.getSymbol(), STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING.getSymbol()) @@ -1470,7 +1456,7 @@ public void blockUploadFailed( @Override public void bytesTransferred(long byteCount) { bytesUploaded.addAndGet(byteCount); - incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING, -byteCount); + incAllGauges(STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING, -byteCount); } @Override diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestPartialRenamesDeletes.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestPartialRenamesDeletes.java index 1ea38af4f6c97..c920be13230dd 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestPartialRenamesDeletes.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestPartialRenamesDeletes.java @@ -77,6 +77,7 @@ import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.toPathList; import static org.apache.hadoop.fs.s3a.test.ExtraAssertions.assertFileCount; import static org.apache.hadoop.fs.s3a.test.ExtraAssertions.extractCause; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsSourceToString; import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; import static org.apache.hadoop.test.LambdaTestUtils.eval; @@ -685,7 +686,8 @@ public void testPartialDirDelete() throws Throwable { readOnlyFiles.size()); rejectionCount.assertDiffEquals("Wrong rejection count", readOnlyFiles.size()); - reset(rejectionCount, deleteVerbCount, deleteObjectCount); + reset(rejectionCount, deleteVerbCount, deleteObjectCount, + bulkDeleteVerbCount); } // all the files are still there? (avoid in scale test due to cost) if (!scaleTest) { @@ -694,9 +696,13 @@ public void testPartialDirDelete() throws Throwable { describe("Trying to delete upper-level directory"); ex = expectDeleteForbidden(basePath); + String iostats = ioStatisticsSourceToString(roleFS); + if (multiDelete) { // multi-delete status checks - deleteVerbCount.assertDiffEquals("Wrong delete count", 1); + deleteVerbCount.assertDiffEquals("Wrong delete request count", 0); + bulkDeleteVerbCount.assertDiffEquals( + "Wrong count of delete operations in " + iostats, 1); MultiObjectDeleteException mde = extractCause( MultiObjectDeleteException.class, ex); List undeletedKeyPaths = diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java index 4a2d695e6a227..c4f8db71937d0 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java @@ -475,7 +475,7 @@ protected OperationCostValidator.ExpectedProbe whenDeleting( /** * Execute a closure expecting a specific number of HEAD/LIST calls - * on raw S3 stores only. + * on raw S3 stores only. The operation is always evaluated. * @param cost expected cost * @param eval closure to evaluate * @param return type of closure @@ -484,7 +484,8 @@ protected OperationCostValidator.ExpectedProbe whenDeleting( protected T verifyRaw( OperationCost cost, Callable eval) throws Exception { - return verifyMetrics(eval, whenRaw(cost)); + return verifyMetrics(eval, + whenRaw(cost), OperationCostValidator.always()); } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java index f5d223932d4c4..2901767128fce 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java @@ -121,17 +121,21 @@ public void testDeleteSingleFileInDir() throws Throwable { with(DIRECTORIES_DELETED, 0), with(FILES_DELETED, 1), + // a single DELETE call is made to delete the object + with(OBJECT_DELETE_REQUEST, DELETE_OBJECT_REQUEST), + // keeping: create no parent dirs or delete parents withWhenKeeping(DIRECTORIES_CREATED, 0), - withWhenKeeping(OBJECT_DELETE_OBJECTS, DELETE_OBJECT_REQUEST), + withWhenKeeping(OBJECT_BULK_DELETE_REQUEST, 0), // deleting: create a parent and delete any of its parents withWhenDeleting(DIRECTORIES_CREATED, 1), - // two objects will be deleted - withWhenDeleting(OBJECT_DELETE_OBJECTS, - DELETE_OBJECT_REQUEST - + DELETE_MARKER_REQUEST) + // a bulk delete for all parents is issued. + // the number of objects in it depends on the depth of the tree; + // don't worry about that + withWhenDeleting(OBJECT_BULK_DELETE_REQUEST, DELETE_MARKER_REQUEST) ); + // there is an empty dir for a parent S3AFileStatus status = verifyRawInnerGetFileStatus(dir, true, StatusProbeEnum.ALL, GET_FILE_STATUS_ON_DIR); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java index 851b1b16eeb75..2b3043f39ac73 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java @@ -37,7 +37,6 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.StorageStatistics; import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3ATestUtils; @@ -49,10 +48,11 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.*; import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; +import static org.apache.hadoop.fs.s3a.Statistic.STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.lookupCounterStatistic; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticGaugeValue; import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.demandStringifyIOStatistics; import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsSourceToString; -import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics; -import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.snapshotIOStatistics; /** * Scale test which creates a huge file. @@ -169,7 +169,8 @@ public void test_010_CreateHugeFile() throws IOException { // there's lots of logging here, so that a tail -f on the output log // can give a view of what is happening. S3AFileSystem fs = getFileSystem(); - StorageStatistics storageStatistics = fs.getStorageStatistics(); + IOStatistics iostats = fs.getIOStatistics(); + String putRequests = Statistic.OBJECT_PUT_REQUESTS.getSymbol(); String putBytes = Statistic.OBJECT_PUT_BYTES.getSymbol(); Statistic putRequestsActive = Statistic.OBJECT_PUT_REQUESTS_ACTIVE; @@ -205,9 +206,9 @@ public void test_010_CreateHugeFile() throws IOException { percentage, writtenMB, filesizeMB, - storageStatistics.getLong(putBytes), + iostats.counters().get(putBytes), gaugeValue(putBytesPending), - storageStatistics.getLong(putRequests), + iostats.counters().get(putRequests), gaugeValue(putRequestsActive), elapsedTime, writtenMB / elapsedTime)); @@ -227,27 +228,27 @@ public void test_010_CreateHugeFile() throws IOException { logFSState(); bandwidth(timer, filesize); LOG.info("Statistics after stream closed: {}", streamStatistics); - IOStatistics iostats = snapshotIOStatistics( - retrieveIOStatistics(getFileSystem())); + LOG.info("IOStatistics after upload: {}", demandStringifyIOStatistics(iostats)); - long putRequestCount = storageStatistics.getLong(putRequests); - Long putByteCount = storageStatistics.getLong(putBytes); + long putRequestCount = lookupCounterStatistic(iostats, putRequests); + long putByteCount = lookupCounterStatistic(iostats, putBytes); Assertions.assertThat(putRequestCount) .describedAs("Put request count from filesystem stats %s", iostats) .isGreaterThan(0); Assertions.assertThat(putByteCount) - .describedAs("putByteCount count from filesystem stats %s", - iostats) + .describedAs("%s count from filesystem stats %s", + putBytes, iostats) .isGreaterThan(0); LOG.info("PUT {} bytes in {} operations; {} MB/operation", putByteCount, putRequestCount, putByteCount / (putRequestCount * _1MB)); LOG.info("Time per PUT {} nS", toHuman(timer.nanosPerOperation(putRequestCount))); - assertEquals("active put requests in \n" + fs, - 0, gaugeValue(putRequestsActive)); + verifyStatisticGaugeValue(iostats, putRequestsActive.getSymbol(), 0); + verifyStatisticGaugeValue(iostats, + STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING.getSymbol(), 0); progress.verifyNoFailures( "Put file " + fileToCreate + " of size " + filesize); if (streamStatistics != null) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java index 33f69aff6c29c..d95b46b10d786 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java @@ -23,11 +23,9 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.AbstractS3ATestBase; import org.apache.hadoop.fs.s3a.S3AInputStream; -import org.apache.hadoop.fs.s3a.S3AInstrumentation; import org.apache.hadoop.fs.s3a.S3ATestConstants; import org.apache.hadoop.fs.s3a.Statistic; import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; -import org.apache.hadoop.metrics2.lib.MutableGaugeLong; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -35,6 +33,7 @@ import java.io.InputStream; import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.lookupGaugeStatistic; /** * Base class for scale tests; here is where the common scale configuration @@ -184,17 +183,15 @@ protected S3AInputStream getS3AInputStream( } /** - * Get the gauge value of a statistic. Raises an assertion if + * Get the gauge value of a statistic from the + * IOStatistics of the filesystem. Raises an assertion if * there is no such gauge. * @param statistic statistic to look up * @return the value. */ public long gaugeValue(Statistic statistic) { - S3AInstrumentation instrumentation = getFileSystem().getInstrumentation(); - MutableGaugeLong gauge = instrumentation.lookupGauge(statistic.getSymbol()); - assertNotNull("No gauge " + statistic - + " in " + instrumentation.dump("", " = ", "\n", true), gauge); - return gauge.value(); + return lookupGaugeStatistic(getFileSystem().getIOStatistics(), + statistic.getSymbol()); } /**