From 5bcb1811994e1cad1a8e9d566754ca9a5a60edde Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 21 Sep 2020 19:24:01 +0100 Subject: [PATCH 01/16] HADOOP-16830. Add public IOStatistics API. Change-Id: Ic0cdd0c3707066fff8b21f9cca6e7265a547267c --- .../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/FutureIOSupport.java | 20 +- .../hadoop/fs/impl/WrappedIOException.java | 7 +- .../hadoop/fs/statistics/DurationTracker.java | 33 + .../fs/statistics/DurationTrackerFactory.java | 58 ++ .../hadoop/fs/statistics/IOStatistics.java | 78 +++ .../fs/statistics/IOStatisticsAggregator.java | 48 ++ .../fs/statistics/IOStatisticsLogging.java | 214 +++++++ .../fs/statistics/IOStatisticsSnapshot.java | 256 ++++++++ .../fs/statistics/IOStatisticsSource.java | 46 ++ .../fs/statistics/IOStatisticsSupport.java | 107 ++++ .../hadoop/fs/statistics/MeanStatistic.java | 285 +++++++++ .../fs/statistics/StoreStatisticNames.java | 278 ++++++++ .../fs/statistics/StreamStatisticNames.java | 188 ++++++ .../impl/AbstractIOStatisticsImpl.java | 30 + .../impl/BufferedIOStatisticsInputStream.java | 74 +++ .../BufferedIOStatisticsOutputStream.java | 156 +++++ .../statistics/impl/DynamicIOStatistics.java | 132 ++++ .../impl/DynamicIOStatisticsBuilder.java | 248 ++++++++ .../fs/statistics/impl/EmptyIOStatistics.java | 74 +++ .../impl/EvaluatingStatisticsMap.java | 202 ++++++ .../statistics/impl/IOStatisticsBinding.java | 338 ++++++++++ .../fs/statistics/impl/IOStatisticsStore.java | 249 ++++++++ .../impl/IOStatisticsStoreBuilder.java | 75 +++ .../impl/IOStatisticsStoreBuilderImpl.java | 93 +++ .../impl/IOStatisticsStoreImpl.java | 424 +++++++++++++ .../impl/SourceWrappedStatistics.java | 44 ++ .../impl/StatisticDurationTracker.java | 83 +++ .../statistics/impl/StubDurationTracker.java | 39 ++ .../impl/StubDurationTrackerFactory.java | 43 ++ .../statistics/impl/WrappedIOStatistics.java | 108 ++++ .../fs/statistics/impl/package-info.java | 31 + .../hadoop/fs/statistics/package-info.java | 133 ++++ .../io/compress/CompressionInputStream.java | 18 +- .../io/compress/CompressionOutputStream.java | 14 +- .../org/apache/hadoop/util/LineReader.java | 16 +- .../apache/hadoop/util/OperationDuration.java | 52 +- .../util/SemaphoredDelegatingExecutor.java | 41 +- .../util/functional/BiFunctionRaisingIOE.java | 40 ++ .../util/functional/CallableRaisingIOE.java | 36 ++ .../util/functional/ConsumerRaisingIOE.java | 51 ++ .../util/functional/FunctionRaisingIOE.java | 38 ++ .../util/functional/RemoteIterators.java | 594 ++++++++++++++++++ .../functional/RuntimeIOException.java} | 53 +- .../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 | 282 +++++++++ ...TestLocalFSContractStreamIOStatistics.java | 80 +++ .../fs/statistics/IOStatisticAssertions.java | 466 ++++++++++++++ .../statistics/TestDynamicIOStatistics.java | 311 +++++++++ .../fs/statistics/TestEmptyIOStatistics.java | 110 ++++ .../statistics/TestIOStatisticsSnapshot.java | 135 ++++ .../fs/statistics/TestIOStatisticsStore.java | 189 ++++++ .../fs/statistics/TestMeanStatistic.java | 219 +++++++ .../util/functional/TestRemoteIterators.java | 416 ++++++++++++ 67 files changed, 8063 insertions(+), 100 deletions(-) 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/BufferedIOStatisticsInputStream.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/BufferedIOStatisticsOutputStream.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/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/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/RemoteIterators.java rename hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/{fs/impl/FunctionsRaisingIOE.java => util/functional/RuntimeIOException.java} (51%) 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/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 9e601e26cf944..fc3ba71ab45f3 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 8d11043937612..829f205e22eb2 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 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 cc9c284c9fa55..68189b3112a0c 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 672ab15f16c3b..373120ce3078f 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSInputStream.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSInputStream.java @@ -24,6 +24,9 @@ import com.google.common.base.Preconditions; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.statistics.IOStatisticsLogging; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -134,4 +137,23 @@ public void readFully(long position, byte[] buffer) throws IOException { readFully(position, buffer, 0, buffer.length); } + + /** + * toString method returns the superclass toString, but if the subclass + * implements {@link IOStatisticsSource} then those statistics are + * extracted and included in the output. + * That is: statistics of subclasses are automatically reported. + * @return a string value. + */ + @Override + public String toString() { + final StringBuilder sb = new StringBuilder(super.toString()); + sb.append('{'); + if (this instanceof IOStatisticsSource) { + sb.append(IOStatisticsLogging.ioStatisticsSourceToString( + (IOStatisticsSource) this)); + } + sb.append('}'); + return sb.toString(); + } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/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 72eeb99a4ea5d..e25e1483def9a 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.impl.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, 1); } return value; } catch (IOException e) { // unexpected exception + ioStatistics.incrementCounter(STREAM_READ_EXCEPTIONS, 1); 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, 1); 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, 1); throw new FSError(e); } } @Override public long skip(long n) throws IOException { + ioStatistics.incrementCounter(STREAM_READ_SKIP_OPERATIONS, 1); 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, 1); 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, 1); } catch (IOException e) { // unexpected exception + ioStatistics.incrementCounter(STREAM_WRITE_EXCEPTIONS, 1); 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/FutureIOSupport.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureIOSupport.java index f13d701803d7e..8d1a8b3342851 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,6 +32,8 @@ 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.RuntimeIOException; /** * Support for future IO and the FS Builder subclasses. @@ -130,10 +132,13 @@ public static T raiseInnerCause(final CompletionException 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. + *
    + *
  1. If it is an IOE: Return.
  2. + *
  3. If it is a {@link RuntimeIOException}: return the cause
  4. + *
  5. Completion/Execution Exceptions: extract and repeat
  6. + *
  7. If it is an RTE: 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. @@ -145,8 +150,9 @@ 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 RuntimeIOException) { + // this is always an IOException + return ((RuntimeIOException) cause).getCause(); } else if (cause instanceof CompletionException) { return unwrapInnerException(cause); } else if (cause instanceof ExecutionException) { @@ -236,7 +242,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 1de1ecb785368..0ad4cda13da3b 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 @@ -25,18 +25,21 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.util.functional.RuntimeIOException; /** * A wrapper for an IOException which * {@link FutureIOSupport#raiseInnerCause(ExecutionException)} knows to * always extract the exception. - * + *

* The constructor signature guarantees the cause will be an IOException, * and as it checks for a null-argument, non-null. + * @deprecated: use the public {@link RuntimeIOException}. */ +@Deprecated @InterfaceAudience.Private @InterfaceStability.Unstable -public class WrappedIOException extends RuntimeException { +public class WrappedIOException extends RuntimeIOException { private static final long serialVersionUID = 2510210974235779294L; 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..9d1f6f42df199 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/DurationTracker.java @@ -0,0 +1,33 @@ +/* + * 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 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. + */ +public interface DurationTracker extends AutoCloseable { + + /** + * Finish tracking: update the statistics with the timings. + */ + void close(); +} 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..c9c3fcc9c2c0e --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/DurationTrackerFactory.java @@ -0,0 +1,58 @@ +/* + * 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, int 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..3a4e9859d92f1 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsLogging.java @@ -0,0 +1,214 @@ +/* + * 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 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 ""; + } + } + + /** + * Given a map, add its entryset to the string. + * @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 mapToString(StringBuilder sb, + final String type, + final Map map) { + int count = 0; + sb.append(type); + sb.append("=("); + for (Map.Entry entry : map.entrySet()) { + if (count > 0) { + sb.append(' '); + } + count++; + sb.append(IOStatisticsBinding.entryToString( + entry.getKey(), entry.getValue())); + } + sb.append("); "); + } + + /** + * 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 + * @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); + } + } + } + + /** + * 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..c6dd490767fd2 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSnapshot.java @@ -0,0 +1,256 @@ +/* + * 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.Map; +import java.util.TreeMap; +import java.util.concurrent.ConcurrentHashMap; + +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 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. + *

+ * 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; + + /** + * 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 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 Map counters() { + return counters; + } + + @Override + public Map gauges() { + return gauges; + } + + @Override + public Map minimums() { + return minimums; + } + + @Override + public Map maximums() { + return maximums; + } + + @Override + public 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 synchronized 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()); + } + +} 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..8b0287ad20952 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSource.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.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..3eaeccd547947 --- /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..69683ae0500e9 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/MeanStatistic.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 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 sum == that.sum && + samples == that.samples; + } + + @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("(sum=%d, samples=%d, mean=%.4f)", + sum, samples, 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..6c7572944accc --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java @@ -0,0 +1,278 @@ +/* + * 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 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 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 OP_HTTP_DELETE_REQUEST + = "op_http_delete_request"; + + /** + * 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"; + + + /** + * The name of the statistic collected for executor aquisition if + * a duration tracker factory is passed in to the constructor. + * {@value}. + */ + public static 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"; + + + 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..37da830512497 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StreamStatisticNames.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.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 { + + /** {@value}. */ + public static final String STREAM_READ_ABORTED = "stream_aborted"; + + /** {@value}. */ + public static final String STREAM_READ_BYTES_DISCARDED_ABORT + = "stream_read_bytes_discarded_in_abort"; + + /** {@value}. */ + public static final String STREAM_READ_BYTES_DISCARDED_SEEK + = "stream_read_bytes_discarded_in_seek"; + + /** {@value}. */ + public static final String STREAM_READ_CLOSED = "stream_read_closed"; + + /** {@value}. */ + public static final String STREAM_READ_CLOSE_BYTES_READ + = "stream_read_bytes_read_in_close"; + + /** {@value}. */ + public static final String STREAM_READ_CLOSE_OPERATIONS + = "stream_read_close_operations"; + + /** {@value}. */ + public static final String STREAM_READ_OPENED = "stream_read_opened"; + + /** {@value}. */ + public static final String STREAM_READ_BYTES + = "stream_read_bytes"; + + /** {@value}. */ + public static final String STREAM_READ_EXCEPTIONS = + "stream_read_exceptions"; + + /** {@value}. */ + public static final String STREAM_READ_FULLY_OPERATIONS + = "stream_read_fully_operations"; + + /** {@value}. */ + public static final String STREAM_READ_OPERATIONS = + "stream_read_operations"; + + /** {@value}. */ + public static final String STREAM_READ_OPERATIONS_INCOMPLETE + = "stream_read_operations_incomplete"; + + /** {@value}. */ + public static final String STREAM_READ_VERSION_MISMATCHES + = "stream_read_version_mismatches"; + + /** {@value}. */ + public static final String STREAM_READ_SEEK_BYTES_BACKWARDS + = "stream_read_bytes_backwards_on_seek"; + + /** {@value}. */ + public static final String STREAM_READ_SEEK_FORWARD_OPERATIONS + = "stream_read_seek_forward_operations"; + + /** {@value}. */ + public static final String STREAM_READ_SEEK_POLICY_CHANGED = + "stream_read_seek_policy_changed"; + + /** {@value}. */ + public static final String STREAM_READ_SEEK_OPERATIONS = + "stream_read_seek_operations"; + + /** {@value}. */ + public static final String STREAM_READ_SEEK_BACKWARD_OPERATIONS = + "stream_read_seek_backward_operations"; + + /** {@value}. */ + public static final String STREAM_READ_SEEK_BYTES_READ = + "stream_read_seek_bytes_read"; + + /** {@value}. */ + public static final String STREAM_READ_SEEK_BYTES_SKIPPED + = "stream_read_bytes_skipped_on_seek"; + + /** {@value}. */ + public static final String STREAM_READ_SKIP_OPERATIONS = + "stream_read_skip_operations"; + + /** {@value}. */ + public static final String STREAM_READ_SKIP_BYTES = + "stream_read_skip_bytes"; + + /** {@value}. */ + public static final String STREAM_READ_TOTAL_BYTES + = "stream_read_total_bytes"; + + /** {@value}. */ + public static final String STREAM_WRITE_EXCEPTIONS = + "stream_write_exceptions"; + + /** Failures when finalizing a multipart upload: {@value}. */ + public static final String STREAM_WRITE_EXCEPTIONS_COMPLETING_UPLOADS = + "stream_write_exceptions_completing_upload"; + + /** {@value}. */ + public static final String STREAM_WRITE_BLOCK_UPLOADS + = "stream_write_block_uploads"; + + /** {@value}. */ + public static final String STREAM_WRITE_BLOCK_UPLOADS_ABORTED + = "stream_write_block_uploads_aborted"; + + /** {@value}. */ + public static final String STREAM_WRITE_BLOCK_UPLOADS_ACTIVE + = "stream_write_block_uploads_active"; + + /** {@value}. */ + public static final String STREAM_WRITE_BLOCK_UPLOADS_COMMITTED + = "stream_write_block_uploads_committed"; + + /** {@value}. */ + public static final String STREAM_WRITE_BLOCK_UPLOADS_PENDING + = "stream_write_block_uploads_pending"; + + /** {@value}. */ + public static final String STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING = + "stream_write_block_uploads_data_pending"; + + /** + * "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"; + + 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/BufferedIOStatisticsInputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/BufferedIOStatisticsInputStream.java new file mode 100644 index 0000000000000..718487a7d7242 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/BufferedIOStatisticsInputStream.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.io.BufferedInputStream; +import java.io.InputStream; + +import org.apache.hadoop.fs.StreamCapabilities; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; + +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics; + +/** + * An extension of {@code BufferedInputStream} which implements + * {@link IOStatisticsSource} and forwards requests for the + * {@link IOStatistics} to the wrapped stream. + *

+ * This should be used when any input stream needs buffering while + * allowing the inner stream to be a source of statistics. + *

+ * It also implements {@link StreamCapabilities} and forwards the probe + * to the inner stream, if possible. + */ +public class BufferedIOStatisticsInputStream + extends BufferedInputStream + implements IOStatisticsSource, StreamCapabilities { + + public BufferedIOStatisticsInputStream(final InputStream in) { + super(in); + } + + public BufferedIOStatisticsInputStream(final InputStream in, final int size) { + super(in, size); + } + + @Override + public IOStatistics getIOStatistics() { + return retrieveIOStatistics(in); + } + + /** + * 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/impl/BufferedIOStatisticsOutputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/BufferedIOStatisticsOutputStream.java new file mode 100644 index 0000000000000..86744c5f7e4f5 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/BufferedIOStatisticsOutputStream.java @@ -0,0 +1,156 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics.impl; + +import java.io.BufferedOutputStream; +import java.io.IOException; +import java.io.OutputStream; + +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 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/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..9a8c0e8557782 --- /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..0150048229e44 --- /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 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..07e0bd6d770b3 --- /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..33b62d387e97a --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java @@ -0,0 +1,338 @@ +/* + * 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.Iterator; +import java.util.Map; +import java.util.TreeMap; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.BiFunction; +import java.util.function.Function; + +import com.google.common.annotations.VisibleForTesting; + +import org.apache.hadoop.fs.StorageStatistics; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.fs.statistics.MeanStatistic; + +import static org.apache.hadoop.fs.statistics.IOStatistics.MIN_UNSET_VALUE; + +/** + * 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); + } + +} 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..b4dbcfe22b18a --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStore.java @@ -0,0 +1,249 @@ +/* + * 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. + * @param key statistics key + * @param value value to increment + * @return old 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 old value or 0 + */ + 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 old value or 0 + */ + 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 old value or 0 + */ + 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 maximum 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 maximum 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 maximum 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..5068b24043a54 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStoreBuilderImpl.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.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +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 implementation. + */ +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); + withMinimums(p + SUFFIX_MIN); + withMaximums(p + SUFFIX_MAX); + withMeanStatistics(p + 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..b0c5ddd35d033 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStoreImpl.java @@ -0,0 +1,424 @@ +/* + * 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.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}. + */ +final class IOStatisticsStoreImpl extends WrappedIOStatistics + implements IOStatisticsStore { + + /** + * Log changes at debug. + * Noisy, but occasionally useful. + */ + private static final Logger LOG = + LoggerFactory.getLogger(IOStatisticsStoreImpl.class); + + private final Map counterMap = new ConcurrentHashMap<>(); + + private final Map gaugeMap = new ConcurrentHashMap<>(); + + private final Map minimumMap = new ConcurrentHashMap<>(); + + private final Map maximumMap = new ConcurrentHashMap<>(); + + 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; -ve for a decrement + * @return final value or 0 + */ + private long incAtomicLong(final AtomicLong aLong, + final long increment) { + if (aLong != null) { + return aLong.addAndGet(increment); + } 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) { + long l = incAtomicLong(counterMap.get(key), 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 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. + */ + @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()); + } + + @Override + public DurationTracker trackDuration(final String key, final int count) { + return new StatisticDurationTracker(this, key, count); + } +} 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..ba2c9015d6311 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StatisticDurationTracker.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics.impl; + +import org.apache.hadoop.fs.statistics.DurationTracker; +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; + + /** + * 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 xero, 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 int count) { + this.iostats = iostats; + this.key = key; + if (count > 0) { + iostats.incrementCounter(key, count); + } + } + + /** + * Set the finished time and then update the statistics. + */ + @Override + public void close() { + finished(); + iostats.addTimedOperation(key, asDuration()); + } +} 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..87cc77877f4bd --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StubDurationTracker.java @@ -0,0 +1,39 @@ +/* + * 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; + +/** + * 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 close() { + + } +} 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..3bb13d2e0473a --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StubDurationTrackerFactory.java @@ -0,0 +1,43 @@ +/* + * 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 int 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..6617ba40edac4 --- /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 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..fafa775163f79 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/package-info.java @@ -0,0 +1,133 @@ +/* + * 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 are includes to assist use + *

    + *
  • + * {@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/SemaphoredDelegatingExecutor.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SemaphoredDelegatingExecutor.java index 4ec77e75ba520..687fa2cf8d6f1 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SemaphoredDelegatingExecutor.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SemaphoredDelegatingExecutor.java @@ -24,6 +24,8 @@ import com.google.common.util.concurrent.ListeningExecutorService; import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.fs.statistics.DurationTracker; +import org.apache.hadoop.fs.statistics.DurationTrackerFactory; import java.util.Collection; import java.util.List; @@ -34,6 +36,10 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.stubDurationTrackerFactory; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.ACTION_EXECUTOR_ACQUIRED; + /** * This ExecutorService blocks the submission of new tasks when its queue is * already full by using a semaphore. Task submissions require permits, task @@ -54,20 +60,39 @@ public class SemaphoredDelegatingExecutor extends private final Semaphore queueingPermits; private final ListeningExecutorService executorDelegatee; private final int permitCount; + private final DurationTrackerFactory trackerFactory; /** * Instantiate. * @param executorDelegatee Executor to delegate to * @param permitCount number of permits into the queue permitted * @param fair should the semaphore be "fair" + * @param trackerFactory duration tracker factory. */ public SemaphoredDelegatingExecutor( ListeningExecutorService executorDelegatee, int permitCount, - boolean fair) { + boolean fair, + DurationTrackerFactory trackerFactory) { this.permitCount = permitCount; queueingPermits = new Semaphore(permitCount, fair); - this.executorDelegatee = executorDelegatee; + this.executorDelegatee = requireNonNull(executorDelegatee); + this.trackerFactory = trackerFactory != null + ? trackerFactory + : stubDurationTrackerFactory(); + } + + /** + * Instantiate without collecting executor aquisition duration information. + * @param executorDelegatee Executor to delegate to + * @param permitCount number of permits into the queue permitted + * @param fair should the semaphore be "fair" + */ + public SemaphoredDelegatingExecutor( + ListeningExecutorService executorDelegatee, + int permitCount, + boolean fair) { + this(executorDelegatee, permitCount, fair, null); } @Override @@ -103,7 +128,8 @@ public T invokeAny(Collection> tasks, long timeout, @Override public ListenableFuture submit(Callable task) { - try { + try (DurationTracker ignored = + trackerFactory.trackDuration(ACTION_EXECUTOR_ACQUIRED)) { queueingPermits.acquire(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); @@ -114,7 +140,8 @@ public ListenableFuture submit(Callable task) { @Override public ListenableFuture submit(Runnable task, T result) { - try { + try (DurationTracker ignored = + trackerFactory.trackDuration(ACTION_EXECUTOR_ACQUIRED)) { queueingPermits.acquire(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); @@ -125,7 +152,8 @@ public ListenableFuture submit(Runnable task, T result) { @Override public ListenableFuture submit(Runnable task) { - try { + try (DurationTracker ignored = + trackerFactory.trackDuration(ACTION_EXECUTOR_ACQUIRED)) { queueingPermits.acquire(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); @@ -136,7 +164,8 @@ public ListenableFuture submit(Runnable task) { @Override public void execute(Runnable command) { - try { + try (DurationTracker ignored = + trackerFactory.trackDuration(ACTION_EXECUTOR_ACQUIRED)) { queueingPermits.acquire(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); 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/RemoteIterators.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RemoteIterators.java new file mode 100644 index 0000000000000..47a04f101256c --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RemoteIterators.java @@ -0,0 +1,594 @@ +/* + * 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 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 FromIterator<>(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 FromIterator<>(iterable.iterator()); + } + + /** + * Create a remote iterator from an array. + * @param type + * @return a remote iterator + */ + public static RemoteIterator remoteIteratorFromArray(T[] array) { + return new FromIterator<>(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 an iterator from an iterator and a filter. + *

+ * 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; + + while (source.hasNext()) { + count++; + consumer.accept(source.next()); + } + + // maybe log the results + logIOStatisticsAtDebug(LOG, "RemoteIterator Statistics: {}", 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 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 FromIterator + implements RemoteIterator, IOStatisticsSource, Closeable { + + /** + * inner iterator.. + */ + private final Iterator source; + + private final Closeable sourceToClose; + + + /** + * Construct from an interator. + * @param source source iterator. + */ + private FromIterator(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(); + } + + @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 getSource().hasNext(); + } + + @Override + public T next() throws IOException { + return mapper.apply(getSource().next()); + } + + @Override + public String toString() { + return "FunctionRemoteIterator{" + getSource() + '}'; + } + } + + /** + * 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 && getSource().hasNext()) { + 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 CloseRemoteIterator( + final RemoteIterator source, + final Closeable toClose) { + super(source); + this.toClose = new MaybeClose(Objects.requireNonNull(toClose)); + } + + @Override + public boolean hasNext() throws IOException { + return getSource().hasNext(); + } + + @Override + public S next() throws IOException { + return getSource().next(); + } + + @Override + public void close() throws IOException { + try { + super.close(); + } finally { + toClose.close(); + } + } + } + + /** + * Class to help with Closeable logic, where sources may/may not + * be closeable, only one invocation is allowed. + */ + 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/fs/impl/FunctionsRaisingIOE.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RuntimeIOException.java similarity index 51% rename from hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FunctionsRaisingIOE.java rename to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RuntimeIOException.java index 7bbb34622647d..ae93d0c48fd74 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/util/functional/RuntimeIOException.java @@ -16,54 +16,41 @@ * limitations under the License. */ -package org.apache.hadoop.fs.impl; +package org.apache.hadoop.util.functional; import java.io.IOException; +import java.util.Objects; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; /** - * Evolving support for functional programming/lambda-expressions. + * A RuntimeException which always contains an IOException. + *

+ * The constructor signature guarantees the cause will be an IOException, + * and as it checks for a null-argument, non-null. */ -@InterfaceAudience.Private -@InterfaceStability.Unstable -public final class FunctionsRaisingIOE { +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class RuntimeIOException extends RuntimeException { - private FunctionsRaisingIOE() { - } + private static final long serialVersionUID = 7543456147436005830L; /** - * Function of arity 1 which may raise an IOException. - * @param type of arg1 - * @param type of return value. + * Construct from a non-null IOException. + * @param cause inner cause + * @throws NullPointerException if the cause is null. */ - @FunctionalInterface - public interface FunctionRaisingIOE { - - R apply(T t) throws IOException; + public RuntimeIOException(final IOException cause) { + super(Objects.requireNonNull(cause)); } /** - * Function of arity 2 which may raise an IOException. - * @param type of arg1 - * @param type of arg2 - * @param type of return value. + * Return the cause, cast to an IOException. + * @return cause of this exception. */ - @FunctionalInterface - public interface BiFunctionRaisingIOE { - - R apply(T t, U u) throws IOException; + @Override + public synchronized IOException getCause() { + return (IOException) super.getCause(); } - - /** - * This is a callable which only raises an IOException. - * @param return type - */ - @FunctionalInterface - public interface CallableRaisingIOE { - - R apply() throws IOException; - } - } 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..f705424e78f0a --- /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 need to +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 8a520fae8a2c3..f08fab3972964 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..24d0c2840caa0 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractStreamIOStatisticsTest.java @@ -0,0 +1,282 @@ +/* + * 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.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.IOStatisticsLogging; + +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.verifyCounterStatisticValue; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.demandStringifyIOStatisticsSource; +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); + + @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 + verifyCounterStatisticValue(statistics, STREAM_WRITE_BYTES, 0); + out.write('0'); + verifyCounterStatisticValue(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); + verifyCounterStatisticValue(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); + verifyCounterStatisticValue(statistics, STREAM_WRITE_BYTES, + writesInBlocks ? 0 : len); + out.write(bytes); + out.flush(); + verifyCounterStatisticValue(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); + verifyCounterStatisticValue(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); + verifyCounterStatisticValue(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); + verifyCounterStatisticValue(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 = verifyCounterStatisticValue(statistics, STREAM_READ_BYTES, + current + sublen); + Assertions.assertThat(in.read(pos, buf128, 0, bufferLen)) + .describedAs("Read(buffer) overlapping EOF") + .isEqualTo(sublen); + verifyCounterStatisticValue(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)); + } + verifyCounterStatisticValue(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..caf01b3db52de --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/IOStatisticAssertions.java @@ -0,0 +1,466 @@ +/* + * 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.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import java.util.Map; + +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, stats.counters()); + } + + /** + * 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, 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, 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, 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, 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 verifyCounterStatisticValue( + final IOStatistics stats, + final String key, + final long value) { + return verifyStatisticValue(COUNTER, key, 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 verifyGaugeStatisticValue( + final IOStatistics stats, + final String key, + final long value) { + return verifyStatisticValue(GAUGE, key, 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 verifyMaximumStatisticValue( + final IOStatistics stats, + final String key, + final long value) { + return verifyStatisticValue(MAXIMUM, key, 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 verifyMinimumStatisticValue( + final IOStatistics stats, + final String key, + final long value) { + return verifyStatisticValue(MINIMUM, key, 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 verifyMeanStatisticValue( + final IOStatistics stats, + final String key, + final MeanStatistic value) { + return verifyStatisticValue(MEAN, key, 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 assertThatLongStatistic( + 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 assertThatCounterStatistic( + final IOStatistics stats, + final String key) { + return assertThatLongStatistic(COUNTER, key, 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 assertThatGaugeStatistic( + final IOStatistics stats, + final String key) { + return assertThatLongStatistic(GAUGE, key, 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 assertThatMinimumStatistic( + final IOStatistics stats, + final String key) { + return assertThatLongStatistic(MINIMUM, key, 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 assertThatMaximumStatistic( + final IOStatistics stats, + final String key) { + return assertThatLongStatistic(MAXIMUM, key, 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 assertThatMeanStatistic( + final IOStatistics stats, + final String key) { + return assertThatStatistic(MEAN, key, 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 assertThatMeanStatisticMatches( + final IOStatistics stats, + final String key, + final long samples, + final long sum) { + return assertThatMeanStatistic(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 assertCounterStatisticIsTracked( + final IOStatistics stats, + final String key) { + assertTracked(stats, COUNTER, key, stats.counters()); + } + + /** + * Assert that a given counter statistic is untracked. + * @param stats statistics source + * @param key statistic key + */ + public static void assertCounterStatisticIsUntracked( + final IOStatistics stats, + final String key) { + assertUntracked(stats, COUNTER, key, 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 javaRoundTrip(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 ObjectInputStream(bais)) { + deser = (IOStatistics) ois.readObject(); + } + return deser; + } + +} 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..6760ac5378050 --- /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.assertCounterStatisticIsTracked; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertCounterStatisticIsUntracked; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyCounterStatisticValue; +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 { + verifyCounterStatisticValue(statistics, EVAL, 0); + evalLong = 10; + verifyCounterStatisticValue(statistics, EVAL, 10); + } + + /** + * Atomic Long statistic. + */ + @Test + public void testAlong() throws Throwable { + verifyCounterStatisticValue(statistics, ALONG, 0); + aLong.addAndGet(1); + verifyCounterStatisticValue(statistics, ALONG, 1); + } + + /** + * Atomic Int statistic. + */ + @Test + public void testAint() throws Throwable { + verifyCounterStatisticValue(statistics, AINT, 0); + aInt.addAndGet(1); + verifyCounterStatisticValue(statistics, AINT, 1); + } + + /** + * Metrics2 counter. + */ + @Test + public void testCounter() throws Throwable { + verifyCounterStatisticValue(statistics, COUNT, 0); + counter.incr(); + verifyCounterStatisticValue(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 { + assertCounterStatisticIsUntracked(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(() -> + assertCounterStatisticIsTracked(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(() -> + verifyCounterStatisticValue(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.javaRoundTrip(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..f8c1d7a1dc226 --- /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.assertCounterStatisticIsTracked; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertCounterStatisticIsUntracked; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyCounterStatisticValue; +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 { + assertCounterStatisticIsUntracked(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(() -> + assertCounterStatisticIsTracked(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(() -> + verifyCounterStatisticValue(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.javaRoundTrip(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..299c7add79ae0 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestIOStatisticsSnapshot.java @@ -0,0 +1,135 @@ +/* + * 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 { + verifyCounterStatisticValue(snapshot, "c1", 0L); + verifyGaugeStatisticValue(snapshot, "g1", 1L); + verifyMinimumStatisticValue(snapshot, "m1", -1L); + verifyMeanStatisticValue(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(() -> + verifyCounterStatisticValue(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); + LOG.info("deserialized {}", deser); + verifyCounterStatisticValue(deser, "c1", 0L); + verifyGaugeStatisticValue(deser, "g1", 1L); + verifyMinimumStatisticValue(deser, "m1", -1L); + verifyMeanStatisticValue(deser, "mean0", + new MeanStatistic(0, 1)); + verifyMeanStatisticValue(deser, "mean1", + snapshot.meanStatistics().get("mean1")); + } + + @Test + public void testJavaRoundTrip() throws Throwable { + IOStatisticsSnapshot deser = (IOStatisticsSnapshot) IOStatisticAssertions.javaRoundTrip(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..cccf2744df980 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestIOStatisticsStore.java @@ -0,0 +1,189 @@ +/* + * 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.assertThatMaximumStatistic; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatMeanStatistic; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatMeanStatisticMatches; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatMinimumStatistic; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyCounterStatisticValue; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyGaugeStatisticValue; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyMaximumStatisticValue; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyMinimumStatisticValue; +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.snapshotIOStatistics; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_LIST_REQUEST; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore; + +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) + .withDurationTracking(OBJECT_LIST_REQUEST) + .build(); + } + + @After + public void teardown() { + LOG.info("stats {}", stats); + } + + /** + * Gauges go up and down. + */ + @Test + public void testGauges() throws Throwable { + stats.setGauge(GAUGE, 1); + verifyGaugeStatisticValue(stats, GAUGE, 1); + stats.incrementGauge(GAUGE, 1); + verifyGaugeStatisticValue(stats, GAUGE, 2); + stats.setGauge(GAUGE, -1); + verifyGaugeStatisticValue(stats, GAUGE, -1); + stats.incrementGauge(GAUGE, -1); + verifyGaugeStatisticValue(stats, GAUGE, -2); + Assertions.assertThat(stats.getGaugeReference(GAUGE).get()) + .isEqualTo(-2); + stats.incrementGauge(UNKNOWN, 1); + stats.setGauge(UNKNOWN, 1); + } + + @Test + public void testMinimums() throws Throwable { + stats.setMinimum(MIN, 100); + verifyMinimumStatisticValue(stats, MIN, 100); + stats.setMinimum(MIN, 100); + // will do nothing as it is higher + stats.addMinimumSample(MIN, 200); + verifyMinimumStatisticValue(stats, MIN, 100); + stats.addMinimumSample(MIN, 10); + verifyMinimumStatisticValue(stats, MIN, 10); + stats.setMinimum(UNKNOWN, 100); + stats.addMinimumSample(UNKNOWN, 200); + } + + @Test + public void testMaximums() throws Throwable { + stats.setMaximum(MAX, 100); + verifyMaximumStatisticValue(stats, MAX, 100); + stats.setMaximum(MAX, 100); + stats.addMaximumSample(MAX, 200); + verifyMaximumStatisticValue(stats, MAX, 200); + stats.addMaximumSample(MAX, 10); + verifyMaximumStatisticValue(stats, MAX, 200); + stats.setMaximum(UNKNOWN, 100); + stats.addMaximumSample(UNKNOWN, 200); + } + + @Test + public void testMeans() throws Throwable { + stats.setMeanStatistic(MEAN, + new MeanStatistic(1, 1)); + + assertThatMeanStatisticMatches(stats, MEAN, 1, 1) + .matches(p -> p.mean() == 1, "mean"); + stats.addMeanStatisticSample(MEAN, 9); + assertThatMeanStatisticMatches(stats, MEAN, 2, 10) + .matches(p -> p.mean() == 5, "mean"); + } + + @Test + public void testRoundTrip() throws Throwable { + JsonSerialization serializer + = IOStatisticsSnapshot.serializer(); + stats.incrementCounter(COUNT, 1); + 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); + verifyCounterStatisticValue(deser, COUNT, 1L); + verifyGaugeStatisticValue(deser, GAUGE, -1); + verifyMaximumStatisticValue(deser, MAX, 200); + verifyMinimumStatisticValue(deser, MIN, -100); + assertThatMeanStatisticMatches(deser, MEAN, 2, 10) + .matches(p -> p.mean() == 5, "mean"); + + } + + /** + * Duration tracking. + */ + @Test + public void testDuration() throws Throwable { + DurationTracker tracker = + stats.trackDuration(OBJECT_LIST_REQUEST); + verifyCounterStatisticValue(stats, OBJECT_LIST_REQUEST, 1L); + Thread.sleep(1000); + tracker.close(); + try (DurationTracker ignored = + stats.trackDuration(OBJECT_LIST_REQUEST)) { + Thread.sleep(1000); + } + LOG.info("Statistics: {}", stats); + verifyCounterStatisticValue(stats, OBJECT_LIST_REQUEST, 2L); + assertThatMinimumStatistic(stats, OBJECT_LIST_REQUEST + ".min") + .isGreaterThan(0); + assertThatMaximumStatistic(stats, OBJECT_LIST_REQUEST + ".max") + .isGreaterThan(0); + assertThatMeanStatistic(stats, OBJECT_LIST_REQUEST + ".mean") + .hasFieldOrPropertyWithValue("samples", 2L) + .matches(s -> s.getSum() > 0) + .matches(s -> s.mean() > 0.0); + } + +} 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..f40e1303bcd69 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/functional/TestRemoteIterators.java @@ -0,0 +1,416 @@ +/* + * 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 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.closingRemoteIterator; +import static org.apache.hadoop.util.functional.RemoteIterators.filteringRemoteIterator; +import static org.apache.hadoop.util.functional.RemoteIterators.foreach; +import static org.apache.hadoop.util.functional.RemoteIterators.mappingRemoteIterator; +import static org.apache.hadoop.util.functional.RemoteIterators.remoteIteratorFromArray; +import static org.apache.hadoop.util.functional.RemoteIterators.remoteIteratorFromIterable; +import static org.apache.hadoop.util.functional.RemoteIterators.remoteIteratorFromIterator; +import static org.apache.hadoop.util.functional.RemoteIterators.remoteIteratorFromSingleton; +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 { + verifySize(remoteIteratorFromArray(data), data.length, + (s) -> LOG.info(s)); + } + + @Test + public void testIterateArrayMapped() throws Throwable { + verifySize( + 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); + verifySize( + 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); + verifySize(it, 1, this::log); + close(it); + closeCounter.assertCloseCount(0); + } + + @Test + public void testSingletonStats() throws Throwable { + IOStatsInstance singleton = new IOStatsInstance(); + RemoteIterator it + = remoteIteratorFromSingleton(singleton); + extractStatistics(it); + } + + @Test + public void testMappedSingletonStats() throws Throwable { + IOStatsInstance singleton = new IOStatsInstance(); + RemoteIterator it + = mappingRemoteIterator(remoteIteratorFromSingleton(singleton), + Object::toString); + verifySize(it, 1, this::log); + extractStatistics(it); + } + + @Test + public void testIteratorPassthrough() throws Throwable { + CountdownRemoteIterator it = new CountdownRemoteIterator(0); + verifySize(it, 0, this::log); + extractStatistics(it); + it.close(); + it.assertCloseCount(1); + } + + @Test + public void testMapping() throws Throwable { + CountdownRemoteIterator countdown = new CountdownRemoteIterator(100); + RemoteIterator it = mappingRemoteIterator( + countdown, + i -> i); + verifySize(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); + verifySize(it, 50, c -> counter++); + assertCounterValue(50); + extractStatistics(it); + close(it); + countdown.assertCloseCount(1); + } + + @Test + public void testFilterNoneAccepted() throws Throwable { + // nothing gets through + RemoteIterator it = filteringRemoteIterator( + new CountdownRemoteIterator(100), + i -> false); + verifySize(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); + verifySize(it, 100, c -> counter++); + assertStringValueContains(it, "CountdownRemoteIterator"); + } + + @Test + public void testIteratorSupport() throws Throwable { + CountdownIterator countdownIterator = new CountdownIterator(100); + RemoteIterator it = remoteIteratorFromIterator( + countdownIterator); + verifySize(it, 100, c -> counter++); + assertStringValueContains(it, "CountdownIterator"); + extractStatistics(it); + close(it); + countdownIterator.assertCloseCount(1); + } + + @Test + public void testIterableSupport() throws Throwable { + CountdownIterable countdown = new CountdownIterable(100); + RemoteIterator it = remoteIteratorFromIterable( + countdown); + verifySize(it, 100, c -> counter++); + assertStringValueContains(it, "CountdownIterator"); + extractStatistics(it); + // close the iterator + close(it); + countdown.assertCloseCount(0); + // and a new iterator can be crated + verifySize(remoteIteratorFromIterable(countdown), + 100, c -> counter++); + + } + + /** + * If a RemoteIterator is constructed from an iterable + * and that is to be closed, we add it. + * @throws Throwable + */ + @Test + public void testIterableClose() throws Throwable { + CountdownIterable countdown = new CountdownIterable(100); + RemoteIterator it = closingRemoteIterator( + remoteIteratorFromIterable(countdown), + countdown); + verifySize(it, 100, c -> counter++); + assertStringValueContains(it, "CountdownIterator"); + extractStatistics(it); + // close the iterator + close(it); + countdown.assertCloseCount(1); + // and a new iterator cannot be created + intercept(IllegalStateException.class, () -> + remoteIteratorFromIterable(countdown)); + } + + /** + * 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("Iterator 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 verifySize(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 = 0; + + @Override + public void close() throws IOException { + 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 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 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() { + return limit--; + } + + @Override + public String toString() { + return "CountdownIterator{" + + "limit=" + limit + + '}'; + } + } + + /** + * Iterable for countdown iterators. + * Once closed, calls to iterator() raise an exception. + */ + private static 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 9a899361a0904e459a093ab3b96f47b5ae9a7bce Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 22 Sep 2020 14:44:25 +0100 Subject: [PATCH 02/16] HADOOP-16830. Reinstate the deleted FunctionsRaisingIOE class. This is to allow hadoop-aws to compile without needing to move to the moved interfaces; it will also allow anything external which used those methods (unlikely) to keep working. Also: throw synchronized at methods to get findbugs to STFU. Tempting just to turn it off on the basis that it is overkill, but it could maybe be correct. Making all of MeanStatistics synchronized is irritating, as it will hurt performance on what should be a lightweight class. But it is needed to ensure samples and sum are consistently set. Change-Id: I4c3e2726e1e97d705c55dbb43a507ea4d0e81e28 --- .../hadoop/fs/impl/FunctionsRaisingIOE.java | 73 +++++++++++++++++++ .../fs/statistics/IOStatisticsSnapshot.java | 10 +-- .../hadoop/fs/statistics/MeanStatistic.java | 4 +- 3 files changed, 80 insertions(+), 7 deletions(-) create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FunctionsRaisingIOE.java 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 new file mode 100644 index 0000000000000..b406dcae42667 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FunctionsRaisingIOE.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.impl; + +import java.io.IOException; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Support for functional programming/lambda-expressions. + * @deprecated use {@code org.apache.hadoop.util.functional} classes + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public final class FunctionsRaisingIOE { + + 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 { + + R apply(T t) throws IOException; + } + + /** + * Function of arity 2 which may raise an IOException. + * @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 { + + R apply(T t, U u) throws IOException; + } + + /** + * 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 { + + R apply() throws IOException; + } + +} 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 index c6dd490767fd2..1e47f9ccb1c38 100644 --- 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 @@ -179,27 +179,27 @@ public synchronized boolean aggregate( } @Override - public Map counters() { + public synchronized Map counters() { return counters; } @Override - public Map gauges() { + public synchronized Map gauges() { return gauges; } @Override - public Map minimums() { + public synchronized Map minimums() { return minimums; } @Override - public Map maximums() { + public synchronized Map maximums() { return maximums; } @Override - public Map meanStatistics() { + public synchronized Map meanStatistics() { return meanStatistics; } 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 index 69683ae0500e9..af43a71a6deaf 100644 --- 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 @@ -258,8 +258,8 @@ public synchronized boolean equals(final Object o) { // if we are empty, then so must the other. return that.isEmpty(); } - return sum == that.sum && - samples == that.samples; + return getSum() == that.getSum() && + getSamples() == that.getSamples(); } @Override From 645cc2488adf9c34689d47c2abc2f46863af7225 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 23 Sep 2020 12:12:01 +0100 Subject: [PATCH 03/16] HADOOP-16830. Checkstyles and findbugs Change-Id: I5f64704a82a196fd8ff66cbde10d4970722e1fd7 --- .../hadoop/fs/statistics/IOStatisticsSnapshot.java | 2 +- .../hadoop/fs/statistics/StoreStatisticNames.java | 5 +++-- .../fs/statistics/impl/IOStatisticsBinding.java | 1 - .../fs/statistics/impl/IOStatisticsStoreImpl.java | 10 +++++----- .../impl/StubDurationTrackerFactory.java | 3 ++- .../hadoop/util/functional/RemoteIterators.java | 2 +- .../AbstractContractStreamIOStatisticsTest.java | 1 - .../fs/statistics/TestIOStatisticsSnapshot.java | 14 +++++++++++++- .../util/functional/TestRemoteIterators.java | 14 +++++++------- 9 files changed, 32 insertions(+), 20 deletions(-) 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 index 1e47f9ccb1c38..bf6df9186698f 100644 --- 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 @@ -111,7 +111,7 @@ public IOStatisticsSnapshot(IOStatistics source) { /** * Create the maps. */ - private void createMaps() { + private synchronized void createMaps() { counters = new ConcurrentHashMap<>(); gauges = new ConcurrentHashMap<>(); minimums = new ConcurrentHashMap<>(); 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 index 6c7572944accc..058e1935914a2 100644 --- 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 @@ -253,11 +253,12 @@ public final class StoreStatisticNames { /** - * The name of the statistic collected for executor aquisition if + * The name of the statistic collected for executor acquisition if * a duration tracker factory is passed in to the constructor. * {@value}. */ - public static String ACTION_EXECUTOR_ACQUIRED = "action_executor_acquired"; + public static final String ACTION_EXECUTOR_ACQUIRED = + "action_executor_acquired"; /** * An HTTP HEAD request was made: {@value}. 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 index 33b62d387e97a..97de5a9f2fe44 100644 --- 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 @@ -21,7 +21,6 @@ import java.io.Serializable; import java.util.Iterator; import java.util.Map; -import java.util.TreeMap; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicLong; import java.util.function.BiFunction; 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 index b0c5ddd35d033..bcfbe540ab1a4 100644 --- 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 @@ -257,11 +257,11 @@ public synchronized boolean aggregate( Map sourceCounters = source.counters(); counterMap.entrySet(). forEach(e -> { - Long sourceValue = lookupQuietly(sourceCounters, e.getKey()); - if (sourceValue != null) { - e.getValue().addAndGet(sourceValue); - } - }); + 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 -> { 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 index 3bb13d2e0473a..4a0d590425e7f 100644 --- 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 @@ -25,7 +25,8 @@ * 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 { +public final class StubDurationTrackerFactory + implements DurationTrackerFactory { /** * Single instance. 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 index 47a04f101256c..9bfbe454d5375 100644 --- 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 @@ -568,7 +568,7 @@ private MaybeClose(Object o) { } /** - * Construct -close the object if it is closeable and close==true + * Construct -close the object if it is closeable and close==true. * @param o object to close. * @param close should close? */ 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 index 24d0c2840caa0..ad11859e1e9a9 100644 --- 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 @@ -31,7 +31,6 @@ 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.IOStatisticsLogging; import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.extractStatistics; 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 index 299c7add79ae0..3cd52ba066f51 100644 --- 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 @@ -115,6 +115,16 @@ public void testJsonRoundTrip() throws Throwable { 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); verifyCounterStatisticValue(deser, "c1", 0L); verifyGaugeStatisticValue(deser, "g1", 1L); @@ -127,7 +137,9 @@ public void testJsonRoundTrip() throws Throwable { @Test public void testJavaRoundTrip() throws Throwable { - IOStatisticsSnapshot deser = (IOStatisticsSnapshot) IOStatisticAssertions.javaRoundTrip(snapshot); + verifyDeserializedInstance( + IOStatisticAssertions.javaRoundTrip( + snapshot)); } 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 index f40e1303bcd69..436c2f43585e1 100644 --- 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 @@ -54,14 +54,14 @@ public class TestRemoteIterators extends AbstractHadoopTestBase { private static final Logger LOG = LoggerFactory.getLogger( TestRemoteIterators.class); - private static final String[] data = {"a", "b", "c"}; + private static final String[] DATA = {"a", "b", "c"}; /** Counter for lambda-expressions. */ private int counter; @Test public void testIterateArray() throws Throwable { - verifySize(remoteIteratorFromArray(data), data.length, + verifySize(remoteIteratorFromArray(DATA), DATA.length, (s) -> LOG.info(s)); } @@ -69,12 +69,12 @@ public void testIterateArray() throws Throwable { public void testIterateArrayMapped() throws Throwable { verifySize( mappingRemoteIterator( - remoteIteratorFromArray(data), + remoteIteratorFromArray(DATA), (d) -> { counter += d.length(); return d; }), - data.length, + DATA.length, this::log); assertCounterValue(3); } @@ -335,7 +335,7 @@ public IOStatistics getIOStatistics() { /** * Iterator which counts down. */ - private static class CountdownRemoteIterator extends CloseCounter + private static final class CountdownRemoteIterator extends CloseCounter implements RemoteIterator { private int limit; @@ -365,7 +365,7 @@ public String toString() { /** * Iterator which counts down. */ - private static class CountdownIterator extends CloseCounter + private static final class CountdownIterator extends CloseCounter implements Iterator { private int limit; @@ -396,7 +396,7 @@ public String toString() { * Iterable for countdown iterators. * Once closed, calls to iterator() raise an exception. */ - private static class CountdownIterable extends CloseCounter + private static final class CountdownIterable extends CloseCounter implements Iterable { private int limit; From 5c0858efdc232dc24c76ee7fa6670ef798ae7a54 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 24 Sep 2020 14:20:50 +0100 Subject: [PATCH 04/16] HADOOP-16830. findbugs is finally silent Change-Id: I3e9bb83bc32eddc5c7d84c1df7be77cea3e60f5d --- .../org/apache/hadoop/fs/statistics/IOStatisticsSnapshot.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 index bf6df9186698f..45943dcbef14f 100644 --- 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 @@ -235,7 +235,7 @@ private synchronized void writeObject(ObjectOutputStream s) * Deserialize by loading each TreeMap, and building concurrent * hash maps from them. */ - private synchronized void readObject(final ObjectInputStream s) + private void readObject(final ObjectInputStream s) throws IOException, ClassNotFoundException { // read in core s.defaultReadObject(); From b3d4c1f91d5ddd4127b46bd1d295dd6dac9896cb Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 29 Sep 2020 17:29:03 +0100 Subject: [PATCH 05/16] HADOOP-16830. DurationTracker to handle failures - failures are registered and reported differently - helper in statistics.impl to wrap functions and callable with handling for this Move BufferedIOStatistics streams from impl to fs.statistics, so its declared OK for others to use counter increment to ignore negative values; returns existing value. successful increment returns the latest value, as documented. Tests for the counter and gauge behaviour Change-Id: Ic851240ebe94033bf93e0a11abefb7adda534191 --- .../apache/hadoop/fs/RawLocalFileSystem.java | 18 +-- .../BufferedIOStatisticsInputStream.java | 13 +- .../BufferedIOStatisticsOutputStream.java | 13 +- .../hadoop/fs/statistics/DurationTracker.java | 8 ++ .../fs/statistics/StoreStatisticNames.java | 7 ++ .../statistics/impl/IOStatisticsBinding.java | 116 ++++++++++++++++++ .../fs/statistics/impl/IOStatisticsStore.java | 3 +- .../impl/IOStatisticsStoreBuilderImpl.java | 17 ++- .../impl/IOStatisticsStoreImpl.java | 21 +++- .../impl/StatisticDurationTracker.java | 27 +++- .../statistics/impl/StubDurationTracker.java | 5 + .../site/markdown/filesystem/iostatistics.md | 8 +- .../fs/statistics/TestIOStatisticsStore.java | 99 +++++++++++++-- 13 files changed, 311 insertions(+), 44 deletions(-) rename hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/{impl => }/BufferedIOStatisticsInputStream.java (89%) rename hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/{impl => }/BufferedIOStatisticsOutputStream.java (93%) 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 e25e1483def9a..fefd4deb14526 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 @@ -50,7 +50,7 @@ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.statistics.IOStatisticsSource; -import org.apache.hadoop.fs.statistics.impl.BufferedIOStatisticsOutputStream; +import org.apache.hadoop.fs.statistics.BufferedIOStatisticsOutputStream; import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.nativeio.NativeIO; @@ -178,11 +178,11 @@ public int read() throws IOException { if (value >= 0) { this.position++; statistics.incrementBytesRead(1); - ioStatistics.incrementCounter(STREAM_READ_BYTES, 1); + ioStatistics.incrementCounter(STREAM_READ_BYTES); } return value; } catch (IOException e) { // unexpected exception - ioStatistics.incrementCounter(STREAM_READ_EXCEPTIONS, 1); + ioStatistics.incrementCounter(STREAM_READ_EXCEPTIONS); throw new FSError(e); // assume native fs error } } @@ -200,7 +200,7 @@ public int read(byte[] b, int off, int len) throws IOException { } return value; } catch (IOException e) { // unexpected exception - ioStatistics.incrementCounter(STREAM_READ_EXCEPTIONS, 1); + ioStatistics.incrementCounter(STREAM_READ_EXCEPTIONS); throw new FSError(e); // assume native fs error } } @@ -223,14 +223,14 @@ public int read(long position, byte[] b, int off, int len) } return value; } catch (IOException e) { - ioStatistics.incrementCounter(STREAM_READ_EXCEPTIONS, 1); + ioStatistics.incrementCounter(STREAM_READ_EXCEPTIONS); throw new FSError(e); } } @Override public long skip(long n) throws IOException { - ioStatistics.incrementCounter(STREAM_READ_SKIP_OPERATIONS, 1); + ioStatistics.incrementCounter(STREAM_READ_SKIP_OPERATIONS); long value = fis.skip(n); if (value > 0) { this.position += value; @@ -338,7 +338,7 @@ public void write(byte[] b, int off, int len) throws IOException { fos.write(b, off, len); ioStatistics.incrementCounter(STREAM_WRITE_BYTES, len); } catch (IOException e) { // unexpected exception - ioStatistics.incrementCounter(STREAM_WRITE_EXCEPTIONS, 1); + ioStatistics.incrementCounter(STREAM_WRITE_EXCEPTIONS); throw new FSError(e); // assume native fs error } } @@ -347,9 +347,9 @@ 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, 1); + ioStatistics.incrementCounter(STREAM_WRITE_BYTES); } catch (IOException e) { // unexpected exception - ioStatistics.incrementCounter(STREAM_WRITE_EXCEPTIONS, 1); + ioStatistics.incrementCounter(STREAM_WRITE_EXCEPTIONS); throw new FSError(e); // assume native fs error } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/BufferedIOStatisticsInputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/BufferedIOStatisticsInputStream.java similarity index 89% rename from hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/BufferedIOStatisticsInputStream.java rename to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/BufferedIOStatisticsInputStream.java index 718487a7d7242..b89651bc12bca 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/BufferedIOStatisticsInputStream.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/BufferedIOStatisticsInputStream.java @@ -16,14 +16,12 @@ * limitations under the License. */ -package org.apache.hadoop.fs.statistics.impl; +package org.apache.hadoop.fs.statistics; import java.io.BufferedInputStream; import java.io.InputStream; import org.apache.hadoop.fs.StreamCapabilities; -import org.apache.hadoop.fs.statistics.IOStatistics; -import org.apache.hadoop.fs.statistics.IOStatisticsSource; import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics; @@ -42,10 +40,19 @@ 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); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/BufferedIOStatisticsOutputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/BufferedIOStatisticsOutputStream.java similarity index 93% rename from hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/BufferedIOStatisticsOutputStream.java rename to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/BufferedIOStatisticsOutputStream.java index 86744c5f7e4f5..f85aa7bbb72b6 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/BufferedIOStatisticsOutputStream.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/BufferedIOStatisticsOutputStream.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.fs.statistics.impl; +package org.apache.hadoop.fs.statistics; import java.io.BufferedOutputStream; import java.io.IOException; @@ -24,8 +24,6 @@ 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 static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics; @@ -59,7 +57,8 @@ public class BufferedIOStatisticsOutputStream * @param out output stream to buffer * @param downgradeSyncable should Syncable calls downgrade? */ - public BufferedIOStatisticsOutputStream(final OutputStream out, + public BufferedIOStatisticsOutputStream( + final OutputStream out, final boolean downgradeSyncable) { super(out); this.downgradeSyncable = downgradeSyncable; @@ -72,8 +71,10 @@ public BufferedIOStatisticsOutputStream(final OutputStream out, * @param size buffer. * @param downgradeSyncable should Syncable calls downgrade? */ - public BufferedIOStatisticsOutputStream(final OutputStream out, - final int size, final boolean downgradeSyncable) { + public BufferedIOStatisticsOutputStream( + final OutputStream out, + final int size, + final boolean downgradeSyncable) { super(out, size); this.downgradeSyncable = downgradeSyncable; } 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 index 9d1f6f42df199..b3fa1d319d8ac 100644 --- 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 @@ -23,9 +23,17 @@ * 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. */ 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 index 058e1935914a2..f065641357f00 100644 --- 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 @@ -251,6 +251,13 @@ public final class StoreStatisticNames { */ 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 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 index 97de5a9f2fe44..fec197fa5fb5f 100644 --- 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 @@ -18,9 +18,11 @@ package org.apache.hadoop.fs.statistics.impl; +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; @@ -29,9 +31,13 @@ import 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.FunctionRaisingIOE; import static org.apache.hadoop.fs.statistics.IOStatistics.MIN_UNSET_VALUE; @@ -334,4 +340,114 @@ public static void maybeUpdateMinimum(AtomicLong dest, long sample) { } 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( + DurationTrackerFactory factory, + String statistic, + FunctionRaisingIOE inputFn) { + return (A x) -> { + // create the tracker outside try-with-resources so + // that failures can be set in the catcher. + DurationTracker tracker = factory.trackDuration(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 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 trackDuration( + 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 = factory.trackDuration(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 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( + 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 = factory.trackDuration(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 the catch() call would have + // set the failed flag. + tracker.close(); + } + }; + } + + } 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 index b4dbcfe22b18a..a79a43fd40055 100644 --- 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 @@ -49,9 +49,10 @@ default long incrementCounter(String key) { * 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 old value or, if the counter is unknown: 0 + * @return the updated value or, if the counter is unknown: 0 */ long incrementCounter(String key, long value); 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 index 5068b24043a54..0562271db3ef8 100644 --- 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 @@ -22,12 +22,13 @@ 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 implementation. + * Builder for an IOStatistics store.. */ final class IOStatisticsStoreBuilderImpl implements IOStatisticsStoreBuilder { @@ -77,10 +78,16 @@ public IOStatisticsStoreBuilderImpl withMeanStatistics( public IOStatisticsStoreBuilderImpl withDurationTracking( final String... prefixes) { for (String p : prefixes) { - withCounters(p); - withMinimums(p + SUFFIX_MIN); - withMaximums(p + SUFFIX_MAX); - withMeanStatistics(p + SUFFIX_MEAN); + 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; } 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 index bcfbe540ab1a4..c9b75445289ad 100644 --- 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 @@ -159,10 +159,23 @@ public void setCounter(final String key, final long value) { @Override public long incrementCounter(final String key, final long value) { - long l = incAtomicLong(counterMap.get(key), value); - LOG.debug("Incrementing counter {} by {} with final value {}", - key, value, l); - return l; + 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 incrment 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 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 index ba2c9015d6311..d3bacbaa510ed 100644 --- 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 @@ -19,6 +19,7 @@ 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; /** @@ -35,7 +36,7 @@ public class StatisticDurationTracker extends OperationDuration implements DurationTracker { /** - * statistics to update. + * Statistics to update. */ private final IOStatisticsStore iostats; @@ -44,12 +45,18 @@ public class StatisticDurationTracker extends OperationDuration */ 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, + public StatisticDurationTracker( + final IOStatisticsStore iostats, final String key) { this(iostats, key, 1); } @@ -62,7 +69,8 @@ public StatisticDurationTracker(final IOStatisticsStore iostats, * @param key Key to use as prefix of values. * @param count #of times to increment the matching counter. */ - public StatisticDurationTracker(final IOStatisticsStore iostats, + public StatisticDurationTracker( + final IOStatisticsStore iostats, final String key, final int count) { this.iostats = iostats; @@ -72,12 +80,23 @@ public StatisticDurationTracker(final IOStatisticsStore iostats, } } + @Override + public void failed() { + failed = true; + } + /** * Set the finished time and then update the statistics. */ @Override public void close() { finished(); - iostats.addTimedOperation(key, asDuration()); + 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/StubDurationTracker.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StubDurationTracker.java index 87cc77877f4bd..82c894ce96a21 100644 --- 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 @@ -32,6 +32,11 @@ public final class StubDurationTracker implements DurationTracker { private StubDurationTracker() { } + @Override + public void failed() { + + } + @Override public void close() { 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 index f705424e78f0a..bd77dc7e0f8a7 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/iostatistics.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/iostatistics.md @@ -57,10 +57,10 @@ 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| +| `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| +| `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. @@ -427,6 +427,6 @@ the provisioning of a public implementation MAY BE raised via the Hadoop develop 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 need to -be aware that the implementation this code is unstable and may change across +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/statistics/TestIOStatisticsStore.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestIOStatisticsStore.java index cccf2744df980..61702eab792b0 100644 --- 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 @@ -28,6 +28,7 @@ import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; import org.apache.hadoop.test.AbstractHadoopTestBase; import org.apache.hadoop.util.JsonSerialization; +import org.apache.hadoop.util.functional.FunctionRaisingIOE; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatMaximumStatistic; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatMeanStatistic; @@ -39,8 +40,18 @@ import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyMinimumStatisticValue; import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.snapshotIOStatistics; import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_LIST_REQUEST; +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; import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackFunctionDuration; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; +/** + * + * Test the IOStatisticStore implementation. + */ public class TestIOStatisticsStore extends AbstractHadoopTestBase { private static final Logger LOG = @@ -89,12 +100,14 @@ public void testGauges() throws Throwable { verifyGaugeStatisticValue(stats, GAUGE, 2); stats.setGauge(GAUGE, -1); verifyGaugeStatisticValue(stats, GAUGE, -1); - stats.incrementGauge(GAUGE, -1); + Assertions.assertThat(stats.incrementGauge(GAUGE, -1)) + .isEqualTo(-2); verifyGaugeStatisticValue(stats, GAUGE, -2); Assertions.assertThat(stats.getGaugeReference(GAUGE).get()) .isEqualTo(-2); - stats.incrementGauge(UNKNOWN, 1); stats.setGauge(UNKNOWN, 1); + Assertions.assertThat(stats.incrementGauge(UNKNOWN, 1)) + .isEqualTo(0); } @Test @@ -140,7 +153,7 @@ public void testMeans() throws Throwable { public void testRoundTrip() throws Throwable { JsonSerialization serializer = IOStatisticsSnapshot.serializer(); - stats.incrementCounter(COUNT, 1); + stats.incrementCounter(COUNT); stats.setGauge(GAUGE, -1); stats.addMaximumSample(MAX, 200); stats.addMinimumSample(MIN, -100); @@ -160,6 +173,19 @@ public void testRoundTrip() throws Throwable { } + @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); + } + /** * Duration tracking. */ @@ -168,22 +194,79 @@ public void testDuration() throws Throwable { DurationTracker tracker = stats.trackDuration(OBJECT_LIST_REQUEST); verifyCounterStatisticValue(stats, OBJECT_LIST_REQUEST, 1L); - Thread.sleep(1000); + sleep(); tracker.close(); try (DurationTracker ignored = stats.trackDuration(OBJECT_LIST_REQUEST)) { - Thread.sleep(1000); + sleep(); } LOG.info("Statistics: {}", stats); verifyCounterStatisticValue(stats, OBJECT_LIST_REQUEST, 2L); - assertThatMinimumStatistic(stats, OBJECT_LIST_REQUEST + ".min") + assertThatMinimumStatistic(stats, + OBJECT_LIST_REQUEST + SUFFIX_MIN) .isGreaterThan(0); - assertThatMaximumStatistic(stats, OBJECT_LIST_REQUEST + ".max") + assertThatMaximumStatistic(stats, + OBJECT_LIST_REQUEST + SUFFIX_MAX) .isGreaterThan(0); - assertThatMeanStatistic(stats, OBJECT_LIST_REQUEST + ".mean") + assertThatMeanStatistic(stats, + OBJECT_LIST_REQUEST + SUFFIX_MEAN) .hasFieldOrPropertyWithValue("samples", 2L) .matches(s -> s.getSum() > 0) .matches(s -> s.mean() > 0.0); } + public void sleep() { + try { + Thread.sleep(1000); + } catch (InterruptedException ignored) { + } + } + + @Test + public void testDurationSuccess() throws Throwable { + FunctionRaisingIOE fn + = trackFunctionDuration(stats, OBJECT_LIST_REQUEST, + (Integer x) -> x); + Assertions.assertThat(fn.apply(1)).isEqualTo(1); + verifyCounterStatisticValue(stats, OBJECT_LIST_REQUEST, 1L); + assertThatMinimumStatistic(stats, + OBJECT_LIST_REQUEST + SUFFIX_MIN) + .isGreaterThanOrEqualTo(0); + assertThatMaximumStatistic(stats, + OBJECT_LIST_REQUEST + SUFFIX_MAX) + .isGreaterThanOrEqualTo(0); + + } + + /** + * Trigger a failure and verify its the failure statistics + * which go up. + */ + @Test + public void testDurationFailure() throws Throwable { + FunctionRaisingIOE fn = + trackFunctionDuration(stats, OBJECT_LIST_REQUEST, + (Integer x) -> { + sleep(); + return 100 / x; + }); + intercept(ArithmeticException.class, + () -> fn.apply(0)); + verifyCounterStatisticValue(stats, OBJECT_LIST_REQUEST, 1L); + verifyCounterStatisticValue(stats, + OBJECT_LIST_REQUEST + SUFFIX_FAILURES, 1L); + assertThatMinimumStatistic(stats, + OBJECT_LIST_REQUEST + SUFFIX_MIN) + .isLessThan(0); + assertThatMaximumStatistic(stats, + OBJECT_LIST_REQUEST + SUFFIX_MAX) + .isLessThan(0); + assertThatMinimumStatistic(stats, + OBJECT_LIST_REQUEST + SUFFIX_FAILURES + SUFFIX_MIN) + .isGreaterThan(0); + assertThatMaximumStatistic(stats, + OBJECT_LIST_REQUEST + SUFFIX_FAILURES + SUFFIX_MAX) + .isGreaterThan(0); + } + } From cfa4daa13d2ebbfa964f7b3045401eb243fd461f Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 30 Sep 2020 13:59:23 +0100 Subject: [PATCH 06/16] HADOOP-16830. Closeable iterator is self closing. on hasNext/Next, close() is called, so as to cleanup any open state/handles etc. Avoids having to tell developers downstream to look for an iterator being closeable, and call it if so. + minor improvements in iterator test suite + fix typo found by Mehakmeet Change-Id: Ibd103941278b8c0bc6c93a09ea469be4c60a58b1 --- .../impl/IOStatisticsStoreImpl.java | 2 +- .../util/functional/RemoteIterators.java | 86 +++++++++--- .../util/functional/TestRemoteIterators.java | 131 ++++++++++++------ 3 files changed, 162 insertions(+), 57 deletions(-) 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 index c9b75445289ad..80650e56aa2a0 100644 --- 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 @@ -166,7 +166,7 @@ public long incrementCounter(final String key, final long value) { return 0; } if (value < 0) { - LOG.debug("Ignoring negative incrment value {} for counter {}", + LOG.debug("Ignoring negative increment value {} for counter {}", value, key); // returns old value return counter.get(); 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 index 9bfbe454d5375..f5041cbb5fc6b 100644 --- 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 @@ -36,6 +36,7 @@ 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; @@ -102,7 +103,7 @@ public static RemoteIterator remoteIteratorFromSingleton( */ public static RemoteIterator remoteIteratorFromIterator( Iterator iterator) { - return new FromIterator<>(iterator); + return new WrappedJavaIterator<>(iterator); } /** @@ -113,7 +114,7 @@ public static RemoteIterator remoteIteratorFromIterator( */ public static RemoteIterator remoteIteratorFromIterable( Iterable iterable) { - return new FromIterator<>(iterable.iterator()); + return new WrappedJavaIterator<>(iterable.iterator()); } /** @@ -122,7 +123,7 @@ public static RemoteIterator remoteIteratorFromIterable( * @return a remote iterator */ public static RemoteIterator remoteIteratorFromArray(T[] array) { - return new FromIterator<>(Arrays.stream(array).iterator()); + return new WrappedJavaIterator<>(Arrays.stream(array).iterator()); } /** @@ -217,17 +218,24 @@ public static long foreach( ConsumerRaisingIOE consumer) throws IOException { long count = 0; - while (source.hasNext()) { - count++; - consumer.accept(source.next()); + 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); + } } - // maybe log the results - logIOStatisticsAtDebug(LOG, "RemoteIterator Statistics: {}", source); return count; } - /** * A remote iterator from a singleton. It has a single next() * value, after which hasNext() returns false and next() fails. @@ -242,7 +250,7 @@ private static final class SingletonIterator /** * Single entry. */ - private T singleton; + private final T singleton; /** Has the entry been processed? */ private boolean processed; @@ -297,7 +305,7 @@ public String toString() { * Closeable; * @param iterator type. */ - private static final class FromIterator + private static final class WrappedJavaIterator implements RemoteIterator, IOStatisticsSource, Closeable { /** @@ -312,7 +320,7 @@ private static final class FromIterator * Construct from an interator. * @param source source iterator. */ - private FromIterator(Iterator source) { + private WrappedJavaIterator(Iterator source) { this.source = requireNonNull(source); sourceToClose = new MaybeClose(source); } @@ -379,6 +387,43 @@ 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 + * @return the next value + * @throws IOException failure + * @throws NoSuchElementException no more data + */ + protected S sourceNext() throws IOException { + try { + return getSource().next(); + } catch (NoSuchElementException | IOException e) { + IOUtils.cleanupWithLogger(LOG, this); + throw e; + } + } + @Override public String toString() { return source.toString(); @@ -408,12 +453,12 @@ private MappingRemoteIterator( @Override public boolean hasNext() throws IOException { - return getSource().hasNext(); + return sourceHasNext(); } @Override public T next() throws IOException { - return mapper.apply(getSource().next()); + return mapper.apply(sourceNext()); } @Override @@ -466,7 +511,7 @@ private FilteringRemoteIterator( * @throws IOException failure in retrieval from source or mapping */ private boolean fetch() throws IOException { - while (next == null && getSource().hasNext()) { + while (next == null && sourceHasNext()) { S candidate = getSource().next(); if (filter.apply(candidate)) { next = candidate; @@ -523,6 +568,7 @@ private static final class CloseRemoteIterator extends WrappingRemoteIterator { private final MaybeClose toClose; + private boolean closed = false; private CloseRemoteIterator( final RemoteIterator source, @@ -533,16 +579,21 @@ private CloseRemoteIterator( @Override public boolean hasNext() throws IOException { - return getSource().hasNext(); + return sourceHasNext(); } @Override public S next() throws IOException { - return getSource().next(); + return sourceNext(); } @Override public void close() throws IOException { + if (closed) { + return; + } + closed = true; + LOG.debug("Closing {}", this); try { super.close(); } finally { @@ -554,6 +605,7 @@ public void close() throws IOException { /** * 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 { 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 index 436c2f43585e1..f9cd94f452228 100644 --- 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 @@ -21,6 +21,7 @@ import java.io.Closeable; import java.io.IOException; import java.util.Iterator; +import java.util.NoSuchElementException; import com.google.common.base.Preconditions; import org.junit.Test; @@ -35,14 +36,7 @@ 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.closingRemoteIterator; -import static org.apache.hadoop.util.functional.RemoteIterators.filteringRemoteIterator; -import static org.apache.hadoop.util.functional.RemoteIterators.foreach; -import static org.apache.hadoop.util.functional.RemoteIterators.mappingRemoteIterator; -import static org.apache.hadoop.util.functional.RemoteIterators.remoteIteratorFromArray; -import static org.apache.hadoop.util.functional.RemoteIterators.remoteIteratorFromIterable; -import static org.apache.hadoop.util.functional.RemoteIterators.remoteIteratorFromIterator; -import static org.apache.hadoop.util.functional.RemoteIterators.remoteIteratorFromSingleton; +import static org.apache.hadoop.util.functional.RemoteIterators.*; import static org.assertj.core.api.Assertions.assertThat; /** @@ -61,13 +55,13 @@ public class TestRemoteIterators extends AbstractHadoopTestBase { @Test public void testIterateArray() throws Throwable { - verifySize(remoteIteratorFromArray(DATA), DATA.length, + verifyInvoked(remoteIteratorFromArray(DATA), DATA.length, (s) -> LOG.info(s)); } @Test public void testIterateArrayMapped() throws Throwable { - verifySize( + verifyInvoked( mappingRemoteIterator( remoteIteratorFromArray(DATA), (d) -> { @@ -94,7 +88,7 @@ public void testSingleton() throws Throwable { RemoteIterator it = remoteIteratorFromSingleton(name); assertStringValueContains(it, "SingletonIterator"); assertStringValueContains(it, name); - verifySize( + verifyInvoked( it, 1, (s) -> result.append(s)); @@ -106,11 +100,25 @@ public void testSingleton() throws Throwable { public void testSingletonNotClosed() throws Throwable { CloseCounter closeCounter = new CloseCounter(); RemoteIterator it = remoteIteratorFromSingleton(closeCounter); - verifySize(it, 1, this::log); + 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(); @@ -119,23 +127,35 @@ public void testSingletonStats() throws Throwable { 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); - verifySize(it, 1, this::log); + verifyInvoked(it, 1, this::log); extractStatistics(it); } + /** + * Close() calls are passed through. + */ @Test - public void testIteratorPassthrough() throws Throwable { - CountdownRemoteIterator it = new CountdownRemoteIterator(0); - verifySize(it, 0, this::log); - extractStatistics(it); - it.close(); - it.assertCloseCount(1); + 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 @@ -144,7 +164,7 @@ public void testMapping() throws Throwable { RemoteIterator it = mappingRemoteIterator( countdown, i -> i); - verifySize(it, 100, c -> counter++); + verifyInvoked(it, 100, c -> counter++); assertCounterValue(100); extractStatistics(it); assertStringValueContains(it, "CountdownRemoteIterator"); @@ -159,20 +179,24 @@ public void testFiltering() throws Throwable { RemoteIterator it = filteringRemoteIterator( countdown, i -> (i % 2) == 0); - verifySize(it, 50, c -> counter++); + 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); - verifySize(it, 0, c -> counter++); + verifyInvoked(it, 0, c -> counter++); assertCounterValue(0); extractStatistics(it); } @@ -183,16 +207,16 @@ public void testFilterAllAccepted() throws Throwable { RemoteIterator it = filteringRemoteIterator( new CountdownRemoteIterator(100), i -> true); - verifySize(it, 100, c -> counter++); + verifyInvoked(it, 100, c -> counter++); assertStringValueContains(it, "CountdownRemoteIterator"); } @Test - public void testIteratorSupport() throws Throwable { + public void testJavaIteratorSupport() throws Throwable { CountdownIterator countdownIterator = new CountdownIterator(100); RemoteIterator it = remoteIteratorFromIterator( countdownIterator); - verifySize(it, 100, c -> counter++); + verifyInvoked(it, 100, c -> counter++); assertStringValueContains(it, "CountdownIterator"); extractStatistics(it); close(it); @@ -200,37 +224,39 @@ public void testIteratorSupport() throws Throwable { } @Test - public void testIterableSupport() throws Throwable { + public void testJavaIterableSupport() throws Throwable { CountdownIterable countdown = new CountdownIterable(100); RemoteIterator it = remoteIteratorFromIterable( countdown); - verifySize(it, 100, c -> counter++); + 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 - verifySize(remoteIteratorFromIterable(countdown), + verifyInvoked(remoteIteratorFromIterable(countdown), 100, c -> counter++); - } /** * If a RemoteIterator is constructed from an iterable - * and that is to be closed, we add it. - * @throws Throwable + * and that is to be closed, we close it. */ @Test - public void testIterableClose() throws Throwable { + public void testJavaIterableClose() throws Throwable { CountdownIterable countdown = new CountdownIterable(100); RemoteIterator it = closingRemoteIterator( remoteIteratorFromIterable(countdown), countdown); - verifySize(it, 100, c -> counter++); + verifyInvoked(it, 100, c -> counter++); assertStringValueContains(it, "CountdownIterator"); extractStatistics(it); - // close the iterator + + // 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 @@ -238,6 +264,29 @@ public void testIterableClose() throws Throwable { 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. @@ -248,7 +297,7 @@ protected void assertStringValueContains( final Object o, final String expected) { assertThat(o.toString()) - .describedAs("Iterator string value") + .describedAs("Object string value") .contains(expected); } @@ -269,8 +318,9 @@ protected void assertCounterValue(final int expected) { * @param length expected size * @param consumer consumer */ - protected void verifySize(final RemoteIterator it, - int length, ConsumerRaisingIOE consumer) + protected void verifyInvoked(final RemoteIterator it, + int length, + ConsumerRaisingIOE consumer) throws IOException { assertThat(foreach(it, consumer)) .describedAs("Scan through iterator %s", it) @@ -288,18 +338,18 @@ private void close(final RemoteIterator it) throws IOException { } } - /** * Class whose close() call increments a counter. */ private static class CloseCounter extends IOStatsInstance implements Closeable { - private int closeCount = 0; + private int closeCount; @Override public void close() throws IOException { closeCount++; + LOG.info("close ${}", closeCount); } public int getCloseCount() { @@ -381,6 +431,9 @@ public boolean hasNext() { @Override public Integer next() { + if (!hasNext()) { + throw new NoSuchElementException("limit reached"); + } return limit--; } From b15d7ae17f6c3eda6845308c841b2309c11ad503 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 1 Oct 2020 20:54:45 +0100 Subject: [PATCH 07/16] HADOOP-16830. Duration Tracking testing + add class which can be used to store duration results, primarily for testing. Made public for others to play with too Change-Id: I0a7f274f5a2a180e1800102be177b308050725c0 --- .../statistics/DurationStatisticSummary.java | 154 ++++++++++ .../statistics/impl/IOStatisticsBinding.java | 40 ++- .../fs/statistics/TestDurationTracking.java | 283 ++++++++++++++++++ .../fs/statistics/TestIOStatisticsStore.java | 97 +----- 4 files changed, 477 insertions(+), 97 deletions(-) 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/test/java/org/apache/hadoop/fs/statistics/TestDurationTracking.java 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. + *
    - *
  1. If it is an IOE: Return.
  2. - *
  3. If it is a {@link RuntimeIOException}: return the cause
  4. - *
  5. Completion/Execution Exceptions: extract and repeat
  6. - *
  7. If it is an RTE: 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. - */ - private static IOException unwrapInnerException(final Throwable e) { - Throwable cause = e.getCause(); - if (cause instanceof IOException) { - return (IOException) cause; - } else if (cause instanceof RuntimeIOException) { - // this is always an IOException - return ((RuntimeIOException) 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); } /** 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 index b89651bc12bca..eb61ec27daaed 100644 --- 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 @@ -49,7 +49,7 @@ public BufferedIOStatisticsInputStream(final InputStream in) { } /** - * Buffer an input stream with the chosen buffer size + * Buffer an input stream with the chosen buffer size. * @param in input stream * @param size buffer size */ @@ -57,6 +57,10 @@ 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); 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 index f6c92ca69ca80..401c970ad01bd 100644 --- 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 @@ -414,6 +414,22 @@ public static Function trackJavaFunctionDuration( }; } + /** + * 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, * return a new one which wraps the inner and tracks @@ -425,7 +441,7 @@ public static Function trackJavaFunctionDuration( * @param return type. * @return a new callable which tracks duration and failure. */ - public static CallableRaisingIOE trackDuration( + public static CallableRaisingIOE trackDurationOfOperation( DurationTrackerFactory factory, String statistic, CallableRaisingIOE input) { @@ -479,13 +495,11 @@ public static Callable trackDurationOfCallable( throw e; } finally { // update the tracker. - // this is called after the catch() call would have + // this is called after any catch() call will have // set the failed flag. tracker.close(); } }; } - - } 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..e6136b37c6f36 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/FutureIO.java @@ -0,0 +1,178 @@ +/* + * 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.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 + * {@link RuntimeIOException} 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. + *
    + *
  1. If it is an IOE: Return.
  2. + *
  3. If it is a {@link RuntimeIOException}: return the cause
  4. + *
  5. Completion/Execution Exceptions: extract and repeat
  6. + *
  7. If it is an RTE: 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. + */ + public static IOException unwrapInnerException(final Throwable e) { + Throwable cause = e.getCause(); + if (cause instanceof IOException) { + return (IOException) cause; + } else if (cause instanceof RuntimeIOException) { + // this is always an IOException + return ((RuntimeIOException) 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); + } + } + +} 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 index f5041cbb5fc6b..5158311e21730 100644 --- 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 @@ -410,7 +410,7 @@ protected boolean sourceHasNext() throws IOException { } /** - * Get the next source value + * Get the next source value. * @return the next value * @throws IOException failure * @throws NoSuchElementException no more data @@ -568,7 +568,7 @@ private static final class CloseRemoteIterator extends WrappingRemoteIterator { private final MaybeClose toClose; - private boolean closed = false; + private boolean closed; private CloseRemoteIterator( final RemoteIterator source, diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RuntimeIOException.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RuntimeIOException.java index ae93d0c48fd74..ff97eea184232 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RuntimeIOException.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RuntimeIOException.java @@ -25,10 +25,15 @@ import org.apache.hadoop.classification.InterfaceStability; /** - * A RuntimeException which always contains an IOException. + * A {@code RuntimeException} which always contains an + * @code IOException}. *

- * The constructor signature guarantees the cause will be an IOException, - * and as it checks for a null-argument, non-null. + * The constructor signature guarantees the cause will be an + * IOException, and, as it checks for a null-argument, non-null. + *

+ * The methods in {@link FutureIO} to await a Future's completion + * will automatically extract the wrapped IOE as part of their + * processing of caught exceptions. */ @InterfaceAudience.Public @InterfaceStability.Evolving 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 index d784df3505d45..0ff89efbf0515 100644 --- 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 @@ -18,7 +18,9 @@ package org.apache.hadoop.fs.statistics; +import java.io.FileNotFoundException; import java.io.IOException; +import java.util.concurrent.CompletableFuture; import java.util.function.Function; import org.junit.After; @@ -27,14 +29,17 @@ 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; @@ -107,9 +112,9 @@ protected int sleepf(final int millis) { */ @Test public void testDurationFunctionIOE() throws Throwable { - FunctionRaisingIOE fn - = trackFunctionDuration(stats, REQUESTS, - (Integer x) -> x); + FunctionRaisingIOE fn = + trackFunctionDuration(stats, REQUESTS, + (Integer x) -> x); assertThat(fn.apply(1)).isEqualTo(1); assertSummaryValues( fetchSuccessSummary(stats, REQUESTS), @@ -198,13 +203,13 @@ public void testCallableFailureDuration() throws Throwable { } /** - * Duration of the successful execution of a CallableRaisingIOE + * Duration of the successful execution of a CallableRaisingIOE. */ @Test public void testCallableIOEDuration() throws Throwable { // call the operation assertThat( - trackDuration(stats, REQUESTS, () -> sleepf(100)).apply()) + trackDuration(stats, REQUESTS, () -> sleepf(100))) .isEqualTo(100); DurationStatisticSummary summary = fetchSuccessSummary(stats, REQUESTS); assertSummaryValues(summary, 1, 0, 0); @@ -221,7 +226,7 @@ public void testCallableIOEFailureDuration() throws Throwable { trackDuration(stats, REQUESTS, () -> { sleepf(100); throw new IOException("oops"); - }).apply()); + })); assertSummaryValues( fetchSuccessSummary(stats, REQUESTS), 1, -1, -1); @@ -230,6 +235,23 @@ public void testCallableIOEFailureDuration() throws Throwable { 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. */ @@ -241,6 +263,27 @@ public void testUnknownDuration() throws Throwable { 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 From 004907f70c25a9ed0d7499eef6d916ab00ddd733 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 7 Oct 2020 16:30:27 +0100 Subject: [PATCH 09/16] HADOOP-16830. Checkstyles &c Fix checkstyle in RemoteIOException unwrapping of exception looks for Error and rethrows Change-Id: I673b1e487cae2ec3204ce6ba1103cdae14cfe48c --- .../java/org/apache/hadoop/util/functional/FutureIO.java | 9 +++++++-- .../hadoop/util/functional/RuntimeIOException.java | 2 +- 2 files changed, 8 insertions(+), 3 deletions(-) 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 index e6136b37c6f36..3627ca4212f97 100644 --- 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 @@ -138,12 +138,13 @@ public static T raiseInnerCause(final CompletionException e) } /** - * From the inner cause of an execution exception, extract the inner cause. + * 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 RuntimeIOException}: return the cause
  4. *
  5. Completion/Execution Exceptions: extract and repeat
  6. - *
  7. If it is an RTE: throw.
  8. + *
  9. If it is an RTE or Error: throw.
  10. *
  11. Any other type: wrap in an IOE
  12. *
* @@ -152,7 +153,9 @@ public static T raiseInnerCause(final CompletionException e) * @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) { @@ -166,6 +169,8 @@ public static IOException unwrapInnerException(final Throwable e) { 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); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RuntimeIOException.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RuntimeIOException.java index ff97eea184232..ff619ad9eeb6b 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RuntimeIOException.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RuntimeIOException.java @@ -26,7 +26,7 @@ /** * A {@code RuntimeException} which always contains an - * @code IOException}. + * {@code IOException}. *

* The constructor signature guarantees the cause will be an * IOException, and, as it checks for a null-argument, non-null. From 7079ae2afe7b709bff9bcd1777bf630652cfe485 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 12 Oct 2020 17:49:00 +0100 Subject: [PATCH 10/16] HADOOP-16830. IOStatistics feature creep * the trackDuration(DurationTrackerFactory,...) methods support null factory and switch to a stub tracker * there's an overloaded logIOStatisticsAtDebug() method to use the IOStatisticsLogging own logger Change-Id: Ie786dc7aa8920a3fc8b22b55916f4b811810dab3 --- .../fs/statistics/IOStatisticsLogging.java | 18 ++++++++++- .../statistics/impl/IOStatisticsBinding.java | 32 ++++++++++++++----- .../util/functional/RemoteIterators.java | 6 ++++ 3 files changed, 47 insertions(+), 9 deletions(-) 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 index 3a4e9859d92f1..d0c1105eca340 100644 --- 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 @@ -146,7 +146,7 @@ public static Object demandStringifyIOStatistics( * 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 + * statistics report to actually get logged. * @param source source object */ public static void logIOStatisticsAtDebug( @@ -162,6 +162,22 @@ public static void logIOStatisticsAtDebug( } } + /** + * 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. *

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 index 401c970ad01bd..9c1d63f0f67b3 100644 --- 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 @@ -18,6 +18,7 @@ package org.apache.hadoop.fs.statistics.impl; +import javax.annotation.Nullable; import java.io.IOException; import java.io.Serializable; import java.util.Iterator; @@ -40,6 +41,7 @@ import org.apache.hadoop.util.functional.FunctionRaisingIOE; 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. @@ -353,13 +355,13 @@ public static void maybeUpdateMinimum(AtomicLong dest, long sample) { * @return a new function which tracks duration and failure. */ public static FunctionRaisingIOE trackFunctionDuration( - DurationTrackerFactory factory, + @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 = factory.trackDuration(statistic); + DurationTracker tracker = createTracker(factory, statistic); try { // exec the input function and return its value return inputFn.apply(x); @@ -390,13 +392,13 @@ public static FunctionRaisingIOE trackFunctionDuration( * @return a new function which tracks duration and failure. */ public static Function trackJavaFunctionDuration( - DurationTrackerFactory factory, + @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 = factory.trackDuration(statistic); + DurationTracker tracker = createTracker(factory, statistic); try { // exec the input function and return its value return inputFn.apply(x); @@ -442,13 +444,13 @@ public static B trackDuration( * @return a new callable which tracks duration and failure. */ public static CallableRaisingIOE trackDurationOfOperation( - DurationTrackerFactory factory, + @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 = factory.trackDuration(statistic); + DurationTracker tracker = createTracker(factory, statistic); try { // exec the input function and return its value return input.apply(); @@ -478,13 +480,13 @@ public static CallableRaisingIOE trackDurationOfOperation( * @return a new callable which tracks duration and failure. */ public static Callable trackDurationOfCallable( - DurationTrackerFactory factory, + @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 = factory.trackDuration(statistic); + DurationTracker tracker = createTracker(factory, statistic); try { // exec the input function and return its value return input.call(); @@ -502,4 +504,18 @@ public static Callable trackDurationOfCallable( }; } + /** + * 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; + } } 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 index 5158311e21730..39d847042ed07 100644 --- 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 @@ -411,12 +411,17 @@ protected boolean sourceHasNext() throws IOException { /** * 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); @@ -584,6 +589,7 @@ public boolean hasNext() throws IOException { @Override public S next() throws IOException { + return sourceNext(); } From 04cd4f50724964d7c0edba01017d19219fa7a39b Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 14 Oct 2020 15:59:20 +0100 Subject: [PATCH 11/16] HADOOP-16830. Improving integration with S3A code * PairedDurationTrackerFactory allows listing code to update both FS and instance level stats. * test assertions to become verifyStatistic* assertStatistic* to make clear these are statistic assertions; also helps code completion * stream_read_unbuffered stream key Change-Id: I25272fffeb3e66b5cec90ae6c6af74808c139b26 --- .../fs/statistics/StreamStatisticNames.java | 4 + .../statistics/impl/IOStatisticsBinding.java | 13 +++ .../impl/IOStatisticsStoreImpl.java | 16 ++- .../impl/PairedDurationTrackerFactory.java | 83 ++++++++++++++++ ...bstractContractStreamIOStatisticsTest.java | 24 ++--- .../fs/statistics/IOStatisticAssertions.java | 98 ++++++++++++------- .../fs/statistics/TestDurationTracking.java | 2 +- .../statistics/TestDynamicIOStatistics.java | 30 +++--- .../fs/statistics/TestEmptyIOStatistics.java | 14 +-- .../statistics/TestIOStatisticsSnapshot.java | 22 ++--- .../fs/statistics/TestIOStatisticsStore.java | 44 ++++----- 11 files changed, 248 insertions(+), 102 deletions(-) create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/PairedDurationTrackerFactory.java 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 index 37da830512497..a446c1d390136 100644 --- 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 @@ -129,6 +129,10 @@ public final class StreamStatisticNames { public static final String STREAM_READ_TOTAL_BYTES = "stream_read_total_bytes"; + /** {@value}. */ + public static final String STREAM_READ_UNBUFFERED + = "stream_read_unbuffered"; + /** {@value}. */ public static final String STREAM_WRITE_EXCEPTIONS = "stream_write_exceptions"; 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 index 9c1d63f0f67b3..c906674cf66a7 100644 --- 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 @@ -518,4 +518,17 @@ private static DurationTracker createTracker( ? factory.trackDuration(statistic) : STUB_DURATION_TRACKER; } + + /** + * Create a DurationTrackerFactory which aggregates the tracking + * of two other factories. + * @param local local tracker factory + * @param global global tracker factory + * @return a factory + */ + public static DurationTrackerFactory pairedTrackerFactory( + final DurationTrackerFactory local, + final DurationTrackerFactory global) { + return new PairedDurationTrackerFactory(local, global); +} } 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 index 80650e56aa2a0..f96bd0a62ba2a 100644 --- 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 @@ -33,6 +33,7 @@ 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; @@ -430,8 +431,21 @@ 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 int count) { - return new StatisticDurationTracker(this, key, 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..b9f0b1b23788d --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/PairedDurationTrackerFactory.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics.impl; + +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 int 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 globalDuration; + private final DurationTracker localDuration; + + private PairedDurationTracker( + final DurationTracker globalDuration, + final DurationTracker localDuration) { + this.globalDuration = globalDuration; + this.localDuration = localDuration; + } + + @Override + public void failed() { + globalDuration.failed(); + localDuration.failed(); + } + + @Override + public void close() { + globalDuration.close(); + localDuration.close(); + } + } + +} 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 index ad11859e1e9a9..196239adbeaeb 100644 --- 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 @@ -34,7 +34,7 @@ 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.verifyCounterStatisticValue; +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.StreamStatisticNames.STREAM_READ_BYTES; import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_WRITE_BYTES; @@ -94,9 +94,9 @@ public void testWriteSingleByte() throws Throwable { try (FSDataOutputStream out = fs.create(path, true)) { IOStatistics statistics = extractStatistics(out); // before a write, no bytes - verifyCounterStatisticValue(statistics, STREAM_WRITE_BYTES, 0); + verifyStatisticCounterValue(statistics, STREAM_WRITE_BYTES, 0); out.write('0'); - verifyCounterStatisticValue(statistics, STREAM_WRITE_BYTES, + verifyStatisticCounterValue(statistics, STREAM_WRITE_BYTES, writesInBlocks ? 0 : 1); // close the stream out.close(); @@ -105,7 +105,7 @@ public void testWriteSingleByte() throws Throwable { statistics = extractStatistics(out); final String strVal = statistics.toString(); LOG.info("Statistics = {}", strVal); - verifyCounterStatisticValue(statistics, STREAM_WRITE_BYTES, 1); + verifyStatisticCounterValue(statistics, STREAM_WRITE_BYTES, 1); } finally { fs.delete(path, false); } @@ -129,11 +129,11 @@ public void testWriteByteArrays() throws Throwable { out.flush(); LOG.info("stats {}", demandStatsString); IOStatistics statistics = extractStatistics(out); - verifyCounterStatisticValue(statistics, STREAM_WRITE_BYTES, + verifyStatisticCounterValue(statistics, STREAM_WRITE_BYTES, writesInBlocks ? 0 : len); out.write(bytes); out.flush(); - verifyCounterStatisticValue(statistics, STREAM_WRITE_BYTES, + verifyStatisticCounterValue(statistics, STREAM_WRITE_BYTES, writesInBlocks ? 0 : len * 2); // close the stream out.close(); @@ -141,7 +141,7 @@ public void testWriteByteArrays() throws Throwable { // statistics are still valid after the close // always call the output stream to check that behavior statistics = extractStatistics(out); - verifyCounterStatisticValue(statistics, STREAM_WRITE_BYTES, len * 2); + 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)); @@ -165,7 +165,7 @@ public void testInputStreamStatisticKeys() throws Throwable { Assertions.assertThat(keys) .describedAs("Statistics supported by the stream %s", in) .contains(STREAM_READ_BYTES); - verifyCounterStatisticValue(statistics, STREAM_READ_BYTES, 0); + verifyStatisticCounterValue(statistics, STREAM_READ_BYTES, 0); } finally { fs.delete(path, false); } @@ -183,7 +183,7 @@ public void testInputStreamStatisticRead() throws Throwable { try (FSDataInputStream in = fs.open(path)) { long current = 0; IOStatistics statistics = extractStatistics(in); - verifyCounterStatisticValue(statistics, STREAM_READ_BYTES, 0); + 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 @@ -216,12 +216,12 @@ public void testInputStreamStatisticRead() throws Throwable { Assertions.assertThat(in.read(buf128)) .describedAs("Read overlapping EOF") .isEqualTo(sublen); - current = verifyCounterStatisticValue(statistics, STREAM_READ_BYTES, + current = verifyStatisticCounterValue(statistics, STREAM_READ_BYTES, current + sublen); Assertions.assertThat(in.read(pos, buf128, 0, bufferLen)) .describedAs("Read(buffer) overlapping EOF") .isEqualTo(sublen); - verifyCounterStatisticValue(statistics, STREAM_READ_BYTES, + verifyStatisticCounterValue(statistics, STREAM_READ_BYTES, current + sublen); } } finally { @@ -248,7 +248,7 @@ public long verifyBytesRead(final IOStatistics statistics, // plus the current buffer, multiplied by that buffer size expected = bufferSize * (1 + (current / bufferSize)); } - verifyCounterStatisticValue(statistics, STREAM_READ_BYTES, expected); + verifyStatisticCounterValue(statistics, STREAM_READ_BYTES, expected); return finalPos; } 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 index caf01b3db52de..f9fc375a72431 100644 --- 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 @@ -66,7 +66,23 @@ private IOStatisticAssertions() { public static long lookupCounterStatistic( final IOStatistics stats, final String key) { - return lookupStatistic(COUNTER, key, stats.counters()); + 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; } /** @@ -78,7 +94,8 @@ public static long lookupCounterStatistic( public static long lookupGaugeStatistic( final IOStatistics stats, final String key) { - return lookupStatistic(GAUGE, key, stats.gauges()); + return lookupStatistic(GAUGE, key, + verifyStatisticsNotNull(stats).gauges()); } /** @@ -90,7 +107,8 @@ public static long lookupGaugeStatistic( public static long lookupMaximumStatistic( final IOStatistics stats, final String key) { - return lookupStatistic(MAXIMUM, key, stats.maximums()); + return lookupStatistic(MAXIMUM, key, + verifyStatisticsNotNull(stats).maximums()); } /** @@ -102,7 +120,8 @@ public static long lookupMaximumStatistic( public static long lookupMinimumStatistic( final IOStatistics stats, final String key) { - return lookupStatistic(MINIMUM, key, stats.minimums()); + return lookupStatistic(MINIMUM, key, + verifyStatisticsNotNull(stats).minimums()); } /** @@ -114,7 +133,8 @@ public static long lookupMinimumStatistic( public static MeanStatistic lookupMeanStatistic( final IOStatistics stats, final String key) { - return lookupStatistic(MEAN, key, stats.meanStatistics()); + return lookupStatistic(MEAN, key, + verifyStatisticsNotNull(stats).meanStatistics()); } /** @@ -143,11 +163,12 @@ private static E lookupStatistic( * @param value expected value. * @return the value (which always equals the expected value) */ - public static long verifyCounterStatisticValue( + public static long verifyStatisticCounterValue( final IOStatistics stats, final String key, final long value) { - return verifyStatisticValue(COUNTER, key, stats.counters(), value); + return verifyStatisticValue(COUNTER, key, + verifyStatisticsNotNull(stats).counters(), value); } /** @@ -157,11 +178,12 @@ public static long verifyCounterStatisticValue( * @param value expected value. * @return the value (which always equals the expected value) */ - public static long verifyGaugeStatisticValue( + public static long verifyStatisticGaugeValue( final IOStatistics stats, final String key, final long value) { - return verifyStatisticValue(GAUGE, key, stats.gauges(), value); + return verifyStatisticValue(GAUGE, key, + verifyStatisticsNotNull(stats).gauges(), value); } /** @@ -171,11 +193,12 @@ public static long verifyGaugeStatisticValue( * @param value expected value. * @return the value (which always equals the expected value) */ - public static long verifyMaximumStatisticValue( + public static long verifyStatisticMaximumValue( final IOStatistics stats, final String key, final long value) { - return verifyStatisticValue(MAXIMUM, key, stats.maximums(), value); + return verifyStatisticValue(MAXIMUM, key, + verifyStatisticsNotNull(stats).maximums(), value); } /** @@ -185,11 +208,12 @@ public static long verifyMaximumStatisticValue( * @param value expected value. * @return the value (which always equals the expected value) */ - public static long verifyMinimumStatisticValue( + public static long verifyStatisticMinimumValue( final IOStatistics stats, final String key, final long value) { - return verifyStatisticValue(MINIMUM, key, stats.minimums(), value); + return verifyStatisticValue(MINIMUM, key, + verifyStatisticsNotNull(stats).minimums(), value); } /** @@ -199,11 +223,12 @@ public static long verifyMinimumStatisticValue( * @param value expected value. * @return the value (which always equals the expected value) */ - public static MeanStatistic verifyMeanStatisticValue( + public static MeanStatistic verifyStatisticMeanValue( final IOStatistics stats, final String key, final MeanStatistic value) { - return verifyStatisticValue(MEAN, key, stats.meanStatistics(), value); + return verifyStatisticValue(MEAN, key, + verifyStatisticsNotNull(stats).meanStatistics(), value); } /** @@ -254,7 +279,7 @@ private static ObjectAssert assertThatStatistic( * @param map map to look up * @return an ongoing assertion */ - private static AbstractLongAssert assertThatLongStatistic( + private static AbstractLongAssert assertThatStatisticLong( final String type, final String key, final Map map) { @@ -270,10 +295,11 @@ private static AbstractLongAssert assertThatLongStatistic( * @param key statistic key * @return an ongoing assertion */ - public static AbstractLongAssert assertThatCounterStatistic( + public static AbstractLongAssert assertThatStatisticCounter( final IOStatistics stats, final String key) { - return assertThatLongStatistic(COUNTER, key, stats.counters()); + return assertThatStatisticLong(COUNTER, key, + verifyStatisticsNotNull(stats).counters()); } /** @@ -283,10 +309,11 @@ public static AbstractLongAssert assertThatCounterStatistic( * @param key statistic key * @return an ongoing assertion */ - public static AbstractLongAssert assertThatGaugeStatistic( + public static AbstractLongAssert assertThatStatisticGauge( final IOStatistics stats, final String key) { - return assertThatLongStatistic(GAUGE, key, stats.gauges()); + return assertThatStatisticLong(GAUGE, key, + verifyStatisticsNotNull(stats).gauges()); } /** @@ -296,10 +323,11 @@ public static AbstractLongAssert assertThatGaugeStatistic( * @param key statistic key * @return an ongoing assertion */ - public static AbstractLongAssert assertThatMinimumStatistic( + public static AbstractLongAssert assertThatStatisticMinimum( final IOStatistics stats, final String key) { - return assertThatLongStatistic(MINIMUM, key, stats.minimums()); + return assertThatStatisticLong(MINIMUM, key, + verifyStatisticsNotNull(stats).minimums()); } /** @@ -309,10 +337,11 @@ public static AbstractLongAssert assertThatMinimumStatistic( * @param key statistic key * @return an ongoing assertion */ - public static AbstractLongAssert assertThatMaximumStatistic( + public static AbstractLongAssert assertThatStatisticMaximum( final IOStatistics stats, final String key) { - return assertThatLongStatistic(MAXIMUM, key, stats.maximums()); + return assertThatStatisticLong(MAXIMUM, key, + verifyStatisticsNotNull(stats).maximums()); } /** @@ -322,10 +351,11 @@ public static AbstractLongAssert assertThatMaximumStatistic( * @param key statistic key * @return an ongoing assertion */ - public static ObjectAssert assertThatMeanStatistic( + public static ObjectAssert assertThatStatisticMean( final IOStatistics stats, final String key) { - return assertThatStatistic(MEAN, key, stats.meanStatistics()); + return assertThatStatistic(MEAN, key, + verifyStatisticsNotNull(stats).meanStatistics()); } /** @@ -336,12 +366,12 @@ public static ObjectAssert assertThatMeanStatistic( * @param key statistic key * @return an ongoing assertion */ - public static ObjectAssert assertThatMeanStatisticMatches( + public static ObjectAssert assertThatStatisticMeanMatches( final IOStatistics stats, final String key, final long samples, final long sum) { - return assertThatMeanStatistic(stats, key) + return assertThatStatisticMean(stats, key) .matches(p -> (p.getSamples() == samples), "samples == " + samples) .matches(p -> (p.getSum() == sum), @@ -385,10 +415,11 @@ private static void assertTracked(final IOStatistics stats, * @param stats statistics source * @param key statistic key */ - public static void assertCounterStatisticIsTracked( + public static void assertStatisticCounterIsTracked( final IOStatistics stats, final String key) { - assertTracked(stats, COUNTER, key, stats.counters()); + assertTracked(stats, COUNTER, key, + verifyStatisticsNotNull(stats).counters()); } /** @@ -396,10 +427,11 @@ public static void assertCounterStatisticIsTracked( * @param stats statistics source * @param key statistic key */ - public static void assertCounterStatisticIsUntracked( + public static void assertStatisticCounterIsUntracked( final IOStatistics stats, final String key) { - assertUntracked(stats, COUNTER, key, stats.counters()); + assertUntracked(stats, COUNTER, key, + verifyStatisticsNotNull(stats).counters()); } /** @@ -448,7 +480,7 @@ private static IOStatistics extractStatistics( * @param stat statistic * @return the deserialized version. */ - public static IOStatistics javaRoundTrip(final IOStatistics stat) + public static IOStatistics statisticsJavaRoundTrip(final IOStatistics stat) throws IOException, ClassNotFoundException { assertThat(stat).isInstanceOf(Serializable.class); ByteArrayOutputStream baos = new ByteArrayOutputStream(1024); 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 index 0ff89efbf0515..471e97db9d460 100644 --- 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 @@ -76,7 +76,7 @@ public void teardown() { public void testDurationTryWithResources() throws Throwable { DurationTracker tracker = stats.trackDuration(REQUESTS); - verifyCounterStatisticValue(stats, REQUESTS, 1L); + verifyStatisticCounterValue(stats, REQUESTS, 1L); sleep(); tracker.close(); try (DurationTracker ignored = 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 index 6760ac5378050..9b929ac82ff11 100644 --- 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 @@ -34,9 +34,9 @@ import org.apache.hadoop.metrics2.lib.MutableCounterLong; import org.apache.hadoop.test.AbstractHadoopTestBase; -import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertCounterStatisticIsTracked; -import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertCounterStatisticIsUntracked; -import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyCounterStatisticValue; +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; @@ -101,9 +101,9 @@ public void setUp() throws Exception { */ @Test public void testEval() throws Throwable { - verifyCounterStatisticValue(statistics, EVAL, 0); + verifyStatisticCounterValue(statistics, EVAL, 0); evalLong = 10; - verifyCounterStatisticValue(statistics, EVAL, 10); + verifyStatisticCounterValue(statistics, EVAL, 10); } /** @@ -111,9 +111,9 @@ public void testEval() throws Throwable { */ @Test public void testAlong() throws Throwable { - verifyCounterStatisticValue(statistics, ALONG, 0); + verifyStatisticCounterValue(statistics, ALONG, 0); aLong.addAndGet(1); - verifyCounterStatisticValue(statistics, ALONG, 1); + verifyStatisticCounterValue(statistics, ALONG, 1); } /** @@ -121,9 +121,9 @@ public void testAlong() throws Throwable { */ @Test public void testAint() throws Throwable { - verifyCounterStatisticValue(statistics, AINT, 0); + verifyStatisticCounterValue(statistics, AINT, 0); aInt.addAndGet(1); - verifyCounterStatisticValue(statistics, AINT, 1); + verifyStatisticCounterValue(statistics, AINT, 1); } /** @@ -131,9 +131,9 @@ public void testAint() throws Throwable { */ @Test public void testCounter() throws Throwable { - verifyCounterStatisticValue(statistics, COUNT, 0); + verifyStatisticCounterValue(statistics, COUNT, 0); counter.incr(); - verifyCounterStatisticValue(statistics, COUNT, 1); + verifyStatisticCounterValue(statistics, COUNT, 1); } /** @@ -178,7 +178,7 @@ public void testIteratorIsSnapshot() throws Throwable { @Test public void testUnknownStatistic() throws Throwable { - assertCounterStatisticIsUntracked(statistics, "anything"); + assertStatisticCounterIsUntracked(statistics, "anything"); } @Test @@ -186,7 +186,7 @@ public void testStatisticsTrackedAssertion() throws Throwable { // expect an exception to be raised when an assertion // is made that an unknown statistic is tracked,. assertThatThrownBy(() -> - assertCounterStatisticIsTracked(statistics, "anything")) + assertStatisticCounterIsTracked(statistics, "anything")) .isInstanceOf(AssertionError.class); } @@ -195,7 +195,7 @@ public void testStatisticsValueAssertion() throws Throwable { // expect an exception to be raised when // an assertion is made about the value of an unknown statistics assertThatThrownBy(() -> - verifyCounterStatisticValue(statistics, "anything", 0)) + verifyStatisticCounterValue(statistics, "anything", 0)) .isInstanceOf(AssertionError.class); } @@ -207,7 +207,7 @@ public void testSerDeser() throws Throwable { incrementAllCounters(); IOStatistics stat = IOStatisticsSupport.snapshotIOStatistics(statistics); incrementAllCounters(); - IOStatistics deser = IOStatisticAssertions.javaRoundTrip(stat); + IOStatistics deser = IOStatisticAssertions.statisticsJavaRoundTrip(stat); assertThat(deser.counters().keySet()) .containsExactlyInAnyOrder(KEYS); for (Map.Entry e : deser.counters().entrySet()) { 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 index f8c1d7a1dc226..296470abaa9bf 100644 --- 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 @@ -23,9 +23,9 @@ import org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding; import org.apache.hadoop.test.AbstractHadoopTestBase; -import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertCounterStatisticIsTracked; -import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertCounterStatisticIsUntracked; -import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyCounterStatisticValue; +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; @@ -40,7 +40,7 @@ public class TestEmptyIOStatistics extends AbstractHadoopTestBase { @Test public void testUnknownStatistic() throws Throwable { - assertCounterStatisticIsUntracked(empty, "anything"); + assertStatisticCounterIsUntracked(empty, "anything"); } @Test @@ -48,7 +48,7 @@ public void testStatisticsTrackedAssertion() throws Throwable { // expect an exception to be raised when an assertion // is made that an unknown statistic is tracked,. assertThatThrownBy(() -> - assertCounterStatisticIsTracked(empty, "anything")) + assertStatisticCounterIsTracked(empty, "anything")) .isInstanceOf(AssertionError.class); } @@ -57,7 +57,7 @@ public void testStatisticsValueAssertion() throws Throwable { // expect an exception to be raised when // an assertion is made about the value of an unknown statistics assertThatThrownBy(() -> - verifyCounterStatisticValue(empty, "anything", 0)) + verifyStatisticCounterValue(empty, "anything", 0)) .isInstanceOf(AssertionError.class); } @@ -67,7 +67,7 @@ public void testEmptySnapshot() throws Throwable { assertThat(stat.counters().keySet()) .describedAs("keys of snapshot") .isEmpty(); - IOStatistics deser = IOStatisticAssertions.javaRoundTrip(stat); + IOStatistics deser = IOStatisticAssertions.statisticsJavaRoundTrip(stat); assertThat(deser.counters().keySet()) .describedAs("keys of deserialized snapshot") .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 index 3cd52ba066f51..41e9bffefe834 100644 --- 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 @@ -67,10 +67,10 @@ public void setup() throws Exception { @Test public void testTrackedValues() throws Throwable { - verifyCounterStatisticValue(snapshot, "c1", 0L); - verifyGaugeStatisticValue(snapshot, "g1", 1L); - verifyMinimumStatisticValue(snapshot, "m1", -1L); - verifyMeanStatisticValue(snapshot, "mean0", + verifyStatisticCounterValue(snapshot, "c1", 0L); + verifyStatisticGaugeValue(snapshot, "g1", 1L); + verifyStatisticMinimumValue(snapshot, "m1", -1L); + verifyStatisticMeanValue(snapshot, "mean0", new MeanStatistic(0, 1)); } @@ -79,7 +79,7 @@ public void testStatisticsValueAssertion() throws Throwable { // expect an exception to be raised when // an assertion is made about the value of an unknown statistics assertThatThrownBy(() -> - verifyCounterStatisticValue(snapshot, "anything", 0)) + verifyStatisticCounterValue(snapshot, "anything", 0)) .isInstanceOf(AssertionError.class); } @@ -126,19 +126,19 @@ public void testJsonRoundTrip() throws Throwable { public void verifyDeserializedInstance( final IOStatistics deser) { LOG.info("deserialized {}", deser); - verifyCounterStatisticValue(deser, "c1", 0L); - verifyGaugeStatisticValue(deser, "g1", 1L); - verifyMinimumStatisticValue(deser, "m1", -1L); - verifyMeanStatisticValue(deser, "mean0", + verifyStatisticCounterValue(deser, "c1", 0L); + verifyStatisticGaugeValue(deser, "g1", 1L); + verifyStatisticMinimumValue(deser, "m1", -1L); + verifyStatisticMeanValue(deser, "mean0", new MeanStatistic(0, 1)); - verifyMeanStatisticValue(deser, "mean1", + verifyStatisticMeanValue(deser, "mean1", snapshot.meanStatistics().get("mean1")); } @Test public void testJavaRoundTrip() throws Throwable { verifyDeserializedInstance( - IOStatisticAssertions.javaRoundTrip( + 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 index c3de165215dac..778eab8315aa5 100644 --- 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 @@ -29,11 +29,11 @@ import org.apache.hadoop.test.AbstractHadoopTestBase; import org.apache.hadoop.util.JsonSerialization; -import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatMeanStatisticMatches; -import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyCounterStatisticValue; -import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyGaugeStatisticValue; -import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyMaximumStatisticValue; -import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyMinimumStatisticValue; +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; @@ -82,14 +82,14 @@ public void teardown() { @Test public void testGauges() throws Throwable { stats.setGauge(GAUGE, 1); - verifyGaugeStatisticValue(stats, GAUGE, 1); + verifyStatisticGaugeValue(stats, GAUGE, 1); stats.incrementGauge(GAUGE, 1); - verifyGaugeStatisticValue(stats, GAUGE, 2); + verifyStatisticGaugeValue(stats, GAUGE, 2); stats.setGauge(GAUGE, -1); - verifyGaugeStatisticValue(stats, GAUGE, -1); + verifyStatisticGaugeValue(stats, GAUGE, -1); Assertions.assertThat(stats.incrementGauge(GAUGE, -1)) .isEqualTo(-2); - verifyGaugeStatisticValue(stats, GAUGE, -2); + verifyStatisticGaugeValue(stats, GAUGE, -2); Assertions.assertThat(stats.getGaugeReference(GAUGE).get()) .isEqualTo(-2); stats.setGauge(UNKNOWN, 1); @@ -100,13 +100,13 @@ public void testGauges() throws Throwable { @Test public void testMinimums() throws Throwable { stats.setMinimum(MIN, 100); - verifyMinimumStatisticValue(stats, MIN, 100); + verifyStatisticMinimumValue(stats, MIN, 100); stats.setMinimum(MIN, 100); // will do nothing as it is higher stats.addMinimumSample(MIN, 200); - verifyMinimumStatisticValue(stats, MIN, 100); + verifyStatisticMinimumValue(stats, MIN, 100); stats.addMinimumSample(MIN, 10); - verifyMinimumStatisticValue(stats, MIN, 10); + verifyStatisticMinimumValue(stats, MIN, 10); stats.setMinimum(UNKNOWN, 100); stats.addMinimumSample(UNKNOWN, 200); } @@ -114,12 +114,12 @@ public void testMinimums() throws Throwable { @Test public void testMaximums() throws Throwable { stats.setMaximum(MAX, 100); - verifyMaximumStatisticValue(stats, MAX, 100); + verifyStatisticMaximumValue(stats, MAX, 100); stats.setMaximum(MAX, 100); stats.addMaximumSample(MAX, 200); - verifyMaximumStatisticValue(stats, MAX, 200); + verifyStatisticMaximumValue(stats, MAX, 200); stats.addMaximumSample(MAX, 10); - verifyMaximumStatisticValue(stats, MAX, 200); + verifyStatisticMaximumValue(stats, MAX, 200); stats.setMaximum(UNKNOWN, 100); stats.addMaximumSample(UNKNOWN, 200); } @@ -129,10 +129,10 @@ public void testMeans() throws Throwable { stats.setMeanStatistic(MEAN, new MeanStatistic(1, 1)); - assertThatMeanStatisticMatches(stats, MEAN, 1, 1) + assertThatStatisticMeanMatches(stats, MEAN, 1, 1) .matches(p -> p.mean() == 1, "mean"); stats.addMeanStatisticSample(MEAN, 9); - assertThatMeanStatisticMatches(stats, MEAN, 2, 10) + assertThatStatisticMeanMatches(stats, MEAN, 2, 10) .matches(p -> p.mean() == 5, "mean"); } @@ -151,11 +151,11 @@ public void testRoundTrip() throws Throwable { LOG.info("serialized form\n{}", json); IOStatisticsSnapshot deser = serializer.fromJson(json); LOG.info("deserialized {}", deser); - verifyCounterStatisticValue(deser, COUNT, 1L); - verifyGaugeStatisticValue(deser, GAUGE, -1); - verifyMaximumStatisticValue(deser, MAX, 200); - verifyMinimumStatisticValue(deser, MIN, -100); - assertThatMeanStatisticMatches(deser, MEAN, 2, 10) + 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"); } From 050a1c906df90474035ad602cb51d25b184cab96 Mon Sep 17 00:00:00 2001 From: Mehakmeet Singh Date: Wed, 30 Sep 2020 17:46:48 +0530 Subject: [PATCH 12/16] HADOOP-17272. AbfsOutputStream support IOStatistics --- .../hadoop/fs/azurebfs/AbfsStatistic.java | 19 ++- .../azurebfs/services/AbfsOutputStream.java | 55 +++++--- .../services/AbfsOutputStreamStatistics.java | 13 +- .../AbfsOutputStreamStatisticsImpl.java | 128 ++++++++++-------- .../ITestAbfsOutputStreamStatistics.java | 32 +++++ .../TestAbfsOutputStreamStatistics.java | 28 ++-- 6 files changed, 175 insertions(+), 100 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsStatistic.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsStatistic.java index 2935cd754315d..743ef53105a88 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsStatistic.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsStatistic.java @@ -73,7 +73,24 @@ public enum AbfsStatistic { READ_THROTTLES("read_throttles", "Total number of times a read operation is throttled."), WRITE_THROTTLES("write_throttles", - "Total number of times a write operation is throttled."); + "Total number of times a write operation is throttled."), + + //OutputStream statistics. + BYTES_TO_UPLOAD("bytes_upload", + "Total bytes to upload."), + BYTES_UPLOAD_SUCCESSFUL("bytes_upload_successfully", + "Total bytes uploaded successfully."), + BYTES_UPLOAD_FAILED("bytes_upload_failed", + "Total bytes failed to upload."), + TIME_SPENT_ON_TASK_WAIT("time_spent_task_wait", + "Total time spent on waiting for a task."), + QUEUE_SHRUNK_OPS("queue_shrunk_ops", + "Total number of times blocking queue was shrunk."), + WRITE_CURRENT_BUFFER_OPERATIONS("write_current_buffer_ops", + "Total number of times the current buffer is written to the service."), + TIME_TAKEN_TO_PUT_REQUEST("time_taken_to_put_request", + "Total time taken to complete a put request."); + private String statName; private String statDescription; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index 1991638a66703..5683b5c7b4f08 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -29,7 +29,6 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ExecutorCompletionService; import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.Callable; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; @@ -38,9 +37,14 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.fs.azurebfs.AbfsStatistic; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.utils.CachedSASToken; +import org.apache.hadoop.fs.statistics.DurationTracker; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; import org.apache.hadoop.io.ElasticByteBufferPool; import org.apache.hadoop.fs.FileSystem.Statistics; import org.apache.hadoop.fs.FSExceptionMessages; @@ -90,6 +94,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable, StreamCa private final Statistics statistics; private final AbfsOutputStreamStatistics outputStreamStatistics; + private final IOStatistics ioStatistics; private static final Logger LOG = LoggerFactory.getLogger(AbfsOutputStream.class); @@ -134,6 +139,7 @@ public AbfsOutputStream( this.completionService = new ExecutorCompletionService<>(this.threadExecutor); this.cachedSasToken = new CachedSASToken( abfsOutputStreamContext.getSasTokenRenewPeriodForStreamsInSeconds()); + ioStatistics = outputStreamStatistics.getIOStatistics(); } /** @@ -376,27 +382,31 @@ private synchronized void writeCurrentBufferToService() throws IOException { position += bytesLength; if (threadExecutor.getQueue().size() >= maxRequestsThatCanBeQueued) { - long start = System.currentTimeMillis(); - waitForTaskToComplete(); - outputStreamStatistics.timeSpentTaskWait(start, System.currentTimeMillis()); + //Tracking time spent on waiting for task to complete. + try (DurationTracker ignored = outputStreamStatistics.timeSpentTaskWait()) { + waitForTaskToComplete(); + } } - final Future job = completionService.submit(new Callable() { - @Override - public Void call() throws Exception { - AbfsPerfTracker tracker = client.getAbfsPerfTracker(); - try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker, - "writeCurrentBufferToService", "append")) { - AbfsRestOperation op = client.append(path, offset, bytes, 0, - bytesLength, cachedSasToken.get(), false); - cachedSasToken.update(op.getSasToken()); - perfInfo.registerResult(op.getResult()); - byteBufferPool.putBuffer(ByteBuffer.wrap(bytes)); - perfInfo.registerSuccess(true); - return null; - } - } - }); + final Future job = + completionService.submit(IOStatisticsBinding + .trackDurationOfCallable((IOStatisticsStore) ioStatistics, + AbfsStatistic.TIME_TAKEN_TO_PUT_REQUEST.getStatName(), + () -> { + AbfsPerfTracker tracker = client.getAbfsPerfTracker(); + try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker, + "writeCurrentBufferToService", "append")) { + AbfsRestOperation op = + client.append(path, offset, bytes, 0, + bytesLength, cachedSasToken.get(), false); + cachedSasToken.update(op.getSasToken()); + perfInfo.registerResult(op.getResult()); + byteBufferPool.putBuffer(ByteBuffer.wrap(bytes)); + perfInfo.registerSuccess(true); + return null; + } + }) + ); if (job.isCancelled()) { outputStreamStatistics.uploadFailed(bytesLength); @@ -557,6 +567,11 @@ int getMaxRequestsThatCanBeQueued() { return maxRequestsThatCanBeQueued; } + @VisibleForTesting + public IOStatistics getIoStatistics() { + return ioStatistics; + } + /** * Appending AbfsOutputStream statistics to base toString(). * diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatistics.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatistics.java index c9fe0dd45525d..2c2ddc0e786f9 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatistics.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatistics.java @@ -19,6 +19,8 @@ package org.apache.hadoop.fs.azurebfs.services; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.statistics.DurationTracker; +import org.apache.hadoop.fs.statistics.IOStatistics; /** * Interface for {@link AbfsOutputStream} statistics. @@ -49,11 +51,8 @@ public interface AbfsOutputStreamStatistics { /** * Time spent in waiting for tasks to be completed in the blocking queue. - * - * @param start millisecond at which the wait for task to be complete begins. - * @param end millisecond at which the wait is completed for the task. */ - void timeSpentTaskWait(long start, long end); + DurationTracker timeSpentTaskWait(); /** * Number of times task queue is shrunk. @@ -65,6 +64,12 @@ public interface AbfsOutputStreamStatistics { */ void writeCurrentBuffer(); + /** + * Get the IOStatisticsStore instance from AbfsOutputStreamStatistics. + * @return instance of IOStatisticsStore which extends IOStatistics. + */ + IOStatistics getIOStatistics(); + /** * Method to form a string of all AbfsOutputStream statistics and their * values. diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatisticsImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatisticsImpl.java index cd5a29e217ce5..c362914e75c3e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatisticsImpl.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatisticsImpl.java @@ -18,32 +18,37 @@ package org.apache.hadoop.fs.azurebfs.services; +import com.google.common.annotations.VisibleForTesting; + +import org.apache.hadoop.fs.azurebfs.AbfsStatistic; +import org.apache.hadoop.fs.statistics.DurationTracker; +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.fs.statistics.impl.IOStatisticsStore; + +import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.*; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore; + /** * OutputStream statistics implementation for Abfs. */ public class AbfsOutputStreamStatisticsImpl - implements AbfsOutputStreamStatistics { - private long bytesToUpload; - private long bytesUploadSuccessful; - private long bytesUploadFailed; - /** - * Counter to get the total time spent while waiting for tasks to complete - * in the blocking queue inside the thread executor. - */ - private long timeSpentOnTaskWait; - /** - * Counter to get the total number of queue shrink operations done {@code - * AbfsOutputStream#shrinkWriteOperationQueue()} by AbfsOutputStream to - * remove the write operations which were successfully done by - * AbfsOutputStream from the task queue. - */ - private long queueShrunkOps; - /** - * Counter to get the total number of times the current buffer is written - * to the service {@code AbfsOutputStream#writeCurrentBufferToService()} via - * AbfsClient and appended to the data store by AbfsRestOperation. - */ - private long writeCurrentBufferOperations; + implements AbfsOutputStreamStatistics, IOStatisticsSource { + + private final IOStatisticsStore ioStatisticsStore = iostatisticsStore() + .withCounters( + getStatName(BYTES_TO_UPLOAD), + getStatName(BYTES_UPLOAD_SUCCESSFUL), + getStatName(BYTES_UPLOAD_FAILED), + getStatName(QUEUE_SHRUNK_OPS), + getStatName(WRITE_CURRENT_BUFFER_OPERATIONS) + ) + .withDurationTracking( + getStatName(TIME_TAKEN_TO_PUT_REQUEST), + getStatName(TIME_SPENT_ON_TASK_WAIT) + ) + .build(); /** * Records the need to upload bytes and increments the total bytes that @@ -53,9 +58,7 @@ public class AbfsOutputStreamStatisticsImpl */ @Override public void bytesToUpload(long bytes) { - if (bytes > 0) { - bytesToUpload += bytes; - } + ioStatisticsStore.incrementCounter(getStatName(BYTES_TO_UPLOAD), bytes); } /** @@ -66,9 +69,7 @@ public void bytesToUpload(long bytes) { */ @Override public void uploadSuccessful(long bytes) { - if (bytes > 0) { - bytesUploadSuccessful += bytes; - } + ioStatisticsStore.incrementCounter(getStatName(BYTES_UPLOAD_SUCCESSFUL), bytes); } /** @@ -78,9 +79,7 @@ public void uploadSuccessful(long bytes) { */ @Override public void uploadFailed(long bytes) { - if (bytes > 0) { - bytesUploadFailed += bytes; - } + ioStatisticsStore.incrementCounter(getStatName(BYTES_UPLOAD_FAILED), bytes); } /** @@ -96,14 +95,10 @@ public void uploadFailed(long bytes) { * This time spent while waiting for the task to be completed is being * recorded in this counter. * - * @param startTime time(in milliseconds) before the wait for task to be - * completed is begin. - * @param endTime time(in milliseconds) after the wait for the task to be - * completed is done. */ @Override - public void timeSpentTaskWait(long startTime, long endTime) { - timeSpentOnTaskWait += endTime - startTime; + public DurationTracker timeSpentTaskWait() { + return ioStatisticsStore.trackDuration(getStatName(TIME_SPENT_ON_TASK_WAIT)); } /** @@ -114,7 +109,7 @@ public void timeSpentTaskWait(long startTime, long endTime) { */ @Override public void queueShrunk() { - queueShrunkOps++; + ioStatisticsStore.incrementCounter(getStatName(QUEUE_SHRUNK_OPS)); } /** @@ -125,31 +120,63 @@ public void queueShrunk() { */ @Override public void writeCurrentBuffer() { - writeCurrentBufferOperations++; + ioStatisticsStore.incrementCounter(getStatName(WRITE_CURRENT_BUFFER_OPERATIONS)); + } + + /** + * {@inheritDoc} + * + * A getter for IOStatisticsStore instance which extends IOStatistics. + * + * @return IOStatisticsStore instance. + */ + @Override + public IOStatistics getIOStatistics() { + return ioStatisticsStore; } + @VisibleForTesting public long getBytesToUpload() { - return bytesToUpload; + return ioStatisticsStore.getCounterReference(getStatName(BYTES_TO_UPLOAD)).get(); } + @VisibleForTesting public long getBytesUploadSuccessful() { - return bytesUploadSuccessful; + return ioStatisticsStore.getCounterReference(getStatName(BYTES_UPLOAD_SUCCESSFUL)).get(); } + @VisibleForTesting public long getBytesUploadFailed() { - return bytesUploadFailed; + return ioStatisticsStore.getCounterReference(getStatName(BYTES_UPLOAD_FAILED)).get(); } + @VisibleForTesting public long getTimeSpentOnTaskWait() { - return timeSpentOnTaskWait; + return ioStatisticsStore.getCounterReference(getStatName(TIME_SPENT_ON_TASK_WAIT)).get(); } + @VisibleForTesting public long getQueueShrunkOps() { - return queueShrunkOps; + return ioStatisticsStore.getCounterReference(getStatName(QUEUE_SHRUNK_OPS)).get(); } + @VisibleForTesting public long getWriteCurrentBufferOperations() { - return writeCurrentBufferOperations; + return ioStatisticsStore.getCounterReference(getStatName(WRITE_CURRENT_BUFFER_OPERATIONS)).get(); + } + + @VisibleForTesting + public double getTimeSpentOnPutRequest() { + return ioStatisticsStore.getMeanStatistic(getStatName(TIME_TAKEN_TO_PUT_REQUEST) + ".mean").mean(); + } + + /** + * Method to get Statistic name from the enum class. + * @param statistic AbfsStatistic to get the name of. + * @return String value of AbfsStatistic name. + */ + private String getStatName(AbfsStatistic statistic) { + return statistic.getStatName(); } /** @@ -160,16 +187,7 @@ public long getWriteCurrentBufferOperations() { @Override public String toString() { final StringBuilder outputStreamStats = new StringBuilder( "OutputStream Statistics{"); - outputStreamStats.append(", bytes_upload=").append(bytesToUpload); - outputStreamStats.append(", bytes_upload_successfully=") - .append(bytesUploadSuccessful); - outputStreamStats.append(", bytes_upload_failed=") - .append(bytesUploadFailed); - outputStreamStats.append(", time_spent_task_wait=") - .append(timeSpentOnTaskWait); - outputStreamStats.append(", queue_shrunk_ops=").append(queueShrunkOps); - outputStreamStats.append(", write_current_buffer_ops=") - .append(writeCurrentBufferOperations); + outputStreamStats.append(IOStatisticsLogging.ioStatisticsSourceToString(ioStatisticsStore)); outputStreamStats.append("}"); return outputStreamStats.toString(); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStreamStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStreamStatistics.java index c8640dded3d71..14e260511d4c4 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStreamStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStreamStatistics.java @@ -20,17 +20,24 @@ import java.io.IOException; +import org.assertj.core.api.Assertions; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl; +import org.apache.hadoop.fs.statistics.IOStatistics; /** * Test AbfsOutputStream statistics. */ public class ITestAbfsOutputStreamStatistics extends AbstractAbfsIntegrationTest { + private static final Logger LOG = + LoggerFactory.getLogger(ITestAbfsOutputStreamStatistics.class); + private static final int OPERATIONS = 10; public ITestAbfsOutputStreamStatistics() throws Exception { @@ -219,6 +226,31 @@ public void testAbfsOutputStreamWriteBuffer() throws IOException { } } + /** + * Test to check correct value of time spent on a PUT request in + * AbfsOutputStream. + */ + @Test + public void testAbfsOutputStreamDurationTrackerPutRequest() throws IOException { + describe("Testing to check if DurationTracker for PUT request is working " + + "correctly."); + AzureBlobFileSystem fs = getFileSystem(); + Path pathForPutRequest = path(getMethodName()); + + try(AbfsOutputStream outputStream = + createAbfsOutputStreamWithFlushEnabled(fs, pathForPutRequest)) { + outputStream.write('a'); + outputStream.hflush(); + + AbfsOutputStreamStatisticsImpl abfsOutputStreamStatistics = + getAbfsOutputStreamStatistics(outputStream); + + Assertions.assertThat(abfsOutputStreamStatistics.getTimeSpentOnPutRequest()) + .describedAs("Mismatch in timeTakenForPutRequest DurationTracker") + .isGreaterThan(0.0); + } + } + /** * Method to get the AbfsOutputStream statistics. * diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsOutputStreamStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsOutputStreamStatistics.java index 58f00233710f8..11f317f8c2543 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsOutputStreamStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsOutputStreamStatistics.java @@ -20,6 +20,7 @@ import java.util.Random; +import org.assertj.core.api.Assertions; import org.junit.Test; import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; @@ -94,17 +95,11 @@ public void testAbfsOutputStreamTimeSpentOnWaitTask() { assertEquals("Mismatch in time spent on waiting for tasks to complete", 0, abfsOutputStreamStatistics.getTimeSpentOnTaskWait()); - int smallRandomStartTime = - new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE); - int smallRandomEndTime = - new Random().nextInt(LOW_RANGE_FOR_RANDOM_VALUE) - + smallRandomStartTime; - int smallDiff = smallRandomEndTime - smallRandomStartTime; abfsOutputStreamStatistics - .timeSpentTaskWait(smallRandomStartTime, smallRandomEndTime); - //Test for small random value of timeSpentWaitTask. + .timeSpentTaskWait(); + //Test for one op call value of timeSpentWaitTask. assertEquals("Mismatch in time spent on waiting for tasks to complete", - smallDiff, abfsOutputStreamStatistics.getTimeSpentOnTaskWait()); + 1, abfsOutputStreamStatistics.getTimeSpentOnTaskWait()); //Reset statistics for the next test. abfsOutputStreamStatistics = new AbfsOutputStreamStatisticsImpl(); @@ -113,23 +108,16 @@ public void testAbfsOutputStreamTimeSpentOnWaitTask() { * Entering multiple values for timeSpentTaskWait() to check the * summation is happening correctly. Also calculating the expected result. */ - int expectedRandomDiff = 0; for (int i = 0; i < OPERATIONS; i++) { - int largeRandomStartTime = - new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE); - int largeRandomEndTime = new Random().nextInt(HIGH_RANGE_FOR_RANDOM_VALUE) - + largeRandomStartTime; - abfsOutputStreamStatistics - .timeSpentTaskWait(largeRandomStartTime, largeRandomEndTime); - expectedRandomDiff += largeRandomEndTime - largeRandomStartTime; + abfsOutputStreamStatistics.timeSpentTaskWait(); } /* - * Test to check correct value of timeSpentTaskWait after multiple - * random values are passed in it. + * Test to check correct value of timeSpentTaskWait after OPERATIONS + * number of op calls. */ assertEquals("Mismatch in time spent on waiting for tasks to complete", - expectedRandomDiff, + OPERATIONS, abfsOutputStreamStatistics.getTimeSpentOnTaskWait()); } From 85fff7af92d08cd253d6995c559b2ef8b0bb17b3 Mon Sep 17 00:00:00 2001 From: Mehakmeet Singh Date: Thu, 1 Oct 2020 14:26:42 +0530 Subject: [PATCH 13/16] HADOOP-17272. review fix + clean-up --- .../hadoop/fs/azurebfs/AbfsStatistic.java | 4 ++-- .../fs/azurebfs/services/AbfsOutputStream.java | 10 ++++++---- .../AbfsOutputStreamStatisticsImpl.java | 17 +++++++++-------- .../ITestAbfsOutputStreamStatistics.java | 7 +------ .../TestAbfsOutputStreamStatistics.java | 1 - 5 files changed, 18 insertions(+), 21 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsStatistic.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsStatistic.java index 743ef53105a88..6c78841bd7c0c 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsStatistic.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsStatistic.java @@ -88,8 +88,8 @@ public enum AbfsStatistic { "Total number of times blocking queue was shrunk."), WRITE_CURRENT_BUFFER_OPERATIONS("write_current_buffer_ops", "Total number of times the current buffer is written to the service."), - TIME_TAKEN_TO_PUT_REQUEST("time_taken_to_put_request", - "Total time taken to complete a put request."); + TIME_SPENT_ON_PUT_REQUEST("time_spent_on_put_request", + "Total time spent on a put request."); private String statName; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index 5683b5c7b4f08..501ac89776d7c 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -43,6 +43,7 @@ import org.apache.hadoop.fs.azurebfs.utils.CachedSASToken; 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.fs.statistics.impl.IOStatisticsBinding; import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; import org.apache.hadoop.io.ElasticByteBufferPool; @@ -56,7 +57,8 @@ /** * The BlobFsOutputStream for Rest AbfsClient. */ -public class AbfsOutputStream extends OutputStream implements Syncable, StreamCapabilities { +public class AbfsOutputStream extends OutputStream implements Syncable, + StreamCapabilities, IOStatisticsSource { private final AbfsClient client; private final String path; @@ -391,7 +393,7 @@ private synchronized void writeCurrentBufferToService() throws IOException { final Future job = completionService.submit(IOStatisticsBinding .trackDurationOfCallable((IOStatisticsStore) ioStatistics, - AbfsStatistic.TIME_TAKEN_TO_PUT_REQUEST.getStatName(), + AbfsStatistic.TIME_SPENT_ON_PUT_REQUEST.getStatName(), () -> { AbfsPerfTracker tracker = client.getAbfsPerfTracker(); try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker, @@ -567,8 +569,8 @@ int getMaxRequestsThatCanBeQueued() { return maxRequestsThatCanBeQueued; } - @VisibleForTesting - public IOStatistics getIoStatistics() { + @Override + public IOStatistics getIOStatistics() { return ioStatistics; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatisticsImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatisticsImpl.java index c362914e75c3e..7cc5ccc3a6de4 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatisticsImpl.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatisticsImpl.java @@ -25,6 +25,7 @@ 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.fs.statistics.StoreStatisticNames; import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.*; @@ -45,7 +46,7 @@ public class AbfsOutputStreamStatisticsImpl getStatName(WRITE_CURRENT_BUFFER_OPERATIONS) ) .withDurationTracking( - getStatName(TIME_TAKEN_TO_PUT_REQUEST), + getStatName(TIME_SPENT_ON_PUT_REQUEST), getStatName(TIME_SPENT_ON_TASK_WAIT) ) .build(); @@ -137,37 +138,37 @@ public IOStatistics getIOStatistics() { @VisibleForTesting public long getBytesToUpload() { - return ioStatisticsStore.getCounterReference(getStatName(BYTES_TO_UPLOAD)).get(); + return ioStatisticsStore.counters().get(getStatName(BYTES_TO_UPLOAD)); } @VisibleForTesting public long getBytesUploadSuccessful() { - return ioStatisticsStore.getCounterReference(getStatName(BYTES_UPLOAD_SUCCESSFUL)).get(); + return ioStatisticsStore.counters().get(getStatName(BYTES_UPLOAD_SUCCESSFUL)); } @VisibleForTesting public long getBytesUploadFailed() { - return ioStatisticsStore.getCounterReference(getStatName(BYTES_UPLOAD_FAILED)).get(); + return ioStatisticsStore.counters().get(getStatName(BYTES_UPLOAD_FAILED)); } @VisibleForTesting public long getTimeSpentOnTaskWait() { - return ioStatisticsStore.getCounterReference(getStatName(TIME_SPENT_ON_TASK_WAIT)).get(); + return ioStatisticsStore.counters().get(getStatName(TIME_SPENT_ON_TASK_WAIT)); } @VisibleForTesting public long getQueueShrunkOps() { - return ioStatisticsStore.getCounterReference(getStatName(QUEUE_SHRUNK_OPS)).get(); + return ioStatisticsStore.counters().get(getStatName(QUEUE_SHRUNK_OPS)); } @VisibleForTesting public long getWriteCurrentBufferOperations() { - return ioStatisticsStore.getCounterReference(getStatName(WRITE_CURRENT_BUFFER_OPERATIONS)).get(); + return ioStatisticsStore.counters().get(getStatName(WRITE_CURRENT_BUFFER_OPERATIONS)); } @VisibleForTesting public double getTimeSpentOnPutRequest() { - return ioStatisticsStore.getMeanStatistic(getStatName(TIME_TAKEN_TO_PUT_REQUEST) + ".mean").mean(); + return ioStatisticsStore.meanStatistics().get(getStatName(TIME_SPENT_ON_PUT_REQUEST) + StoreStatisticNames.SUFFIX_MEAN).mean(); } /** diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStreamStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStreamStatistics.java index 14e260511d4c4..ddbdbea5d881f 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStreamStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsOutputStreamStatistics.java @@ -22,21 +22,16 @@ import org.assertj.core.api.Assertions; import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl; -import org.apache.hadoop.fs.statistics.IOStatistics; /** * Test AbfsOutputStream statistics. */ public class ITestAbfsOutputStreamStatistics extends AbstractAbfsIntegrationTest { - private static final Logger LOG = - LoggerFactory.getLogger(ITestAbfsOutputStreamStatistics.class); private static final int OPERATIONS = 10; @@ -246,7 +241,7 @@ public void testAbfsOutputStreamDurationTrackerPutRequest() throws IOException { getAbfsOutputStreamStatistics(outputStream); Assertions.assertThat(abfsOutputStreamStatistics.getTimeSpentOnPutRequest()) - .describedAs("Mismatch in timeTakenForPutRequest DurationTracker") + .describedAs("Mismatch in timeSpentOnPutRequest DurationTracker") .isGreaterThan(0.0); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsOutputStreamStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsOutputStreamStatistics.java index 11f317f8c2543..5f9404302bd2c 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsOutputStreamStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsOutputStreamStatistics.java @@ -20,7 +20,6 @@ import java.util.Random; -import org.assertj.core.api.Assertions; import org.junit.Test; import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; From c7bb49f9adf88e03a0b48480602f94f23aba2f60 Mon Sep 17 00:00:00 2001 From: Mehakmeet Singh Date: Tue, 6 Oct 2020 15:31:34 +0530 Subject: [PATCH 14/16] HADOOP-17272. AbfsInputStream supports IOStatistics --- .../hadoop/fs/azurebfs/AbfsStatistic.java | 16 +- .../fs/azurebfs/services/AbfsInputStream.java | 27 ++- .../services/AbfsInputStreamStatistics.java | 7 + .../AbfsInputStreamStatisticsImpl.java | 157 +++++++++++------- .../ITestAbfsInputStreamStatistics.java | 44 ++++- 5 files changed, 181 insertions(+), 70 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsStatistic.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsStatistic.java index 6c78841bd7c0c..cae8bd9ed3a24 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsStatistic.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsStatistic.java @@ -89,7 +89,21 @@ public enum AbfsStatistic { WRITE_CURRENT_BUFFER_OPERATIONS("write_current_buffer_ops", "Total number of times the current buffer is written to the service."), TIME_SPENT_ON_PUT_REQUEST("time_spent_on_put_request", - "Total time spent on a put request."); + "Total time spent on a put request."), + + //InputStream statistics. + SEEK_IN_BUFFER("seek_in_buffer", + "Total number of seek operations performed in the buffer."), + BYTES_READ_BUFFER("bytes_read_buffer", + "Total number of bytes read from the buffer."), + REMOTE_READ_OP("remote_read_op", + "Total number of remote read operations performed."), + READ_AHEAD_BYTES_READ("read_ahead_bytes_read", + "Total number of bytes from readAhead."), + REMOTE_BYTES_READ("remote_bytes_read", + "Total number of bytes read from remote operations."), + NEGATIVE_SEEK_BYTES_BACKWARDS("negative_seek_bytes_backwards", + "Negative of the total bytes seeked backwards."); private String statName; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index ff3bd63cc7bf7..bca176ff9adbc 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -37,6 +37,11 @@ import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.utils.CachedSASToken; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.fs.statistics.StoreStatisticNames; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; import static org.apache.hadoop.util.StringUtils.toLowerCase; @@ -44,7 +49,7 @@ * The AbfsInputStream for AbfsClient. */ public class AbfsInputStream extends FSInputStream implements CanUnbuffer, - StreamCapabilities { + StreamCapabilities, IOStatisticsSource { private static final Logger LOG = LoggerFactory.getLogger(AbfsInputStream.class); private final AbfsClient client; @@ -73,6 +78,8 @@ public class AbfsInputStream extends FSInputStream implements CanUnbuffer, private long bytesFromReadAhead; // bytes read from readAhead; for testing private long bytesFromRemoteRead; // bytes read remotely; for testing + private IOStatistics ioStatistics; + public AbfsInputStream( final AbfsClient client, final Statistics statistics, @@ -92,6 +99,9 @@ public AbfsInputStream( this.cachedSasToken = new CachedSASToken( abfsInputStreamContext.getSasTokenRenewPeriodForStreamsInSeconds()); this.streamStatistics = abfsInputStreamContext.getStreamStatistics(); + if (streamStatistics != null) { + ioStatistics = streamStatistics.getIOStatistics(); + } } public String getPath() { @@ -279,7 +289,15 @@ int readRemote(long position, byte[] b, int offset, int length) throws IOExcepti AbfsPerfTracker tracker = client.getAbfsPerfTracker(); try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker, "readRemote", "read")) { LOG.trace("Trigger client.read for path={} position={} offset={} length={}", path, position, offset, length); - op = client.read(path, position, b, offset, length, tolerateOobAppends ? "*" : eTag, cachedSasToken.get()); + if (ioStatistics != null) { + op = IOStatisticsBinding.trackDuration((IOStatisticsStore) ioStatistics, + StoreStatisticNames.ACTION_HTTP_GET_REQUEST, + () -> client.read(path, position, b, offset, length, + tolerateOobAppends ? "*" : eTag, cachedSasToken.get())).apply(); + } else { + op = client.read(path, position, b, offset, length, + tolerateOobAppends ? "*" : eTag, cachedSasToken.get()); + } cachedSasToken.update(op.getSasToken()); if (streamStatistics != null) { streamStatistics.remoteReadOperation(); @@ -527,6 +545,11 @@ public long getBytesFromRemoteRead() { return bytesFromRemoteRead; } + @Override + public IOStatistics getIOStatistics() { + return ioStatistics; + } + /** * Get the statistics of the stream. * @return a string value. diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamStatistics.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamStatistics.java index c910a1f75e02e..0e6440f42467c 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamStatistics.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamStatistics.java @@ -19,6 +19,7 @@ package org.apache.hadoop.fs.azurebfs.services; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.statistics.IOStatistics; /** * Interface for statistics for the AbfsInputStream. @@ -96,6 +97,12 @@ public interface AbfsInputStreamStatistics { */ void remoteBytesRead(long bytes); + /** + * Get the IOStatisticsStore instance from AbfsInputStreamStatistics. + * @return instance of IOStatisticsStore which extends IOStatistics. + */ + IOStatistics getIOStatistics(); + /** * Makes the string of all the AbfsInputStream statistics. * @return the string with all the statistics. diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamStatisticsImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamStatisticsImpl.java index 12cc407dcbcf0..804da72b33806 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamStatisticsImpl.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamStatisticsImpl.java @@ -18,23 +18,42 @@ package org.apache.hadoop.fs.azurebfs.services; +import com.google.common.annotations.VisibleForTesting; + +import org.apache.hadoop.fs.azurebfs.AbfsStatistic; +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.fs.statistics.StoreStatisticNames; +import org.apache.hadoop.fs.statistics.StreamStatisticNames; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; + +import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.*; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore; + /** * Stats for the AbfsInputStream. */ public class AbfsInputStreamStatisticsImpl - implements AbfsInputStreamStatistics { - private long seekOperations; - private long forwardSeekOperations; - private long backwardSeekOperations; - private long bytesRead; - private long bytesSkippedOnSeek; - private long bytesBackwardsOnSeek; - private long seekInBuffer; - private long readOperations; - private long bytesReadFromBuffer; - private long remoteReadOperations; - private long readAheadBytesRead; - private long remoteBytesRead; + implements AbfsInputStreamStatistics, IOStatisticsSource { + + private final IOStatisticsStore ioStatisticsStore = iostatisticsStore() + .withCounters( + StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS, + StreamStatisticNames.STREAM_READ_SEEK_FORWARD_OPERATIONS, + StreamStatisticNames.STREAM_READ_SEEK_BACKWARD_OPERATIONS, + StreamStatisticNames.STREAM_READ_BYTES, + StreamStatisticNames.STREAM_READ_SEEK_BYTES_SKIPPED, + StreamStatisticNames.STREAM_READ_OPERATIONS, + getStatName(NEGATIVE_SEEK_BYTES_BACKWARDS), + getStatName(SEEK_IN_BUFFER), + getStatName(BYTES_READ_BUFFER), + getStatName(REMOTE_READ_OP), + getStatName(READ_AHEAD_BYTES_READ), + getStatName(REMOTE_BYTES_READ) + ) + .withDurationTracking(StoreStatisticNames.ACTION_HTTP_GET_REQUEST) + .build(); /** * Seek backwards, incrementing the seek and backward seek counters. @@ -44,9 +63,9 @@ public class AbfsInputStreamStatisticsImpl */ @Override public void seekBackwards(long negativeOffset) { - seekOperations++; - backwardSeekOperations++; - bytesBackwardsOnSeek -= negativeOffset; + ioStatisticsStore.incrementCounter(StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS); + ioStatisticsStore.incrementCounter(StreamStatisticNames.STREAM_READ_SEEK_BACKWARD_OPERATIONS); + ioStatisticsStore.incrementCounter(getStatName(NEGATIVE_SEEK_BYTES_BACKWARDS), negativeOffset); } /** @@ -58,11 +77,9 @@ public void seekBackwards(long negativeOffset) { */ @Override public void seekForwards(long skipped) { - seekOperations++; - forwardSeekOperations++; - if (skipped > 0) { - bytesSkippedOnSeek += skipped; - } + ioStatisticsStore.incrementCounter(StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS); + ioStatisticsStore.incrementCounter(StreamStatisticNames.STREAM_READ_SEEK_FORWARD_OPERATIONS); + ioStatisticsStore.incrementCounter(StreamStatisticNames.STREAM_READ_SEEK_BYTES_SKIPPED, skipped); } /** @@ -90,9 +107,7 @@ public void seek(long seekTo, long currentPos) { */ @Override public void bytesRead(long bytes) { - if (bytes > 0) { - bytesRead += bytes; - } + ioStatisticsStore.incrementCounter(StreamStatisticNames.STREAM_READ_BYTES, bytes); } /** @@ -104,9 +119,7 @@ public void bytesRead(long bytes) { */ @Override public void bytesReadFromBuffer(long bytes) { - if (bytes > 0) { - bytesReadFromBuffer += bytes; - } + ioStatisticsStore.incrementCounter(getStatName(BYTES_READ_BUFFER), bytes); } /** @@ -116,7 +129,7 @@ public void bytesReadFromBuffer(long bytes) { */ @Override public void seekInBuffer() { - seekInBuffer++; + ioStatisticsStore.incrementCounter(getStatName(SEEK_IN_BUFFER)); } /** @@ -127,7 +140,7 @@ public void seekInBuffer() { */ @Override public void readOperationStarted(long pos, long len) { - readOperations++; + ioStatisticsStore.incrementCounter(StreamStatisticNames.STREAM_READ_OPERATIONS); } /** @@ -137,9 +150,7 @@ public void readOperationStarted(long pos, long len) { */ @Override public void readAheadBytesRead(long bytes) { - if (bytes > 0) { - readAheadBytesRead += bytes; - } + ioStatisticsStore.incrementCounter(getStatName(READ_AHEAD_BYTES_READ), bytes); } /** @@ -149,9 +160,7 @@ public void readAheadBytesRead(long bytes) { */ @Override public void remoteBytesRead(long bytes) { - if (bytes > 0) { - remoteBytesRead += bytes; - } + ioStatisticsStore.incrementCounter(getStatName(REMOTE_BYTES_READ), bytes); } /** @@ -161,55 +170,92 @@ public void remoteBytesRead(long bytes) { */ @Override public void remoteReadOperation() { - remoteReadOperations++; + ioStatisticsStore.incrementCounter(getStatName(REMOTE_READ_OP)); } + /** + * Getter for IOStatistics instance used. + * @return IOStatisticsStore instance which extends IOStatistics. + */ + @Override + public IOStatistics getIOStatistics() { + return ioStatisticsStore; + } + + @VisibleForTesting public long getSeekOperations() { - return seekOperations; + return ioStatisticsStore.counters().get(StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS); } + @VisibleForTesting public long getForwardSeekOperations() { - return forwardSeekOperations; + return ioStatisticsStore.counters().get(StreamStatisticNames.STREAM_READ_SEEK_FORWARD_OPERATIONS); } + @VisibleForTesting public long getBackwardSeekOperations() { - return backwardSeekOperations; + return ioStatisticsStore.counters().get(StreamStatisticNames.STREAM_READ_SEEK_BACKWARD_OPERATIONS); } + @VisibleForTesting public long getBytesRead() { - return bytesRead; + return ioStatisticsStore.counters().get(StreamStatisticNames.STREAM_READ_BYTES); } + @VisibleForTesting public long getBytesSkippedOnSeek() { - return bytesSkippedOnSeek; + return ioStatisticsStore.counters().get(StreamStatisticNames.STREAM_READ_SEEK_BYTES_SKIPPED); } - public long getBytesBackwardsOnSeek() { - return bytesBackwardsOnSeek; + @VisibleForTesting + public long getNegativeBytesBackwardsOnSeek() { + return ioStatisticsStore.counters().get(getStatName(NEGATIVE_SEEK_BYTES_BACKWARDS)); } + @VisibleForTesting public long getSeekInBuffer() { - return seekInBuffer; + return ioStatisticsStore.counters().get(getStatName(SEEK_IN_BUFFER)); + } + @VisibleForTesting public long getReadOperations() { - return readOperations; + return ioStatisticsStore.counters().get(StreamStatisticNames.STREAM_READ_OPERATIONS); } + @VisibleForTesting public long getBytesReadFromBuffer() { - return bytesReadFromBuffer; + return ioStatisticsStore.counters().get(getStatName(BYTES_READ_BUFFER)); } + @VisibleForTesting public long getRemoteReadOperations() { - return remoteReadOperations; + return ioStatisticsStore.counters().get(getStatName(REMOTE_READ_OP)); } + @VisibleForTesting public long getReadAheadBytesRead() { - return readAheadBytesRead; + return ioStatisticsStore.counters().get(getStatName(READ_AHEAD_BYTES_READ)); } + @VisibleForTesting public long getRemoteBytesRead() { - return remoteBytesRead; + return ioStatisticsStore.counters().get(getStatName(REMOTE_BYTES_READ)); + } + + @VisibleForTesting + public double getActionHttpGetRequest() { + return ioStatisticsStore.meanStatistics(). + get(StoreStatisticNames.ACTION_HTTP_GET_REQUEST + StoreStatisticNames.SUFFIX_MEAN).mean(); + } + + /** + * Method to get Statistic name from the enum class. + * @param statistic AbfsStatistic to get the name of. + * @return String value of AbfsStatistic name. + */ + private String getStatName(AbfsStatistic statistic) { + return statistic.getStatName(); } /** @@ -223,18 +269,7 @@ public long getRemoteBytesRead() { public String toString() { final StringBuilder sb = new StringBuilder( "StreamStatistics{"); - sb.append(", SeekOperations=").append(seekOperations); - sb.append(", ForwardSeekOperations=").append(forwardSeekOperations); - sb.append(", BackwardSeekOperations=").append(backwardSeekOperations); - sb.append(", BytesSkippedOnSeek=").append(bytesSkippedOnSeek); - sb.append(", BytesBackwardsOnSeek=").append(bytesBackwardsOnSeek); - sb.append(", seekInBuffer=").append(seekInBuffer); - sb.append(", BytesRead=").append(bytesRead); - sb.append(", ReadOperations=").append(readOperations); - sb.append(", bytesReadFromBuffer=").append(bytesReadFromBuffer); - sb.append(", remoteReadOperations=").append(remoteReadOperations); - sb.append(", readAheadBytesRead=").append(readAheadBytesRead); - sb.append(", remoteBytesRead=").append(remoteBytesRead); + sb.append(IOStatisticsLogging.ioStatisticsSourceToString(ioStatisticsStore)); sb.append('}'); return sb.toString(); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java index 52dfdf2a61ca8..45a7bf60daa86 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java @@ -72,7 +72,7 @@ public void testInitValues() throws IOException { checkInitValue(stats.getBackwardSeekOperations(), "backwardSeekOps"); checkInitValue(stats.getBytesRead(), "bytesRead"); checkInitValue(stats.getBytesSkippedOnSeek(), "bytesSkippedOnSeek"); - checkInitValue(stats.getBytesBackwardsOnSeek(), "bytesBackwardsOnSeek"); + checkInitValue(stats.getNegativeBytesBackwardsOnSeek(), "bytesBackwardsOnSeek"); checkInitValue(stats.getSeekInBuffer(), "seekInBuffer"); checkInitValue(stats.getReadOperations(), "readOps"); checkInitValue(stats.getBytesReadFromBuffer(), "bytesReadFromBuffer"); @@ -139,10 +139,9 @@ public void testSeekStatistics() throws IOException { * forwardSeekOps - Since we are doing a forward seek inside a loop * for OPERATION times, total forward seeks would be OPERATIONS. * - * bytesBackwardsOnSeek - Since we are doing backward seeks from end of - * file in a ONE_MB file each time, this would mean the bytes from - * backward seek would be OPERATIONS * ONE_MB. Since this is backward - * seek this value is expected be to be negative. + * negativeBytesBackwardsOnSeek - Since we are doing backward seeks from + * end of file in a ONE_MB file each time, this would mean the bytes from + * backward seek would be OPERATIONS * ONE_MB. * * bytesSkippedOnSeek - Since, we move from start to end in seek, but * our fCursor(position of cursor) always remain at end of file, this @@ -160,7 +159,7 @@ public void testSeekStatistics() throws IOException { assertEquals("Mismatch in forwardSeekOps value", OPERATIONS, stats.getForwardSeekOperations()); assertEquals("Mismatch in bytesBackwardsOnSeek value", - -1 * OPERATIONS * ONE_MB, stats.getBytesBackwardsOnSeek()); + OPERATIONS * ONE_MB, stats.getNegativeBytesBackwardsOnSeek()); assertEquals("Mismatch in bytesSkippedOnSeek value", 0, stats.getBytesSkippedOnSeek()); assertEquals("Mismatch in seekInBuffer value", 2 * OPERATIONS, @@ -366,6 +365,39 @@ public void testReadAheadCounters() throws IOException { } } + /** + * Testing time taken by AbfsInputStream to complete a GET request. + */ + @Test + public void testActionHttpGetRequest() throws IOException { + describe("Test to check the correct value of Time taken by http get " + + "request in AbfsInputStream"); + AzureBlobFileSystem fs = getFileSystem(); + AzureBlobFileSystemStore abfss = fs.getAbfsStore(); + Path actionHttpGetRequestPath = path(getMethodName()); + AbfsInputStream abfsInputStream = null; + AbfsOutputStream abfsOutputStream = null; + try { + abfsOutputStream = createAbfsOutputStreamWithFlushEnabled(fs, + actionHttpGetRequestPath); + abfsOutputStream.write('a'); + abfsOutputStream.hflush(); + + abfsInputStream = + abfss.openFileForRead(actionHttpGetRequestPath, fs.getFsStatistics()); + abfsInputStream.read(); + AbfsInputStreamStatisticsImpl abfsInputStreamStatistics = + (AbfsInputStreamStatisticsImpl) abfsInputStream.getStreamStatistics(); + + Assertions.assertThat( + abfsInputStreamStatistics.getActionHttpGetRequest()) + .describedAs("Mismatch in time taken by a GET request") + .isGreaterThan(0.0); + } finally { + IOUtils.cleanupWithLogger(LOG, abfsInputStream, abfsOutputStream); + } + } + /** * Method to assert the initial values of the statistics. * From ca1ef06693e8981baa38c443b4951565ec9b8fa7 Mon Sep 17 00:00:00 2001 From: Mehakmeet Singh Date: Thu, 8 Oct 2020 17:41:44 +0530 Subject: [PATCH 15/16] HADOOP-17272. fix review comments(AbfsInputStream) + rebase --- .../hadoop/fs/azurebfs/AbfsStatistic.java | 4 +- .../fs/azurebfs/services/AbfsInputStream.java | 2 +- .../AbfsInputStreamStatisticsImpl.java | 48 +++++++++---------- .../ITestAbfsInputStreamStatistics.java | 4 +- 4 files changed, 28 insertions(+), 30 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsStatistic.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsStatistic.java index cae8bd9ed3a24..0f279dcbb6150 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsStatistic.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsStatistic.java @@ -101,9 +101,7 @@ public enum AbfsStatistic { READ_AHEAD_BYTES_READ("read_ahead_bytes_read", "Total number of bytes from readAhead."), REMOTE_BYTES_READ("remote_bytes_read", - "Total number of bytes read from remote operations."), - NEGATIVE_SEEK_BYTES_BACKWARDS("negative_seek_bytes_backwards", - "Negative of the total bytes seeked backwards."); + "Total number of bytes read from remote operations."); private String statName; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index bca176ff9adbc..658c3d886c88b 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -293,7 +293,7 @@ int readRemote(long position, byte[] b, int offset, int length) throws IOExcepti op = IOStatisticsBinding.trackDuration((IOStatisticsStore) ioStatistics, StoreStatisticNames.ACTION_HTTP_GET_REQUEST, () -> client.read(path, position, b, offset, length, - tolerateOobAppends ? "*" : eTag, cachedSasToken.get())).apply(); + tolerateOobAppends ? "*" : eTag, cachedSasToken.get())); } else { op = client.read(path, position, b, offset, length, tolerateOobAppends ? "*" : eTag, cachedSasToken.get()); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamStatisticsImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamStatisticsImpl.java index 804da72b33806..748c5948537eb 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamStatisticsImpl.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamStatisticsImpl.java @@ -25,10 +25,10 @@ import org.apache.hadoop.fs.statistics.IOStatisticsLogging; import org.apache.hadoop.fs.statistics.IOStatisticsSource; import org.apache.hadoop.fs.statistics.StoreStatisticNames; -import org.apache.hadoop.fs.statistics.StreamStatisticNames; import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.*; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.*; import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore; /** @@ -39,13 +39,13 @@ public class AbfsInputStreamStatisticsImpl private final IOStatisticsStore ioStatisticsStore = iostatisticsStore() .withCounters( - StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS, - StreamStatisticNames.STREAM_READ_SEEK_FORWARD_OPERATIONS, - StreamStatisticNames.STREAM_READ_SEEK_BACKWARD_OPERATIONS, - StreamStatisticNames.STREAM_READ_BYTES, - StreamStatisticNames.STREAM_READ_SEEK_BYTES_SKIPPED, - StreamStatisticNames.STREAM_READ_OPERATIONS, - getStatName(NEGATIVE_SEEK_BYTES_BACKWARDS), + STREAM_READ_SEEK_OPERATIONS, + STREAM_READ_SEEK_FORWARD_OPERATIONS, + STREAM_READ_SEEK_BACKWARD_OPERATIONS, + STREAM_READ_BYTES, + STREAM_READ_SEEK_BYTES_SKIPPED, + STREAM_READ_OPERATIONS, + STREAM_READ_SEEK_BYTES_BACKWARDS, getStatName(SEEK_IN_BUFFER), getStatName(BYTES_READ_BUFFER), getStatName(REMOTE_READ_OP), @@ -63,9 +63,9 @@ public class AbfsInputStreamStatisticsImpl */ @Override public void seekBackwards(long negativeOffset) { - ioStatisticsStore.incrementCounter(StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS); - ioStatisticsStore.incrementCounter(StreamStatisticNames.STREAM_READ_SEEK_BACKWARD_OPERATIONS); - ioStatisticsStore.incrementCounter(getStatName(NEGATIVE_SEEK_BYTES_BACKWARDS), negativeOffset); + ioStatisticsStore.incrementCounter(STREAM_READ_SEEK_OPERATIONS); + ioStatisticsStore.incrementCounter(STREAM_READ_SEEK_BACKWARD_OPERATIONS); + ioStatisticsStore.incrementCounter(STREAM_READ_SEEK_BYTES_BACKWARDS, negativeOffset); } /** @@ -77,9 +77,9 @@ public void seekBackwards(long negativeOffset) { */ @Override public void seekForwards(long skipped) { - ioStatisticsStore.incrementCounter(StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS); - ioStatisticsStore.incrementCounter(StreamStatisticNames.STREAM_READ_SEEK_FORWARD_OPERATIONS); - ioStatisticsStore.incrementCounter(StreamStatisticNames.STREAM_READ_SEEK_BYTES_SKIPPED, skipped); + ioStatisticsStore.incrementCounter(STREAM_READ_SEEK_OPERATIONS); + ioStatisticsStore.incrementCounter(STREAM_READ_SEEK_FORWARD_OPERATIONS); + ioStatisticsStore.incrementCounter(STREAM_READ_SEEK_BYTES_SKIPPED, skipped); } /** @@ -107,7 +107,7 @@ public void seek(long seekTo, long currentPos) { */ @Override public void bytesRead(long bytes) { - ioStatisticsStore.incrementCounter(StreamStatisticNames.STREAM_READ_BYTES, bytes); + ioStatisticsStore.incrementCounter(STREAM_READ_BYTES, bytes); } /** @@ -140,7 +140,7 @@ public void seekInBuffer() { */ @Override public void readOperationStarted(long pos, long len) { - ioStatisticsStore.incrementCounter(StreamStatisticNames.STREAM_READ_OPERATIONS); + ioStatisticsStore.incrementCounter(STREAM_READ_OPERATIONS); } /** @@ -184,32 +184,32 @@ public IOStatistics getIOStatistics() { @VisibleForTesting public long getSeekOperations() { - return ioStatisticsStore.counters().get(StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS); + return ioStatisticsStore.counters().get(STREAM_READ_SEEK_OPERATIONS); } @VisibleForTesting public long getForwardSeekOperations() { - return ioStatisticsStore.counters().get(StreamStatisticNames.STREAM_READ_SEEK_FORWARD_OPERATIONS); + return ioStatisticsStore.counters().get(STREAM_READ_SEEK_FORWARD_OPERATIONS); } @VisibleForTesting public long getBackwardSeekOperations() { - return ioStatisticsStore.counters().get(StreamStatisticNames.STREAM_READ_SEEK_BACKWARD_OPERATIONS); + return ioStatisticsStore.counters().get(STREAM_READ_SEEK_BACKWARD_OPERATIONS); } @VisibleForTesting public long getBytesRead() { - return ioStatisticsStore.counters().get(StreamStatisticNames.STREAM_READ_BYTES); + return ioStatisticsStore.counters().get(STREAM_READ_BYTES); } @VisibleForTesting public long getBytesSkippedOnSeek() { - return ioStatisticsStore.counters().get(StreamStatisticNames.STREAM_READ_SEEK_BYTES_SKIPPED); + return ioStatisticsStore.counters().get(STREAM_READ_SEEK_BYTES_SKIPPED); } @VisibleForTesting - public long getNegativeBytesBackwardsOnSeek() { - return ioStatisticsStore.counters().get(getStatName(NEGATIVE_SEEK_BYTES_BACKWARDS)); + public long getBytesBackwardsOnSeek() { + return ioStatisticsStore.counters().get(STREAM_READ_SEEK_BYTES_BACKWARDS); } @VisibleForTesting @@ -220,7 +220,7 @@ public long getSeekInBuffer() { @VisibleForTesting public long getReadOperations() { - return ioStatisticsStore.counters().get(StreamStatisticNames.STREAM_READ_OPERATIONS); + return ioStatisticsStore.counters().get(STREAM_READ_OPERATIONS); } @VisibleForTesting diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java index 45a7bf60daa86..cc46d71f601a9 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java @@ -72,7 +72,7 @@ public void testInitValues() throws IOException { checkInitValue(stats.getBackwardSeekOperations(), "backwardSeekOps"); checkInitValue(stats.getBytesRead(), "bytesRead"); checkInitValue(stats.getBytesSkippedOnSeek(), "bytesSkippedOnSeek"); - checkInitValue(stats.getNegativeBytesBackwardsOnSeek(), "bytesBackwardsOnSeek"); + checkInitValue(stats.getBytesBackwardsOnSeek(), "bytesBackwardsOnSeek"); checkInitValue(stats.getSeekInBuffer(), "seekInBuffer"); checkInitValue(stats.getReadOperations(), "readOps"); checkInitValue(stats.getBytesReadFromBuffer(), "bytesReadFromBuffer"); @@ -159,7 +159,7 @@ public void testSeekStatistics() throws IOException { assertEquals("Mismatch in forwardSeekOps value", OPERATIONS, stats.getForwardSeekOperations()); assertEquals("Mismatch in bytesBackwardsOnSeek value", - OPERATIONS * ONE_MB, stats.getNegativeBytesBackwardsOnSeek()); + OPERATIONS * ONE_MB, stats.getBytesBackwardsOnSeek()); assertEquals("Mismatch in bytesSkippedOnSeek value", 0, stats.getBytesSkippedOnSeek()); assertEquals("Mismatch in seekInBuffer value", 2 * OPERATIONS, From a57cbe84277d62d4163fcd2cc9bf7ca304c547e2 Mon Sep 17 00:00:00 2001 From: Mehakmeet Singh Date: Sun, 18 Oct 2020 10:45:31 +0530 Subject: [PATCH 16/16] HADOOP-17272. Adding counter ref. for frequently updating counters + clean-up --- .../fs/azurebfs/services/AbfsInputStream.java | 17 +++++------ .../services/AbfsInputStreamStatistics.java | 5 +--- .../AbfsInputStreamStatisticsImpl.java | 30 ++++++++++++------- .../AbfsOutputStreamStatisticsImpl.java | 15 ++++++++-- 4 files changed, 40 insertions(+), 27 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index 658c3d886c88b..3e38ca803b924 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -134,7 +134,7 @@ public synchronized int read(final byte[] b, final int off, final int len) throw int lastReadBytes; int totalReadBytes = 0; if (streamStatistics != null) { - streamStatistics.readOperationStarted(off, len); + streamStatistics.readOperationStarted(); } incrementReadOps(); do { @@ -289,15 +289,12 @@ int readRemote(long position, byte[] b, int offset, int length) throws IOExcepti AbfsPerfTracker tracker = client.getAbfsPerfTracker(); try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker, "readRemote", "read")) { LOG.trace("Trigger client.read for path={} position={} offset={} length={}", path, position, offset, length); - if (ioStatistics != null) { - op = IOStatisticsBinding.trackDuration((IOStatisticsStore) ioStatistics, - StoreStatisticNames.ACTION_HTTP_GET_REQUEST, - () -> client.read(path, position, b, offset, length, - tolerateOobAppends ? "*" : eTag, cachedSasToken.get())); - } else { - op = client.read(path, position, b, offset, length, - tolerateOobAppends ? "*" : eTag, cachedSasToken.get()); - } + + op = IOStatisticsBinding.trackDuration((IOStatisticsStore) ioStatistics, + StoreStatisticNames.ACTION_HTTP_GET_REQUEST, + () -> client.read(path, position, b, offset, length, + tolerateOobAppends ? "*" : eTag, cachedSasToken.get())); + cachedSasToken.update(op.getSasToken()); if (streamStatistics != null) { streamStatistics.remoteReadOperation(); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamStatistics.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamStatistics.java index 0e6440f42467c..818bde188845a 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamStatistics.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamStatistics.java @@ -74,11 +74,8 @@ public interface AbfsInputStreamStatistics { /** * 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); + void readOperationStarted(); /** * Records a successful remote read operation. diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamStatisticsImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamStatisticsImpl.java index 748c5948537eb..26450ef273e6e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamStatisticsImpl.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamStatisticsImpl.java @@ -18,11 +18,12 @@ package org.apache.hadoop.fs.azurebfs.services; +import java.util.concurrent.atomic.AtomicLong; + import com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.fs.azurebfs.AbfsStatistic; 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.fs.statistics.StoreStatisticNames; import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; @@ -55,6 +56,13 @@ public class AbfsInputStreamStatisticsImpl .withDurationTracking(StoreStatisticNames.ACTION_HTTP_GET_REQUEST) .build(); + private final AtomicLong bytesRead = + ioStatisticsStore.getCounterReference(STREAM_READ_BYTES); + private final AtomicLong readOps = + ioStatisticsStore.getCounterReference(STREAM_READ_OPERATIONS); + private final AtomicLong seekOps = + ioStatisticsStore.getCounterReference(STREAM_READ_SEEK_OPERATIONS); + /** * Seek backwards, incrementing the seek and backward seek counters. * @@ -63,7 +71,7 @@ public class AbfsInputStreamStatisticsImpl */ @Override public void seekBackwards(long negativeOffset) { - ioStatisticsStore.incrementCounter(STREAM_READ_SEEK_OPERATIONS); + seekOps.incrementAndGet(); ioStatisticsStore.incrementCounter(STREAM_READ_SEEK_BACKWARD_OPERATIONS); ioStatisticsStore.incrementCounter(STREAM_READ_SEEK_BYTES_BACKWARDS, negativeOffset); } @@ -77,7 +85,7 @@ public void seekBackwards(long negativeOffset) { */ @Override public void seekForwards(long skipped) { - ioStatisticsStore.incrementCounter(STREAM_READ_SEEK_OPERATIONS); + seekOps.incrementAndGet(); ioStatisticsStore.incrementCounter(STREAM_READ_SEEK_FORWARD_OPERATIONS); ioStatisticsStore.incrementCounter(STREAM_READ_SEEK_BYTES_SKIPPED, skipped); } @@ -107,7 +115,7 @@ public void seek(long seekTo, long currentPos) { */ @Override public void bytesRead(long bytes) { - ioStatisticsStore.incrementCounter(STREAM_READ_BYTES, bytes); + bytesRead.addAndGet(bytes); } /** @@ -134,13 +142,10 @@ public void seekInBuffer() { /** * 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) { - ioStatisticsStore.incrementCounter(STREAM_READ_OPERATIONS); + public void readOperationStarted() { + readOps.incrementAndGet(); } /** @@ -243,6 +248,11 @@ public long getRemoteBytesRead() { return ioStatisticsStore.counters().get(getStatName(REMOTE_BYTES_READ)); } + /** + * Getter for the mean value of the time taken to complete a HTTP GET + * request by AbfsInputStream. + * @return mean value. + */ @VisibleForTesting public double getActionHttpGetRequest() { return ioStatisticsStore.meanStatistics(). @@ -269,7 +279,7 @@ private String getStatName(AbfsStatistic statistic) { public String toString() { final StringBuilder sb = new StringBuilder( "StreamStatistics{"); - sb.append(IOStatisticsLogging.ioStatisticsSourceToString(ioStatisticsStore)); + sb.append(ioStatisticsStore.toString()); sb.append('}'); return sb.toString(); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatisticsImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatisticsImpl.java index 7cc5ccc3a6de4..0a29c0b11c6be 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatisticsImpl.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamStatisticsImpl.java @@ -18,12 +18,13 @@ package org.apache.hadoop.fs.azurebfs.services; +import java.util.concurrent.atomic.AtomicLong; + import com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.fs.azurebfs.AbfsStatistic; import org.apache.hadoop.fs.statistics.DurationTracker; 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.fs.statistics.StoreStatisticNames; import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; @@ -51,6 +52,9 @@ public class AbfsOutputStreamStatisticsImpl ) .build(); + private final AtomicLong bytesUpload = + ioStatisticsStore.getCounterReference(getStatName(BYTES_TO_UPLOAD)); + /** * Records the need to upload bytes and increments the total bytes that * needs to be uploaded. @@ -59,7 +63,7 @@ public class AbfsOutputStreamStatisticsImpl */ @Override public void bytesToUpload(long bytes) { - ioStatisticsStore.incrementCounter(getStatName(BYTES_TO_UPLOAD), bytes); + bytesUpload.addAndGet(bytes); } /** @@ -166,6 +170,11 @@ public long getWriteCurrentBufferOperations() { return ioStatisticsStore.counters().get(getStatName(WRITE_CURRENT_BUFFER_OPERATIONS)); } + /** + * Getter for mean value of time taken to complete a PUT request by + * AbfsOutputStream. + * @return mean value. + */ @VisibleForTesting public double getTimeSpentOnPutRequest() { return ioStatisticsStore.meanStatistics().get(getStatName(TIME_SPENT_ON_PUT_REQUEST) + StoreStatisticNames.SUFFIX_MEAN).mean(); @@ -188,7 +197,7 @@ private String getStatName(AbfsStatistic statistic) { @Override public String toString() { final StringBuilder outputStreamStats = new StringBuilder( "OutputStream Statistics{"); - outputStreamStats.append(IOStatisticsLogging.ioStatisticsSourceToString(ioStatisticsStore)); + outputStreamStats.append(ioStatisticsStore.toString()); outputStreamStats.append("}"); return outputStreamStats.toString(); }

+ * 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/impl/IOStatisticsBinding.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java index fec197fa5fb5f..f6c92ca69ca80 100644 --- 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 @@ -356,7 +356,7 @@ public static FunctionRaisingIOE trackFunctionDuration( DurationTrackerFactory factory, String statistic, FunctionRaisingIOE inputFn) { - return (A x) -> { + return (x) -> { // create the tracker outside try-with-resources so // that failures can be set in the catcher. DurationTracker tracker = factory.trackDuration(statistic); @@ -377,6 +377,43 @@ public static FunctionRaisingIOE trackFunctionDuration( }; } + /** + * 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( + 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 = factory.trackDuration(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, * return a new one which wraps the inner and tracks @@ -450,4 +487,5 @@ public static Callable trackDurationOfCallable( } + } 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..d784df3505d45 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestDurationTracking.java @@ -0,0 +1,283 @@ +/* + * 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.IOException; +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.statistics.impl.IOStatisticsStore; +import org.apache.hadoop.test.AbstractHadoopTestBase; +import org.apache.hadoop.util.functional.FunctionRaisingIOE; + +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.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; + + @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); + verifyCounterStatisticValue(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. + */ + public void sleep() { + sleepf(10); + } + + /** + * A little sleep function; exceptions are swallowed. + */ + protected int sleepf(final int millis) { + try { + Thread.sleep(millis); + } catch (InterruptedException ignored) { + } + return millis; + } + + /** + * Test that a function raising an IOE can be wrapped. + */ + @Test + public void testDurationFunctionIOE() throws Throwable { + FunctionRaisingIOE fn + = trackFunctionDuration(stats, REQUESTS, + (Integer x) -> x); + assertThat(fn.apply(1)).isEqualTo(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"); + })); + assertSummaryValues( + fetchSuccessSummary(stats, REQUESTS), + 1, -1, -1); + + assertSummaryValues(fetchDurationSummary(stats, REQUESTS, false), + 1, 0, 0); + } + + /** + * Duration of the successful execution of a CallableRaisingIOE + */ + @Test + public void testCallableIOEDuration() throws Throwable { + // call the operation + assertThat( + trackDuration(stats, REQUESTS, () -> sleepf(100)).apply()) + .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"); + }).apply()); + assertSummaryValues( + fetchSuccessSummary(stats, REQUESTS), + 1, -1, -1); + + 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(); + } + + /** + * 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/TestIOStatisticsStore.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestIOStatisticsStore.java index 61702eab792b0..c3de165215dac 100644 --- 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 @@ -28,28 +28,16 @@ import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; import org.apache.hadoop.test.AbstractHadoopTestBase; import org.apache.hadoop.util.JsonSerialization; -import org.apache.hadoop.util.functional.FunctionRaisingIOE; -import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatMaximumStatistic; -import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatMeanStatistic; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatMeanStatisticMatches; -import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatMinimumStatistic; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyCounterStatisticValue; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyGaugeStatisticValue; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyMaximumStatisticValue; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyMinimumStatisticValue; import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.snapshotIOStatistics; -import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_LIST_REQUEST; -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; import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore; -import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackFunctionDuration; -import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** - * * Test the IOStatisticStore implementation. */ public class TestIOStatisticsStore extends AbstractHadoopTestBase { @@ -80,7 +68,6 @@ public void setup() { .withMinimums(MIN) .withMaximums(MAX) .withMeanStatistics(MEAN) - .withDurationTracking(OBJECT_LIST_REQUEST) .build(); } @@ -178,6 +165,7 @@ 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)) @@ -186,87 +174,4 @@ public void testNegativeCounterIncrementIgnored() throws Throwable { .isEqualTo(2); } - /** - * Duration tracking. - */ - @Test - public void testDuration() throws Throwable { - DurationTracker tracker = - stats.trackDuration(OBJECT_LIST_REQUEST); - verifyCounterStatisticValue(stats, OBJECT_LIST_REQUEST, 1L); - sleep(); - tracker.close(); - try (DurationTracker ignored = - stats.trackDuration(OBJECT_LIST_REQUEST)) { - sleep(); - } - LOG.info("Statistics: {}", stats); - verifyCounterStatisticValue(stats, OBJECT_LIST_REQUEST, 2L); - assertThatMinimumStatistic(stats, - OBJECT_LIST_REQUEST + SUFFIX_MIN) - .isGreaterThan(0); - assertThatMaximumStatistic(stats, - OBJECT_LIST_REQUEST + SUFFIX_MAX) - .isGreaterThan(0); - assertThatMeanStatistic(stats, - OBJECT_LIST_REQUEST + SUFFIX_MEAN) - .hasFieldOrPropertyWithValue("samples", 2L) - .matches(s -> s.getSum() > 0) - .matches(s -> s.mean() > 0.0); - } - - public void sleep() { - try { - Thread.sleep(1000); - } catch (InterruptedException ignored) { - } - } - - @Test - public void testDurationSuccess() throws Throwable { - FunctionRaisingIOE fn - = trackFunctionDuration(stats, OBJECT_LIST_REQUEST, - (Integer x) -> x); - Assertions.assertThat(fn.apply(1)).isEqualTo(1); - verifyCounterStatisticValue(stats, OBJECT_LIST_REQUEST, 1L); - assertThatMinimumStatistic(stats, - OBJECT_LIST_REQUEST + SUFFIX_MIN) - .isGreaterThanOrEqualTo(0); - assertThatMaximumStatistic(stats, - OBJECT_LIST_REQUEST + SUFFIX_MAX) - .isGreaterThanOrEqualTo(0); - - } - - /** - * Trigger a failure and verify its the failure statistics - * which go up. - */ - @Test - public void testDurationFailure() throws Throwable { - FunctionRaisingIOE fn = - trackFunctionDuration(stats, OBJECT_LIST_REQUEST, - (Integer x) -> { - sleep(); - return 100 / x; - }); - intercept(ArithmeticException.class, - () -> fn.apply(0)); - verifyCounterStatisticValue(stats, OBJECT_LIST_REQUEST, 1L); - verifyCounterStatisticValue(stats, - OBJECT_LIST_REQUEST + SUFFIX_FAILURES, 1L); - assertThatMinimumStatistic(stats, - OBJECT_LIST_REQUEST + SUFFIX_MIN) - .isLessThan(0); - assertThatMaximumStatistic(stats, - OBJECT_LIST_REQUEST + SUFFIX_MAX) - .isLessThan(0); - assertThatMinimumStatistic(stats, - OBJECT_LIST_REQUEST + SUFFIX_FAILURES + SUFFIX_MIN) - .isGreaterThan(0); - assertThatMaximumStatistic(stats, - OBJECT_LIST_REQUEST + SUFFIX_FAILURES + SUFFIX_MAX) - .isGreaterThan(0); - } - } From 116169fa8fb548531ff7d2e648859a436b1620fd Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 6 Oct 2020 19:12:13 +0100 Subject: [PATCH 08/16] HADOOP-16830. Finishing up API * move the methods in fs.impl.FutureIOSupport which turn out to be needed when working with the async code to the public utils.functional package * duration tracking invocation in IOStatisticsBinding tuning based on the ABFS coding experience. Change-Id: Ide9467fa67a8a5d3d8daec94a7a39ff87f106a47 --- .../hadoop/fs/impl/FutureIOSupport.java | 69 +------ .../BufferedIOStatisticsInputStream.java | 6 +- .../statistics/impl/IOStatisticsBinding.java | 22 ++- .../hadoop/util/functional/FutureIO.java | 178 ++++++++++++++++++ .../util/functional/RemoteIterators.java | 4 +- .../util/functional/RuntimeIOException.java | 11 +- .../fs/statistics/TestDurationTracking.java | 55 +++++- 7 files changed, 270 insertions(+), 75 deletions(-) create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/FutureIO.java 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 8d1a8b3342851..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 @@ -33,10 +33,15 @@ 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.RuntimeIOException; +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 @@ -57,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); } @@ -84,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. @@ -112,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); } /** @@ -127,45 +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. - *