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..42dd28082874d --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsLogging.java @@ -0,0 +1,207 @@ +/* + * 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 ""; + } + } + + 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..b4d91327a11ad --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSnapshot.java @@ -0,0 +1,215 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics; + +import javax.annotation.Nullable; +import java.io.Serializable; +import java.util.Map; +import java.util.TreeMap; + +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 TreeMap counters; + + /** + * Gauges. + */ + @JsonProperty + private TreeMap gauges; + + /** + * Minimum values. + */ + @JsonProperty + private TreeMap minimums; + + /** + * Maximum values. + */ + @JsonProperty + private TreeMap maximums; + + /** + * mean statistics. The JSON key is all lower case.. + */ + @JsonProperty("meanstatistics") + private TreeMap 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 TreeMap<>(); + gauges = new TreeMap<>(); + minimums = new TreeMap<>(); + maximums = new TreeMap<>(); + meanStatistics = new TreeMap<>(); + } + + /** + * 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 + * statistics reference passed in. + *

+ * The operation is synchronized. + * @param statistics statistics; may be null + * @return true if a merge took place. + */ + @Override + public synchronized boolean aggregate( + @Nullable IOStatistics statistics) { + if (statistics == null) { + return false; + } + aggregateMaps(counters, statistics.counters(), + IOStatisticsBinding::aggregateCounters, + IOStatisticsBinding::passthroughFn); + aggregateMaps(gauges, statistics.gauges(), + IOStatisticsBinding::aggregateGauges, + IOStatisticsBinding::passthroughFn); + aggregateMaps(minimums, statistics.minimums(), + IOStatisticsBinding::aggregateMinimums, + IOStatisticsBinding::passthroughFn); + aggregateMaps(maximums, statistics.maximums(), + IOStatisticsBinding::aggregateMaximums, + IOStatisticsBinding::passthroughFn); + aggregateMaps(meanStatistics, statistics.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); + } + +} 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..f390170422d54 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/MeanStatistic.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 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) { + this(that.samples, that.sum); + } + + /** + * Create an empty statistic. + */ + public MeanStatistic() { + } + + /** + * Get the sum of samples. + * @return the sum + */ + public long getSum() { + return sum; + } + + /** + * 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 + */ + @JsonIgnore + public 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. + *

+ * Thread safe. + * @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; + } + if (isEmpty()) { + samples = other.samples; + sum = other.sum; + return this; + } + samples += other.samples; + sum += other.sum; + 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..849f008328b0b --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/BufferedIOStatisticsOutputStream.java @@ -0,0 +1,152 @@ +/* + * 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; + } + + @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..22bc2e77dab2c --- /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<>("counters"); + + private final EvaluatingStatisticsMap gauges + = new EvaluatingStatisticsMap<>("gauges"); + + private final EvaluatingStatisticsMap minimums + = new EvaluatingStatisticsMap<>("minimums"); + + private final EvaluatingStatisticsMap maximums + = new EvaluatingStatisticsMap<>("maximums"); + + private final EvaluatingStatisticsMap meanStatistics + = new EvaluatingStatisticsMap<>("meanStatistics", 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..1249d0c843c21 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/EvaluatingStatisticsMap.java @@ -0,0 +1,205 @@ +/* + * 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.TreeMap; +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; + + /** + * Name for use in getter/error messages. + */ + private final String name; + + /** + * Construct with the copy function being simple passthrough. + * @param name map name + */ + EvaluatingStatisticsMap(final String name) { + this(name, IOStatisticsBinding::passthroughFn); + } + + /** + * Construct with the copy function being that supplied in. + * @param name map name + * @param copyFn copy function. + */ + EvaluatingStatisticsMap(final String name, + final Function copyFn) { + this.name = name; + 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(); + } + + /** + * Takes a snapshot and then provide an iterator around this. + * @return the iterator. + */ + @Override + public Collection values() { + return snapshot().values(); + } + + /** + * Take a snapshot. + * @return a map snapshot. + */ + public TreeMap snapshot() { + return IOStatisticsBinding.snapshotMap(this, copyFn); + } + + /** + * Creating the entry set forces an evaluation of the functions. + *

+ * 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; + } + + String getName() { + return name; + } + + /** + * 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..e3b1568d97636 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java @@ -0,0 +1,336 @@ +/* + * 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.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 TreeMap 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 new map referencing the same values. + */ + public static TreeMap snapshotMap( + Map source, + Function copyFn) { + TreeMap dest = new TreeMap<>(); + 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, + final 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..08bf7e929af9c --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStore.java @@ -0,0 +1,259 @@ +/* + * 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(); + + /** + * Update the counter values from a statistics source. + *

+ * The source must have all keys in this instance; + * extra keys are ignored. + * @param source source of statistics. + */ + void copy(IOStatistics source); + + /** + * 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..285c3647a341a --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStoreImpl.java @@ -0,0 +1,421 @@ +/* + * 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.HashMap; +import java.util.List; +import java.util.Map; +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 HashMap<>(); + + private final Map gaugeMap = new HashMap<>(); + + private final Map minimumMap = new HashMap<>(); + + private final Map maximumMap = new HashMap<>(); + + private final Map meanStatisticMap + = new HashMap<>(); + + /** + * 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 -> { + return 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.setSamplesAndSum(value.getSamples(), value.getSum()); + } + } + + @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. + * Unsynchronized. + */ + @Override + public 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()); + } + + @Override + public void copy(final IOStatistics source) { + counterMap.entrySet().forEach(e -> { + e.getValue().set(lookup(source.counters(), e.getKey())); + }); + gaugeMap.entrySet().forEach(e -> { + e.getValue().set(lookup(source.gauges(), e.getKey())); + }); + maximumMap.entrySet().forEach(e -> { + e.getValue().set(lookup(source.maximums(), e.getKey())); + }); + minimumMap.entrySet().forEach(e -> { + e.getValue().set(lookup(source.minimums(), e.getKey())); + }); + meanStatisticMap.entrySet().forEach(e -> { + String key = e.getKey(); + MeanStatistic statisticValue = lookup(source.meanStatistics(), key); + e.getValue().set(statisticValue); + }); + } + + /** + * Aggregate those statistics which the store is tracking; + * ignore the rest. + * + * @param statistics statistics; may be null + * @return true if a statistics reference was supplied/aggregated. + */ + @Override + public boolean aggregate(@Nullable final IOStatistics statistics) { + if (statistics == null) { + return false; + } + // counters: addition + counterMap.entrySet().forEach(e -> { + e.getValue().addAndGet(lookup(statistics.counters(), e.getKey())); + }); + // gauge: add positive values only + gaugeMap.entrySet().forEach(e -> { + long sourceGauge = lookup(statistics.gauges(), e.getKey()); + if (sourceGauge > 0) { + e.getValue().addAndGet(sourceGauge); + } + }); + // min: min of current and source + minimumMap.entrySet().forEach(e -> { + AtomicLong dest = e.getValue(); + long sourceValue = lookup(statistics.minimums(), e.getKey()); + dest.set(aggregateMinimums(dest.get(), sourceValue)); + }); + // max: max of current and source + maximumMap.entrySet().forEach(e -> { + AtomicLong dest = e.getValue(); + long sourceValue = lookup(statistics.maximums(), e.getKey()); + dest.set(aggregateMaximums(dest.get(), sourceValue)); + }); + // the most complex + meanStatisticMap.entrySet().forEach(e -> { + MeanStatistic current = e.getValue(); + MeanStatistic sourceValue = lookup( + statistics.meanStatistics(), e.getKey()); + 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 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 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 + */ + @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..51a7839be4995 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/FunctionRaisingIOE.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.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..c8de122cd8aed --- /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..4fa6b227ba125 --- /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 roundTrip(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..0efa2b2a1fb82 --- /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.roundTrip(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..a4a64c2bf1952 --- /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.roundTrip(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..19694e28d754b --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestIOStatisticsSnapshot.java @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics; + +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); + + private final IOStatisticsSnapshot snapshot = new IOStatisticsSnapshot(); + + private MeanStatistic mean1; + + private MeanStatistic mean0; + + @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 testRoundTrip() 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")); + } + + +} 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..83b096fc37713 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestMeanStatistic.java @@ -0,0 +1,216 @@ +/* + * 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; + +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 testRoundTrip() 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); + } + } + +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LocatedFileStatusFetcher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LocatedFileStatusFetcher.java index 4cb36a5576bd5..3c68003ee1d93 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LocatedFileStatusFetcher.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LocatedFileStatusFetcher.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.LinkedList; import java.util.List; +import java.util.StringJoiner; import java.util.concurrent.BlockingQueue; import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; @@ -29,7 +30,8 @@ import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; -import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -37,6 +39,9 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; import com.google.common.collect.Iterables; @@ -51,15 +56,22 @@ import org.apache.hadoop.util.concurrent.HadoopExecutors; +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics; +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.snapshotIOStatistics; + /** * Utility class to fetch block locations for specified Input paths using a * configured number of threads. * The thread count is determined from the value of * "mapreduce.input.fileinputformat.list-status.num-threads" in the * configuration. + *

+ * Although historically tagged as private, it is used elsewhere, such as + * in the Parquet-hadoop module. Avoid breaking it where possible. */ -@Private -public class LocatedFileStatusFetcher { +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class LocatedFileStatusFetcher implements IOStatisticsSource { public static final Logger LOG = LoggerFactory.getLogger(LocatedFileStatusFetcher.class.getName()); @@ -86,6 +98,12 @@ public class LocatedFileStatusFetcher { private volatile Throwable unknownError; + /** + * Demand created IO Statistics: only if the filesystem + * returns statistics does this fetch collect them. + */ + private IOStatisticsSnapshot iostats; + /** * Instantiate. * The newApi switch is only used to configure what exception is raised @@ -224,7 +242,46 @@ private void decrementRunningAndCheckCompletion() { lock.unlock(); } } - + + /** + * Return any IOStatistics collected during listing. + * @return IO stats accrued. + */ + @Override + public synchronized IOStatistics getIOStatistics() { + return iostats; + } + + /** + * Add the statistics of an individual thread's scan. + * @param stats possibly null statistics. + */ + private void addResultStatistics(IOStatistics stats) { + if (stats != null) { + // demand creation of IO statistics. + synchronized (this) { + LOG.debug("Adding IOStatistics: {}", stats); + if (iostats == null) { + // demand create the statistics + iostats = snapshotIOStatistics(stats); + } else { + iostats.aggregate(stats); + } + } + } + } + + @Override + public String toString() { + final IOStatistics ioStatistics = getIOStatistics(); + StringJoiner stringJoiner = new StringJoiner(", ", + LocatedFileStatusFetcher.class.getSimpleName() + "[", "]"); + if (ioStatistics != null) { + stringJoiner.add("IOStatistics=" + ioStatistics); + } + return stringJoiner.toString(); + } + /** * Retrieves block locations for the given @link {@link FileStatus}, and adds * additional paths to the process queue if required. @@ -264,6 +321,8 @@ public Result call() throws Exception { } } } + // aggregate any stats + result.stats = retrieveIOStatistics(iter); } else { result.locatedFileStatuses.add(fileStatus); } @@ -274,6 +333,7 @@ private static class Result { private List locatedFileStatuses = new LinkedList<>(); private List dirsNeedingRecursiveCalls = new LinkedList<>(); private FileSystem fs; + private IOStatistics stats; } } @@ -288,6 +348,7 @@ private class ProcessInputDirCallback implements @Override public void onSuccess(ProcessInputDirCallable.Result result) { try { + addResultStatistics(result.stats); if (!result.locatedFileStatuses.isEmpty()) { resultQueue.add(result.locatedFileStatuses); } diff --git a/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml b/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml index b51053603fa7b..1ebf8587e8824 100644 --- a/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml +++ b/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml @@ -74,4 +74,14 @@ + + + + + + diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java index a1c1d969a8258..fcaec509290c8 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java @@ -549,6 +549,13 @@ private Constants() { @InterfaceStability.Unstable public static final String INPUT_FADV_RANDOM = "random"; + /** + * Gauge name for the input policy : {@value}. + * This references an enum currently exclusive to the S3A stream. + */ + public static final String STREAM_READ_GAUGE_INPUT_POLICY = + "stream_read_gauge_input_policy"; + @InterfaceAudience.Private @InterfaceStability.Unstable public static final String S3_CLIENT_FACTORY_IMPL = diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java index 276961bf8b7e1..99c0067fd438d 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java @@ -23,16 +23,26 @@ import com.amazonaws.ClientConfiguration; import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.client.builder.AwsClientBuilder; +import com.amazonaws.metrics.RequestMetricCollector; import com.amazonaws.services.s3.AmazonS3; import com.amazonaws.services.s3.AmazonS3Client; +import com.amazonaws.services.s3.AmazonS3ClientBuilder; import com.amazonaws.services.s3.S3ClientOptions; +import com.amazonaws.services.s3.internal.ServiceUtils; +import com.amazonaws.util.AwsHostNameUtils; +import com.amazonaws.util.RuntimeHttpUtils; +import com.google.common.annotations.VisibleForTesting; import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; +import org.apache.hadoop.fs.s3a.impl.statistics.AwsStatisticsCollector; +import org.apache.hadoop.fs.s3a.impl.statistics.StatisticsFromAwsSdk; import static org.apache.hadoop.fs.s3a.Constants.EXPERIMENTAL_AWS_INTERNAL_THROTTLING; import static org.apache.hadoop.fs.s3a.Constants.ENDPOINT; @@ -41,7 +51,7 @@ /** * The default {@link S3ClientFactory} implementation. - * This which calls the AWS SDK to configure and create an + * This calls the AWS SDK to configure and create an * {@link AmazonS3Client} that communicates with the S3 service. */ @InterfaceAudience.Private @@ -49,13 +59,30 @@ public class DefaultS3ClientFactory extends Configured implements S3ClientFactory { - protected static final Logger LOG = S3AFileSystem.LOG; + private static final String S3_SERVICE_NAME = "s3"; + private static final String S3_SIGNER = "S3SignerType"; + private static final String S3_V4_SIGNER = "AWSS3V4SignerType"; + /** + * Subclasses refer to this. + */ + protected static final Logger LOG = + LoggerFactory.getLogger(DefaultS3ClientFactory.class); + + /** + * Create the client. + *

+ * If the AWS stats are not null then a {@link AwsStatisticsCollector}. + * is created to bind to the two. + * Important: until this binding works properly across regions, + * this should be null. + */ @Override public AmazonS3 createS3Client(URI name, final String bucket, final AWSCredentialsProvider credentials, - final String userAgentSuffix) throws IOException { + final String userAgentSuffix, + final StatisticsFromAwsSdk statisticsFromAwsSdk) throws IOException { Configuration conf = getConf(); final ClientConfiguration awsConf = S3AUtils .createAwsConf(conf, bucket, Constants.AWS_SERVICE_IDENTIFIER_S3); @@ -72,36 +99,123 @@ public AmazonS3 createS3Client(URI name, if (!StringUtils.isEmpty(userAgentSuffix)) { awsConf.setUserAgentSuffix(userAgentSuffix); } - return configureAmazonS3Client( - newAmazonS3Client(credentials, awsConf), conf); + // optional metrics + RequestMetricCollector metrics = statisticsFromAwsSdk != null + ? new AwsStatisticsCollector(statisticsFromAwsSdk) + : null; + + return newAmazonS3Client( + credentials, + awsConf, + metrics, + conf.getTrimmed(ENDPOINT, ""), + conf.getBoolean(PATH_STYLE_ACCESS, false)); } /** - * Wrapper around constructor for {@link AmazonS3} client. + * Create an {@link AmazonS3} client. * Override this to provide an extended version of the client * @param credentials credentials to use * @param awsConf AWS configuration - * @return new AmazonS3 client + * @param metrics metrics collector or null + * @param endpoint endpoint string; may be "" + * @param pathStyleAccess enable path style access? + * @return new AmazonS3 client */ protected AmazonS3 newAmazonS3Client( - AWSCredentialsProvider credentials, ClientConfiguration awsConf) { - return new AmazonS3Client(credentials, awsConf); + final AWSCredentialsProvider credentials, + final ClientConfiguration awsConf, + final RequestMetricCollector metrics, + final String endpoint, + final boolean pathStyleAccess) { + if (metrics != null) { + LOG.debug("Building S3 client using the SDK builder API"); + return buildAmazonS3Client(credentials, awsConf, metrics, endpoint, + pathStyleAccess); + } else { + LOG.debug("Building S3 client using the SDK builder API"); + return classicAmazonS3Client(credentials, awsConf, endpoint, + pathStyleAccess); + } } /** - * Configure S3 client from the Hadoop configuration. - * + * Use the (newer) Builder SDK to create a an AWS S3 client. + *

+ * This has a more complex endpoint configuration in a + * way which does not yet work in this code in a way + * which doesn't trigger regressions. So it is only used + * when SDK metrics are supplied. + * @param credentials credentials to use + * @param awsConf AWS configuration + * @param metrics metrics collector or null + * @param endpoint endpoint string; may be "" + * @param pathStyleAccess enable path style access? + * @return new AmazonS3 client + */ + private AmazonS3 buildAmazonS3Client( + final AWSCredentialsProvider credentials, + final ClientConfiguration awsConf, + final RequestMetricCollector metrics, + final String endpoint, + final boolean pathStyleAccess) { + AmazonS3ClientBuilder b = AmazonS3Client.builder(); + b.withCredentials(credentials); + b.withClientConfiguration(awsConf); + b.withPathStyleAccessEnabled(pathStyleAccess); + if (metrics != null) { + b.withMetricsCollector(metrics); + } + + // endpoint set up is a PITA + // client.setEndpoint("") is no longer available + AwsClientBuilder.EndpointConfiguration epr + = createEndpointConfiguration(endpoint, awsConf); + if (epr != null) { + // an endpoint binding was constructed: use it. + b.withEndpointConfiguration(epr); + } + final AmazonS3 client = b.build(); + return client; + } + + /** + * Wrapper around constructor for {@link AmazonS3} client. + * Override this to provide an extended version of the client. + *

+ * This uses a deprecated constructor -it is currently + * the only one which works for us. + * @param credentials credentials to use + * @param awsConf AWS configuration + * @param endpoint endpoint string; may be "" + * @param pathStyleAccess enable path style access? + * @return new AmazonS3 client + */ + @SuppressWarnings("deprecation") + private AmazonS3 classicAmazonS3Client( + AWSCredentialsProvider credentials, + ClientConfiguration awsConf, + final String endpoint, + final boolean pathStyleAccess) { + final AmazonS3 client = new AmazonS3Client(credentials, awsConf); + return configureAmazonS3Client(client, endpoint, pathStyleAccess); + } + + /** + * Configure classic S3 client. + *

* This includes: endpoint, Path Access and possibly other * options. * - * @param conf Hadoop configuration + * @param s3 S3 Client. + * @param pathStyleAccess enable path style access? * @return S3 client * @throws IllegalArgumentException if misconfigured */ - private static AmazonS3 configureAmazonS3Client(AmazonS3 s3, - Configuration conf) + protected static AmazonS3 configureAmazonS3Client(AmazonS3 s3, + final String endPoint, + final boolean pathStyleAccess) throws IllegalArgumentException { - String endPoint = conf.getTrimmed(ENDPOINT, ""); if (!endPoint.isEmpty()) { try { s3.setEndpoint(endPoint); @@ -111,7 +225,7 @@ private static AmazonS3 configureAmazonS3Client(AmazonS3 s3, throw new IllegalArgumentException(msg, e); } } - return applyS3ClientOptions(s3, conf); + return applyS3ClientOptions(s3, pathStyleAccess); } /** @@ -119,23 +233,23 @@ private static AmazonS3 configureAmazonS3Client(AmazonS3 s3, * the Hadoop configuration. * This is different from the general AWS configuration creation as * it is unique to S3 connections. - * + *

* The {@link Constants#PATH_STYLE_ACCESS} option enables path-style access * to S3 buckets if configured. By default, the * behavior is to use virtual hosted-style access with URIs of the form * {@code http://bucketname.s3.amazonaws.com} + *

* Enabling path-style access and a * region-specific endpoint switches the behavior to use URIs of the form * {@code http://s3-eu-west-1.amazonaws.com/bucketname}. * It is common to use this when connecting to private S3 servers, as it * avoids the need to play with DNS entries. * @param s3 S3 client - * @param conf Hadoop configuration + * @param pathStyleAccess enable path style access? * @return the S3 client */ - private static AmazonS3 applyS3ClientOptions(AmazonS3 s3, - Configuration conf) { - final boolean pathStyleAccess = conf.getBoolean(PATH_STYLE_ACCESS, false); + protected static AmazonS3 applyS3ClientOptions(AmazonS3 s3, + final boolean pathStyleAccess) { if (pathStyleAccess) { LOG.debug("Enabling path style access!"); s3.setS3ClientOptions(S3ClientOptions.builder() @@ -144,4 +258,54 @@ private static AmazonS3 applyS3ClientOptions(AmazonS3 s3, } return s3; } + + /** + * Given an endpoint string, return an endpoint config, or null, if none + * is needed. + *

+ * This is a pretty painful piece of code. It is trying to replicate + * what AwsClient.setEndpoint() does, because you can't + * call that setter on an AwsClient constructed via + * the builder, and you can't pass a metrics collector + * down except through the builder. + *

+ * Note also that AWS signing is a mystery which nobody fully + * understands, especially given all problems surface in a + * "400 bad request" response, which, like all security systems, + * provides minimal diagnostics out of fear of leaking + * secrets. + * + * @param endpoint possibly null endpoint. + * @param awsConf config to build the URI from. + * @return a configuration for the S3 client builder. + */ + @VisibleForTesting + public static AwsClientBuilder.EndpointConfiguration + createEndpointConfiguration( + final String endpoint, final ClientConfiguration awsConf) { + LOG.debug("Creating endpoint configuration for {}", endpoint); + if (endpoint == null || endpoint.isEmpty()) { + // the default endpoint...we should be using null at this point. + LOG.debug("Using default endpoint -no need to generate a configuration"); + return null; + } + + final URI epr = RuntimeHttpUtils.toUri(endpoint, awsConf); + LOG.debug("Endpoint URI = {}", epr); + + String region; + if (!ServiceUtils.isS3USStandardEndpoint(endpoint)) { + LOG.debug("Endpoint {} is not the default; parsing", epr); + region = AwsHostNameUtils.parseRegion( + epr.getHost(), + S3_SERVICE_NAME); + } else { + // US-east, set region == null. + LOG.debug("Endpoint {} is the standard one; declare region as null", epr); + region = null; + } + LOG.debug("Region for endpoint {}, URI {} is determined as {}", + endpoint, epr, region); + return new AwsClientBuilder.EndpointConfiguration(endpoint, region); + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java index 932c472f5bea2..e99ec1ab28f75 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java @@ -20,6 +20,7 @@ import com.amazonaws.ClientConfiguration; import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.metrics.RequestMetricCollector; import com.amazonaws.services.s3.AmazonS3; import org.apache.hadoop.classification.InterfaceAudience; @@ -40,12 +41,21 @@ public class InconsistentS3ClientFactory extends DefaultS3ClientFactory { * Logs a warning that this is being done. * @param credentials credentials to use * @param awsConf AWS configuration + * @param metrics + * @param endpoint + * @param pathStyleAccess * @return an inconsistent client. */ @Override protected AmazonS3 newAmazonS3Client(AWSCredentialsProvider credentials, - ClientConfiguration awsConf) { + ClientConfiguration awsConf, + final RequestMetricCollector metrics, + final String endpoint, + final boolean pathStyleAccess) { LOG.warn("** FAILURE INJECTION ENABLED. Do not run in production! **"); - return new InconsistentAmazonS3Client(credentials, awsConf, getConf()); + InconsistentAmazonS3Client s3 + = new InconsistentAmazonS3Client(credentials, awsConf, getConf()); + configureAmazonS3Client(s3, endpoint, pathStyleAccess); + return s3; } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java index 16413a7620d0d..4f081f09c4ac5 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java @@ -37,6 +37,10 @@ import org.apache.hadoop.fs.s3a.s3guard.MetadataStoreListFilesIterator; import org.apache.hadoop.fs.s3a.s3guard.PathMetadata; import org.apache.hadoop.fs.s3a.s3guard.S3Guard; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; +import org.apache.hadoop.util.functional.RemoteIterators; import com.google.common.base.Preconditions; import org.slf4j.Logger; @@ -47,7 +51,6 @@ import java.time.OffsetDateTime; import java.time.ZoneOffset; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -56,6 +59,7 @@ import java.util.NoSuchElementException; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.StringJoiner; import static org.apache.hadoop.fs.impl.FutureIOSupport.awaitFuture; import static org.apache.hadoop.fs.s3a.Constants.S3N_FOLDER_SUFFIX; @@ -66,6 +70,9 @@ import static org.apache.hadoop.fs.s3a.S3AUtils.stringify; import static org.apache.hadoop.fs.s3a.S3AUtils.translateException; import static org.apache.hadoop.fs.s3a.auth.RoleModel.pathToKey; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_CONTINUE_LIST_REQUEST; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_LIST_REQUEST; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore; /** * Place for the S3A listing classes; keeps all the small classes under control. @@ -171,9 +178,11 @@ public ObjectListingIterator createObjectListingIterator( * @return a new remote iterator */ @VisibleForTesting - public LocatedFileStatusIterator createLocatedFileStatusIterator( + public RemoteIterator createLocatedFileStatusIterator( RemoteIterator statusIterator) { - return new LocatedFileStatusIterator(statusIterator); + return RemoteIterators.mappingRemoteIterator( + statusIterator, + listingOperationCallbacks::toLocatedFileStatus); } /** @@ -185,11 +194,28 @@ public LocatedFileStatusIterator createLocatedFileStatusIterator( * @return a new remote iterator. */ @VisibleForTesting - TombstoneReconcilingIterator createTombstoneReconcilingIterator( - RemoteIterator iterator, Set tombstones) { - return new TombstoneReconcilingIterator(iterator, tombstones); + RemoteIterator createTombstoneReconcilingIterator( + RemoteIterator iterator, + @Nullable Set tombstones) { + if (tombstones == null || tombstones.isEmpty()) { + // no need to filter. + return iterator; + } else { + return RemoteIterators.filteringRemoteIterator( + iterator, + candidate -> !tombstones.contains(candidate.getPath())); + } } + /** + * Create a remote iterator from a single status entry. + * @param status status + * @return iterator. + */ + public RemoteIterator createSingleStatusIterator( + S3ALocatedFileStatus status) { + return RemoteIterators.remoteIteratorFromSingleton(status); + } /** * List files under a path assuming the path to be a directory. @@ -365,64 +391,6 @@ interface FileStatusAcceptor { boolean accept(FileStatus status); } - /** - * A remote iterator which only iterates over a single `LocatedFileStatus` - * value. - * - * If the status value is null, the iterator declares that it has no data. - * This iterator is used to handle {@link S3AFileSystem#listStatus(Path)} - * calls where the path handed in refers to a file, not a directory: - * this is the iterator returned. - */ - static final class SingleStatusRemoteIterator - implements RemoteIterator { - - /** - * The status to return; set to null after the first iteration. - */ - private S3ALocatedFileStatus status; - - /** - * Constructor. - * @param status status value: may be null, in which case - * the iterator is empty. - */ - SingleStatusRemoteIterator(S3ALocatedFileStatus status) { - this.status = status; - } - - /** - * {@inheritDoc} - * @return true if there is a file status to return: this is always false - * for the second iteration, and may be false for the first. - * @throws IOException never - */ - @Override - public boolean hasNext() throws IOException { - return status != null; - } - - /** - * {@inheritDoc} - * @return the non-null status element passed in when the instance was - * constructed, if it ha not already been retrieved. - * @throws IOException never - * @throws NoSuchElementException if this is the second call, or it is - * the first call and a null {@link LocatedFileStatus} entry was passed - * to the constructor. - */ - @Override - public S3ALocatedFileStatus next() throws IOException { - if (hasNext()) { - S3ALocatedFileStatus s = this.status; - status = null; - return s; - } else { - throw new NoSuchElementException(); - } - } - } - /** * This wraps up a provided non-null list of file status as a remote iterator. * @@ -491,7 +459,7 @@ public S3AFileStatus next() throws IOException { * Thread safety: None. */ class FileStatusListingIterator - implements RemoteIterator { + implements RemoteIterator, IOStatisticsSource { /** Source of objects. */ private final ObjectListingIterator source; @@ -694,6 +662,23 @@ private boolean buildNextStatusBatch(S3ListResult objects) { public int getBatchSize() { return batchSize; } + + /** + * Return any IOStatistics provided by the underlying stream. + * @return IO stats from the inner stream. + */ + @Override + public IOStatistics getIOStatistics() { + return source.getIOStatistics(); + } + + @Override + public String toString() { + return new StringJoiner(", ", + FileStatusListingIterator.class.getSimpleName() + "[", "]") + .add(source.toString()) + .toString(); + } } /** @@ -716,7 +701,8 @@ public int getBatchSize() { * * Thread safety: none. */ - class ObjectListingIterator implements RemoteIterator { + class ObjectListingIterator implements RemoteIterator, + IOStatisticsSource { /** The path listed. */ private final Path listPath; @@ -741,6 +727,8 @@ class ObjectListingIterator implements RemoteIterator { */ private int maxKeys; + private final IOStatisticsStore iostats; + /** * Future to store current batch listing result. */ @@ -751,6 +739,11 @@ class ObjectListingIterator implements RemoteIterator { */ private S3ListResult objectsPrev; + /** + * Context to pass into list calls. + */ + private final ListingOperationCallbacks.ListingContext listingContext; + /** * Constructor -calls `listObjects()` on the request to populate the * initial set of results/fail if there was a problem talking to the bucket. @@ -764,10 +757,18 @@ class ObjectListingIterator implements RemoteIterator { S3ListRequest request) throws IOException { this.listPath = listPath; this.maxKeys = listingOperationCallbacks.getMaxKeys(); - this.s3ListResultFuture = listingOperationCallbacks - .listObjectsAsync(request); this.request = request; this.objectsPrev = null; + this.iostats = iostatisticsStore() + .withDurationTracking(OBJECT_LIST_REQUEST) + .withDurationTracking(OBJECT_CONTINUE_LIST_REQUEST) + .build(); + // the context contains only the duration tracker factory, at + // least for now + this.listingContext = new ListingOperationCallbacks.ListingContext( + iostats); + this.s3ListResultFuture = listingOperationCallbacks + .listObjectsAsync(request, listingContext); } /** @@ -831,7 +832,7 @@ private void fetchNextBatchAsyncIfPresent() throws IOException { LOG.debug("[{}], Requesting next {} objects under {}", listingCount, maxKeys, listPath); s3ListResultFuture = listingOperationCallbacks - .continueListObjectsAsync(request, objects); + .continueListObjectsAsync(request, objects, listingContext); } } @@ -839,7 +840,13 @@ private void fetchNextBatchAsyncIfPresent() throws IOException { public String toString() { return "Object listing iterator against " + listPath + "; listing count "+ listingCount - + "; isTruncated=" + objects.isTruncated(); + + "; isTruncated=" + objects.isTruncated() + + "; " + iostats; + } + + @Override + public IOStatistics getIOStatistics() { + return iostats; } /** @@ -902,89 +909,6 @@ public boolean accept(FileStatus status) { } } - /** - * Take a remote iterator over a set of {@link FileStatus} instances and - * return a remote iterator of {@link LocatedFileStatus} instances. - */ - class LocatedFileStatusIterator - implements RemoteIterator { - private final RemoteIterator statusIterator; - - /** - * Constructor. - * @param statusIterator an iterator over the remote status entries - */ - LocatedFileStatusIterator(RemoteIterator statusIterator) { - this.statusIterator = statusIterator; - } - - @Override - public boolean hasNext() throws IOException { - return statusIterator.hasNext(); - } - - @Override - public S3ALocatedFileStatus next() throws IOException { - return listingOperationCallbacks - .toLocatedFileStatus(statusIterator.next()); - } - } - - /** - * Wraps another iterator and filters out files that appear in the provided - * set of tombstones. Will read ahead in the iterator when necessary to - * ensure that emptiness is detected early enough if only deleted objects - * remain in the source iterator. - */ - static class TombstoneReconcilingIterator implements - RemoteIterator { - private S3ALocatedFileStatus next = null; - private final RemoteIterator iterator; - private final Set tombstones; - - /** - * @param iterator Source iterator to filter - * @param tombstones set of tombstone markers to filter out of results - */ - TombstoneReconcilingIterator(RemoteIterator - iterator, Set tombstones) { - this.iterator = iterator; - if (tombstones != null) { - this.tombstones = tombstones; - } else { - this.tombstones = Collections.emptySet(); - } - } - - private boolean fetch() throws IOException { - while (next == null && iterator.hasNext()) { - S3ALocatedFileStatus candidate = iterator.next(); - if (!tombstones.contains(candidate.getPath())) { - next = candidate; - return true; - } - } - return false; - } - - public boolean hasNext() throws IOException { - if (next != null) { - return true; - } - return fetch(); - } - - public S3ALocatedFileStatus next() throws IOException { - if (hasNext()) { - S3ALocatedFileStatus result = next; - next = null; - fetch(); - return result; - } - throw new NoSuchElementException(); - } - } - /** * Accept all entries except those which map to S3N pseudo directory markers. */ @@ -1053,4 +977,9 @@ public boolean accept(FileStatus status) { } } + public static RemoteIterator toLocatedFileStatusIterator( + RemoteIterator iterator) { + return (RemoteIterator < LocatedFileStatus >) iterator; + } + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java index 15f69bde3f119..4cd3e1ff96d39 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java @@ -49,10 +49,15 @@ import org.apache.hadoop.fs.StreamCapabilities; import org.apache.hadoop.fs.s3a.commit.CommitConstants; import org.apache.hadoop.fs.s3a.commit.PutTracker; +import org.apache.hadoop.fs.s3a.impl.statistics.BlockOutputStreamStatistics; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsLogging; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; import org.apache.hadoop.util.Progressable; import static org.apache.hadoop.fs.s3a.S3AUtils.*; import static org.apache.hadoop.fs.s3a.Statistic.*; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.emptyStatistics; import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; /** @@ -67,7 +72,7 @@ @InterfaceAudience.Private @InterfaceStability.Unstable class S3ABlockOutputStream extends OutputStream implements - StreamCapabilities { + StreamCapabilities, IOStatisticsSource { private static final Logger LOG = LoggerFactory.getLogger(S3ABlockOutputStream.class); @@ -81,6 +86,9 @@ class S3ABlockOutputStream extends OutputStream implements /** Size of all blocks. */ private final int blockSize; + /** IO Statistics. */ + private final IOStatistics iostatistics; + /** Total bytes for uploads submitted so far. */ private long bytesSubmitted; @@ -109,7 +117,7 @@ class S3ABlockOutputStream extends OutputStream implements private long blockCount = 0; /** Statistics to build up. */ - private final S3AInstrumentation.OutputStreamStatistics statistics; + private final BlockOutputStreamStatistics statistics; /** * Write operation helper; encapsulation of the filesystem operations. @@ -146,7 +154,7 @@ class S3ABlockOutputStream extends OutputStream implements Progressable progress, long blockSize, S3ADataBlocks.BlockFactory blockFactory, - S3AInstrumentation.OutputStreamStatistics statistics, + BlockOutputStreamStatistics statistics, WriteOperationHelper writeOperationHelper, PutTracker putTracker) throws IOException { @@ -155,6 +163,10 @@ class S3ABlockOutputStream extends OutputStream implements this.blockFactory = blockFactory; this.blockSize = (int) blockSize; this.statistics = statistics; + // test instantiations may not provide statistics; + this.iostatistics = statistics != null + ? statistics.getIOStatistics() + : emptyStatistics(); this.writeOperationHelper = writeOperationHelper; this.putTracker = putTracker; Preconditions.checkArgument(blockSize >= Constants.MULTIPART_MIN_SIZE, @@ -282,6 +294,7 @@ public synchronized void write(byte[] source, int offset, int len) if (len == 0) { return; } + statistics.writeBytes(len); S3ADataBlocks.DataBlock block = createBlockIfNeeded(); int written = block.write(source, offset, len); int remainingCapacity = block.remainingCapacity(); @@ -382,7 +395,8 @@ public void close() throws IOException { // then complete the operation if (putTracker.aboutToComplete(multiPartUpload.getUploadId(), partETags, - bytes)) { + bytes, + iostatistics)) { multiPartUpload.complete(partETags); } else { LOG.info("File {} will be visible when the job is committed", key); @@ -473,6 +487,8 @@ public String toString() { if (block != null) { sb.append(", activeBlock=").append(block); } + sb.append(" Statistics=") + .append(IOStatisticsLogging.ioStatisticsSourceToString(this)); sb.append('}'); return sb.toString(); } @@ -493,7 +509,7 @@ private long now() { * Get the statistics for this stream. * @return stream statistics */ - S3AInstrumentation.OutputStreamStatistics getStatistics() { + BlockOutputStreamStatistics getStatistics() { return statistics; } @@ -520,11 +536,20 @@ public boolean hasCapability(String capability) { case StreamCapabilities.HSYNC: return false; + // yes, we do statistics. + case StreamCapabilities.IOSTATISTICS: + return true; + default: return false; } } + @Override + public IOStatistics getIOStatistics() { + return iostatistics; + } + /** * Multiple partition upload. */ diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java index 156defb7ca031..9fbf0008cf397 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java @@ -37,6 +37,7 @@ import org.slf4j.LoggerFactory; import org.apache.hadoop.fs.FSExceptionMessages; +import org.apache.hadoop.fs.s3a.impl.statistics.BlockOutputStreamStatistics; import org.apache.hadoop.util.DirectBufferPool; import static org.apache.hadoop.fs.s3a.S3ADataBlocks.DataBlock.DestState.*; @@ -180,7 +181,7 @@ protected BlockFactory(S3AFileSystem owner) { * @return a new block. */ abstract DataBlock create(long index, int limit, - S3AInstrumentation.OutputStreamStatistics statistics) + BlockOutputStreamStatistics statistics) throws IOException; /** @@ -210,10 +211,10 @@ enum DestState {Writing, Upload, Closed} private volatile DestState state = Writing; protected final long index; - protected final S3AInstrumentation.OutputStreamStatistics statistics; + private final BlockOutputStreamStatistics statistics; protected DataBlock(long index, - S3AInstrumentation.OutputStreamStatistics statistics) { + BlockOutputStreamStatistics statistics) { this.index = index; this.statistics = statistics; } @@ -372,6 +373,10 @@ protected void blockReleased() { statistics.blockReleased(); } } + + protected BlockOutputStreamStatistics getStatistics() { + return statistics; + } } // ==================================================================== @@ -387,7 +392,7 @@ static class ArrayBlockFactory extends BlockFactory { @Override DataBlock create(long index, int limit, - S3AInstrumentation.OutputStreamStatistics statistics) + BlockOutputStreamStatistics statistics) throws IOException { return new ByteArrayBlock(0, limit, statistics); } @@ -432,7 +437,7 @@ static class ByteArrayBlock extends DataBlock { ByteArrayBlock(long index, int limit, - S3AInstrumentation.OutputStreamStatistics statistics) { + BlockOutputStreamStatistics statistics) { super(index, statistics); this.limit = limit; buffer = new S3AByteArrayOutputStream(limit); @@ -510,7 +515,7 @@ static class ByteBufferBlockFactory extends BlockFactory { @Override ByteBufferBlock create(long index, int limit, - S3AInstrumentation.OutputStreamStatistics statistics) + BlockOutputStreamStatistics statistics) throws IOException { return new ByteBufferBlock(index, limit, statistics); } @@ -560,7 +565,7 @@ class ByteBufferBlock extends DataBlock { */ ByteBufferBlock(long index, int bufferSize, - S3AInstrumentation.OutputStreamStatistics statistics) { + BlockOutputStreamStatistics statistics) { super(index, statistics); this.bufferSize = bufferSize; blockBuffer = requestBuffer(bufferSize); @@ -805,7 +810,7 @@ static class DiskBlockFactory extends BlockFactory { @Override DataBlock create(long index, int limit, - S3AInstrumentation.OutputStreamStatistics statistics) + BlockOutputStreamStatistics statistics) throws IOException { File destFile = getOwner() .createTmpFileForWrite(String.format("s3ablock-%04d-", index), @@ -829,7 +834,7 @@ static class DiskBlock extends DataBlock { DiskBlock(File bufferFile, int limit, long index, - S3AInstrumentation.OutputStreamStatistics statistics) + BlockOutputStreamStatistics statistics) throws FileNotFoundException { super(index, statistics); this.limit = limit; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index 63c80bdd067e1..921b729af5b03 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -115,11 +115,20 @@ import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum; import org.apache.hadoop.fs.s3a.impl.StoreContext; import org.apache.hadoop.fs.s3a.impl.StoreContextBuilder; +import org.apache.hadoop.fs.s3a.impl.statistics.BlockOutputStreamStatistics; +import org.apache.hadoop.fs.s3a.impl.statistics.CommitterStatistics; +import org.apache.hadoop.fs.s3a.impl.statistics.BondedS3AStatisticsContext; import org.apache.hadoop.fs.s3a.impl.statistics.S3AMultipartUploaderStatisticsImpl; +import org.apache.hadoop.fs.s3a.impl.statistics.S3AStatisticsContext; +import org.apache.hadoop.fs.s3a.impl.statistics.StatisticsFromAwsSdk; import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; import org.apache.hadoop.fs.s3a.select.InternalSelectConstants; import org.apache.hadoop.fs.s3a.tools.MarkerToolOperations; import org.apache.hadoop.fs.s3a.tools.MarkerToolOperationsImpl; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.fs.statistics.DurationTracker; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.Text; import org.apache.hadoop.security.token.DelegationTokenIssuer; @@ -172,6 +181,7 @@ import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs; import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.Invoker.*; +import static org.apache.hadoop.fs.s3a.Listing.toLocatedFileStatusIterator; import static org.apache.hadoop.fs.s3a.S3AUtils.*; import static org.apache.hadoop.fs.s3a.Statistic.*; import static org.apache.commons.lang3.StringUtils.isNotEmpty; @@ -182,9 +192,12 @@ import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit; import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletionIgnoringExceptions; import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isUnknownBucket; +import static org.apache.hadoop.fs.s3a.impl.InternalConstants.AWS_SDK_METRICS_ENABLED; import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_404; import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.fixBucketRegion; import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.logDnsLookup; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_CONTINUE_LIST_REQUEST; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_LIST_REQUEST; import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; /** @@ -203,7 +216,7 @@ @InterfaceAudience.Private @InterfaceStability.Evolving public class S3AFileSystem extends FileSystem implements StreamCapabilities, - AWSPolicyProvider, DelegationTokenProvider { + AWSPolicyProvider, DelegationTokenProvider, IOStatisticsSource { /** * Default blocksize as used in blocksize and FS status queries. */ @@ -258,6 +271,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, private S3AInstrumentation instrumentation; private final S3AStorageStatistics storageStatistics = createStorageStatistics(); + + private S3AStatisticsContext statisticsContext; + private long readAhead; private S3AInputPolicy inputPolicy; private ChangeDetectionPolicy changeDetectionPolicy; @@ -363,6 +379,7 @@ public void initialize(URI name, Configuration originalConf) invoker = new Invoker(new S3ARetryPolicy(getConf()), onRetry); instrumentation = new S3AInstrumentation(uri); + initializeStatisticsBinding(); // Username is the current user at the time the FS was instantiated. owner = UserGroupInformation.getCurrentUser(); @@ -372,7 +389,8 @@ public void initialize(URI name, Configuration originalConf) s3guardInvoker = new Invoker(new S3GuardExistsRetryPolicy(getConf()), onRetry); - writeHelper = new WriteOperationHelper(this, getConf()); + writeHelper = new WriteOperationHelper(this, getConf(), + statisticsContext); failOnMetadataWriteError = conf.getBoolean(FAIL_ON_METADATA_WRITE_ERROR, FAIL_ON_METADATA_WRITE_ERROR_DEFAULT); @@ -526,6 +544,31 @@ private void doBucketProbing() throws IOException { } } + /** + * Initialize the statistics binding. + * This is done by creating an {@code IntegratedS3AStatisticsContext} + * with callbacks to get the FS's instrumentation and FileSystem.statistics + * field; the latter may change after {@link #initialize(URI, Configuration)}, + * so needs to be dynamically adapted. + * Protected so that (mock) subclasses can replace it with a + * different statistics binding, if desired. + */ + protected void initializeStatisticsBinding() { + statisticsContext = new BondedS3AStatisticsContext( + new BondedS3AStatisticsContext.S3AFSStatisticsSource() { + + @Override + public S3AInstrumentation getInstrumentation() { + return S3AFileSystem.this.getInstrumentation(); + } + + @Override + public Statistics getInstanceStatistics() { + return S3AFileSystem.this.statistics; + } + }); + } + /** * Initialize the thread pool. * This must be re-invoked after replacing the S3Client during test @@ -605,6 +648,7 @@ protected void verifyBucketExistsV2() * Get S3A Instrumentation. For test purposes. * @return this instance's instrumentation. */ + @VisibleForTesting public S3AInstrumentation getInstrumentation() { return instrumentation; } @@ -672,8 +716,16 @@ private void bindAWSClient(URI name, boolean dtEnabled) throws IOException { S3_CLIENT_FACTORY_IMPL, DEFAULT_S3_CLIENT_FACTORY_IMPL, S3ClientFactory.class); + StatisticsFromAwsSdk awsStats = null; +// TODO: when the S3 client building code works with different regions, +// then non-null stats can be passed in here. + if (AWS_SDK_METRICS_ENABLED) { + awsStats = statisticsContext.newStatisticsFromAwsSdk(); + } + s3 = ReflectionUtils.newInstance(s3ClientFactoryClass, conf) - .createS3Client(getUri(), bucket, credentials, uaSuffix); + .createS3Client(getUri(), bucket, credentials, uaSuffix, + awsStats); } /** @@ -1171,7 +1223,7 @@ private S3AReadOpContext createReadContext( invoker, s3guardInvoker, statistics, - instrumentation, + statisticsContext, fileStatus, seekPolicy, changePolicy, @@ -1268,15 +1320,20 @@ public FSDataOutputStream create(Path f, FsPermission permission, PutTracker putTracker = committerIntegration.createTracker(path, key); String destKey = putTracker.getDestKey(); + final BlockOutputStreamStatistics outputStreamStatistics + = statisticsContext.newOutputStreamStatistics(); return new FSDataOutputStream( new S3ABlockOutputStream(this, destKey, - new SemaphoredDelegatingExecutor(boundedThreadPool, - blockOutputActiveBlocks, true), + new SemaphoredDelegatingExecutor( + boundedThreadPool, + blockOutputActiveBlocks, + true, + outputStreamStatistics), progress, partSize, blockFactory, - instrumentation.newOutputStreamStatistics(statistics), + outputStreamStatistics, getWriteOperationHelper(), putTracker), null); @@ -1650,20 +1707,33 @@ protected class ListingOperationCallbacksImpl implements @Override @Retries.RetryRaw public CompletableFuture listObjectsAsync( - S3ListRequest request) + S3ListRequest request, ListingContext listingContext) throws IOException { return submit(unboundedThreadPool, - () -> listObjects(request)); + () -> { + try (DurationTracker ignored = + listingContext.getDurationTrackerFactory() + .trackDuration(OBJECT_LIST_REQUEST)) { + return listObjects(request); + } + }); } @Override @Retries.RetryRaw public CompletableFuture continueListObjectsAsync( - S3ListRequest request, - S3ListResult prevResult) + S3ListRequest request, + S3ListResult prevResult, + ListingContext listingContext) throws IOException { return submit(unboundedThreadPool, - () -> continueListObjects(request, prevResult)); + () -> { + try (DurationTracker ignored = + listingContext.getDurationTrackerFactory() + .trackDuration(OBJECT_CONTINUE_LIST_REQUEST)) { + return continueListObjects(request, prevResult); + } + }); } @Override @@ -1812,7 +1882,7 @@ protected void incrementStatistic(Statistic statistic) { * @param count the count to increment */ protected void incrementStatistic(Statistic statistic, long count) { - instrumentation.incrementCounter(statistic, count); + statisticsContext.incrementCounter(statistic, count); storageStatistics.incrementCounter(statistic, count); } @@ -1822,7 +1892,7 @@ protected void incrementStatistic(Statistic statistic, long count) { * @param count the count to decrement */ protected void decrementGauge(Statistic statistic, long count) { - instrumentation.decrementGauge(statistic, count); + statisticsContext.decrementGauge(statistic, count); } /** @@ -1831,7 +1901,7 @@ protected void decrementGauge(Statistic statistic, long count) { * @param count the count to increment */ protected void incrementGauge(Statistic statistic, long count) { - instrumentation.incrementGauge(statistic, count); + statisticsContext.incrementGauge(statistic, count); } /** @@ -1844,6 +1914,7 @@ public void operationRetried(Exception ex) { if (isThrottleException(ex)) { operationThrottled(false); } else { + incrementStatistic(STORE_IO_RETRY); incrementStatistic(IGNORED_ERRORS); } } @@ -1895,11 +1966,11 @@ private void operationThrottled(boolean metastore) { LOG.debug("Request throttled on {}", metastore ? "S3": "DynamoDB"); if (metastore) { incrementStatistic(S3GUARD_METADATASTORE_THROTTLED); - instrumentation.addValueToQuantiles(S3GUARD_METADATASTORE_THROTTLE_RATE, + statisticsContext.addValueToQuantiles(S3GUARD_METADATASTORE_THROTTLE_RATE, 1); } else { incrementStatistic(STORE_IO_THROTTLED); - instrumentation.addValueToQuantiles(STORE_IO_THROTTLE_RATE, 1); + statisticsContext.addValueToQuantiles(STORE_IO_THROTTLE_RATE, 1); } } @@ -1912,6 +1983,16 @@ public S3AStorageStatistics getStorageStatistics() { return storageStatistics; } + /** + * Get this filesystem's storage statistics as IO Statistics. + * @return statistics + */ + @Override + public IOStatistics getIOStatistics() { + return IOStatisticsBinding.fromStorageStatistics( + storageStatistics); + } + /** * Request object metadata; increments counters in the process. * Retry policy: retry untranslated. @@ -3587,8 +3668,9 @@ private CopyResult copyFile(String srcKey, String dstKey, long size, ChangeTracker changeTracker = new ChangeTracker( keyToQualifiedPath(srcKey).toString(), changeDetectionPolicy, - readContext.instrumentation.newInputStreamStatistics() - .getVersionMismatchCounter(), + readContext.getS3AStatisticsContext() + .newInputStreamStatistics() + .getChangeTrackerStatistics(), srcAttributes); String action = "copyFile(" + srcKey + ", " + dstKey + ")"; @@ -4287,21 +4369,6 @@ public RemoteIterator listFiles(Path f, new Listing.AcceptFilesOnly(qualify(f)), null, true, false)); } - private static RemoteIterator toLocatedFileStatusIterator( - RemoteIterator iterator) { - return new RemoteIterator() { - @Override - public boolean hasNext() throws IOException { - return iterator.hasNext(); - } - - @Override - public LocatedFileStatus next() throws IOException { - return iterator.next(); - } - }; - } - /** * Recursive List of files and empty directories. * @param f path to list from @@ -4384,7 +4451,7 @@ private RemoteIterator innerListFiles( if (status != null && status.isFile()) { // simple case: File LOG.debug("Path is a file: {}", path); - return new Listing.SingleStatusRemoteIterator( + return listing.createSingleStatusIterator( toLocatedFileStatus(status)); } // Assuming the path to be a directory @@ -4404,7 +4471,7 @@ private RemoteIterator innerListFiles( ? status : (S3AFileStatus) getFileStatus(path); if (fileStatus.isFile()) { - return new Listing.SingleStatusRemoteIterator( + return listing.createSingleStatusIterator( toLocatedFileStatus(fileStatus)); } } @@ -4463,7 +4530,7 @@ public RemoteIterator listLocatedStatus(final Path f, if (fileStatus.isFile()) { // simple case: File LOG.debug("Path is a file"); - return new Listing.SingleStatusRemoteIterator( + return listing.createSingleStatusIterator( filter.accept(path) ? toLocatedFileStatus(fileStatus) : null); @@ -4475,43 +4542,6 @@ public RemoteIterator listLocatedStatus(final Path f, return toLocatedFileStatusIterator(iterator); } - /** - * Generate list located status for a directory. - * Also performing tombstone reconciliation for guarded directories. - * @param dir directory to check. - * @param filter a path filter. - * @return an iterator that traverses statuses of the given dir. - * @throws IOException in case of failure. - */ - private RemoteIterator getLocatedFileStatusIteratorForDir( - Path dir, PathFilter filter) throws IOException { - final String key = maybeAddTrailingSlash(pathToKey(dir)); - final Listing.FileStatusAcceptor acceptor = - new Listing.AcceptAllButSelfAndS3nDirs(dir); - boolean allowAuthoritative = allowAuthoritative(dir); - DirListingMetadata meta = - S3Guard.listChildrenWithTtl(metadataStore, dir, - ttlTimeProvider, allowAuthoritative); - Set tombstones = meta != null - ? meta.listTombstones() - : null; - final RemoteIterator cachedFileStatusIterator = - listing.createProvidedFileStatusIterator( - S3Guard.dirMetaToStatuses(meta), filter, acceptor); - return (allowAuthoritative && meta != null - && meta.isAuthoritative()) - ? listing.createLocatedFileStatusIterator( - cachedFileStatusIterator) - : listing.createTombstoneReconcilingIterator( - listing.createLocatedFileStatusIterator( - listing.createFileStatusListingIterator(dir, - createListObjectsRequest(key, "/"), - filter, - acceptor, - cachedFileStatusIterator)), - tombstones); - } - /** * Build a {@link S3ALocatedFileStatus} from a {@link FileStatus} instance. * @param status file status @@ -4610,8 +4640,8 @@ void abortMultipartUpload(MultipartUpload upload) { * Create a new instance of the committer statistics. * @return a new committer statistics instance */ - public S3AInstrumentation.CommitterStatistics newCommitterStatistics() { - return instrumentation.newCommitterStatistics(); + public CommitterStatistics newCommitterStatistics() { + return statisticsContext.newCommitterStatistics(); } @SuppressWarnings("deprecation") @@ -4745,8 +4775,9 @@ private FSDataInputStream select(final Path source, ChangeTracker changeTracker = new ChangeTracker(uri.toString(), changeDetectionPolicy, - readContext.instrumentation.newInputStreamStatistics() - .getVersionMismatchCounter(), + readContext.getS3AStatisticsContext() + .newInputStreamStatistics() + .getChangeTrackerStatistics(), objectAttributes); // will retry internally if wrong version detected @@ -4913,7 +4944,7 @@ public StoreContext createStoreContext() { .setExecutor(boundedThreadPool) .setExecutorCapacity(executorCapacity) .setInvoker(invoker) - .setInstrumentation(getInstrumentation()) + .setStatisticsContext(statisticsContext) .setStorageStatistics(getStorageStatistics()) .setInputPolicy(getInputPolicy()) .setChangeDetectionPolicy(changeDetectionPolicy) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java index 9c8b9ae7a156e..f1133903e8b02 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java @@ -33,10 +33,14 @@ import org.apache.hadoop.fs.CanSetReadahead; import org.apache.hadoop.fs.CanUnbuffer; import org.apache.hadoop.fs.FSExceptionMessages; -import org.apache.hadoop.fs.FSInputStream; +import org.apache.hadoop.fs.s3a.impl.statistics.S3AInputStreamStatistics; +import org.apache.hadoop.fs.s3a.impl.ChangeTracker; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.StreamCapabilities; -import org.apache.hadoop.fs.s3a.impl.ChangeTracker; +import org.apache.hadoop.fs.FSInputStream; +import org.apache.hadoop.fs.statistics.DurationTracker; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -68,7 +72,7 @@ @InterfaceAudience.Private @InterfaceStability.Evolving public class S3AInputStream extends FSInputStream implements CanSetReadahead, - CanUnbuffer, StreamCapabilities { + CanUnbuffer, StreamCapabilities, IOStatisticsSource { public static final String E_NEGATIVE_READAHEAD_VALUE = "Negative readahead value"; @@ -97,7 +101,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead, private final String uri; private static final Logger LOG = LoggerFactory.getLogger(S3AInputStream.class); - private final S3AInstrumentation.InputStreamStatistics streamStatistics; + private final S3AInputStreamStatistics streamStatistics; private S3AEncryptionMethods serverSideEncryptionAlgorithm; private String serverSideEncryptionKey; private S3AInputPolicy inputPolicy; @@ -123,6 +127,11 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead, /** change tracker. */ private final ChangeTracker changeTracker; + /** + * IOStatistics report. + */ + private final IOStatistics ioStatistics; + /** * Create the stream. * This does not attempt to open it; that is only done on the first @@ -146,13 +155,15 @@ public S3AInputStream(S3AReadOpContext ctx, this.contentLength = l; this.client = client; this.uri = "s3a://" + this.bucket + "/" + this.key; - this.streamStatistics = ctx.instrumentation.newInputStreamStatistics(); + this.streamStatistics = ctx.getS3AStatisticsContext() + .newInputStreamStatistics(); + this.ioStatistics = streamStatistics.getIOStatistics(); this.serverSideEncryptionAlgorithm = s3Attributes.getServerSideEncryptionAlgorithm(); this.serverSideEncryptionKey = s3Attributes.getServerSideEncryptionKey(); this.changeTracker = new ChangeTracker(uri, ctx.getChangeDetectionPolicy(), - streamStatistics.getVersionMismatchCounter(), + streamStatistics.getChangeTrackerStatistics(), s3Attributes); setInputPolicy(ctx.getInputPolicy()); setReadahead(ctx.getReadahead()); @@ -202,8 +213,11 @@ private synchronized void reopen(String reason, long targetPos, long length, String text = String.format("%s %s at %d", operation, uri, targetPos); changeTracker.maybeApplyConstraint(request); - S3Object object = Invoker.once(text, uri, - () -> client.getObject(request)); + S3Object object; + try (DurationTracker ignored = streamStatistics.initiateGetRequest()) { + object = Invoker.once(text, uri, + () -> client.getObject(request)); + } changeTracker.processResponse(object, operation, targetPos); @@ -286,13 +300,11 @@ private void seekInStream(long targetPos, long length) throws IOException { if (skipForward) { // the forward seek range is within the limits LOG.debug("Forward seek on {}, of {} bytes", uri, diff); - streamStatistics.seekForwards(diff); long skipped = wrappedStream.skip(diff); if (skipped > 0) { pos += skipped; - // as these bytes have been read, they are included in the counter - incrementBytesRead(diff); } + streamStatistics.seekForwards(diff, skipped); if (pos == targetPos) { // all is well @@ -304,6 +316,9 @@ private void seekInStream(long targetPos, long length) throws IOException { LOG.warn("Failed to seek on {} to {}. Current position {}", uri, targetPos, pos); } + } else { + // not attempting to read any bytes from the stream + streamStatistics.seekForwards(diff, 0); } } else if (diff < 0) { // backwards seek @@ -348,7 +363,7 @@ private void lazySeek(long targetPos, long len) throws IOException { // open. After that, an exception generally means the file has changed // and there is no point retrying anymore. Invoker invoker = context.getReadInvoker(); - invoker.maybeRetry(streamStatistics.openOperations == 0, + invoker.maybeRetry(streamStatistics.getOpenOperations() == 0, "lazySeek", pathStr, true, () -> { //For lazy seek @@ -693,7 +708,7 @@ public String toString() { sb.append(" contentRangeFinish=").append(contentRangeFinish); sb.append(" remainingInCurrentRequest=") .append(remainingInCurrentRequest()); - sb.append(changeTracker); + sb.append(" ").append(changeTracker); sb.append('\n').append(s); sb.append('}'); return sb.toString(); @@ -747,7 +762,7 @@ public void readFully(long position, byte[] buffer, int offset, int length) */ @InterfaceAudience.Private @InterfaceStability.Unstable - public S3AInstrumentation.InputStreamStatistics getS3AStreamStatistics() { + public S3AInputStreamStatistics getS3AStreamStatistics() { return streamStatistics; } @@ -831,13 +846,14 @@ public synchronized void unbuffer() { try { closeStream("unbuffer()", contentRangeFinish, false); } finally { - streamStatistics.merge(false); + streamStatistics.unbuffered(); } } @Override public boolean hasCapability(String capability) { switch (toLowerCase(capability)) { + case StreamCapabilities.IOSTATISTICS: case StreamCapabilities.READAHEAD: case StreamCapabilities.UNBUFFER: return true; @@ -850,4 +866,9 @@ public boolean hasCapability(String capability) { boolean isObjectStreamOpen() { return wrappedStream != null; } + + @Override + public IOStatistics getIOStatistics() { + return ioStatistics; + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java index cb0a43430dbd9..1521887367fd8 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java @@ -18,14 +18,29 @@ package org.apache.hadoop.fs.s3a; +import javax.annotation.Nullable; + import com.google.common.annotations.VisibleForTesting; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.fs.FileSystem.Statistics; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.s3a.impl.statistics.AbstractS3AStatisticsSource; +import org.apache.hadoop.fs.s3a.impl.statistics.ChangeTrackerStatistics; +import org.apache.hadoop.fs.s3a.impl.statistics.CommitterStatistics; +import org.apache.hadoop.fs.s3a.impl.statistics.CountersAndGauges; +import org.apache.hadoop.fs.s3a.impl.statistics.CountingChangeTracker; +import org.apache.hadoop.fs.s3a.impl.statistics.DelegationTokenStatistics; +import org.apache.hadoop.fs.s3a.impl.statistics.S3AInputStreamStatistics; +import org.apache.hadoop.fs.s3a.impl.statistics.BlockOutputStreamStatistics; import org.apache.hadoop.fs.s3a.s3guard.MetastoreInstrumentation; +import org.apache.hadoop.fs.statistics.IOStatisticsLogging; +import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; +import org.apache.hadoop.fs.statistics.StreamStatisticNames; +import org.apache.hadoop.fs.statistics.DurationTracker; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; import org.apache.hadoop.metrics2.AbstractMetric; import org.apache.hadoop.metrics2.MetricStringBuilder; import org.apache.hadoop.metrics2.MetricsCollector; @@ -49,12 +64,34 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; +import static org.apache.hadoop.fs.s3a.Constants.STREAM_READ_GAUGE_INPUT_POLICY; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.demandStringifyIOStatistics; +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.snapshotIOStatistics; +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.stubDurationTracker; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.ACTION_EXECUTOR_ACQUIRED; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.ACTION_HTTP_GET_REQUEST; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore; import static org.apache.hadoop.fs.s3a.Statistic.*; /** * Instrumentation of S3a. - * Derived from the {@code AzureFileSystemInstrumentation}. - * + *

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

* Counters and metrics are generally addressed in code by their name or * {@link Statistic} key. There may be some Statistics which do * not have an entry here. To avoid attempts to access such counters failing, @@ -63,39 +100,40 @@ */ @InterfaceAudience.Private @InterfaceStability.Evolving -public class S3AInstrumentation implements Closeable, MetricsSource { +public class S3AInstrumentation implements Closeable, MetricsSource, + CountersAndGauges { private static final Logger LOG = LoggerFactory.getLogger( S3AInstrumentation.class); private static final String METRICS_SOURCE_BASENAME = "S3AMetrics"; /** - * {@value #METRICS_SYSTEM_NAME} The name of the s3a-specific metrics + * {@value} The name of the s3a-specific metrics * system instance used for s3a metrics. */ public static final String METRICS_SYSTEM_NAME = "s3a-file-system"; /** - * {@value #CONTEXT} Currently all s3a metrics are placed in a single + * {@value} Currently all s3a metrics are placed in a single * "context". Distinct contexts may be used in the future. */ public static final String CONTEXT = "s3aFileSystem"; /** - * {@value #METRIC_TAG_FILESYSTEM_ID} The name of a field added to metrics + * {@value} The name of a field added to metrics * records that uniquely identifies a specific FileSystem instance. */ public static final String METRIC_TAG_FILESYSTEM_ID = "s3aFileSystemId"; /** - * {@value #METRIC_TAG_BUCKET} The name of a field added to metrics records + * {@value} The name of a field added to metrics records * that indicates the hostname portion of the FS URL. */ public static final String METRIC_TAG_BUCKET = "bucket"; // metricsSystemLock must be used to synchronize modifications to // metricsSystem and the following counters. - private static Object metricsSystemLock = new Object(); + private static final Object METRICS_SYSTEM_LOCK = new Object(); private static MetricsSystem metricsSystem = null; private static int metricsSourceNameCounter = 0; private static int metricsSourceActiveCounter = 0; @@ -104,22 +142,6 @@ public class S3AInstrumentation implements Closeable, MetricsSource { private final MetricsRegistry registry = new MetricsRegistry("s3aFileSystem").setContext(CONTEXT); - private final MutableCounterLong streamOpenOperations; - private final MutableCounterLong streamCloseOperations; - private final MutableCounterLong streamClosed; - private final MutableCounterLong streamAborted; - private final MutableCounterLong streamSeekOperations; - private final MutableCounterLong streamReadExceptions; - private final MutableCounterLong streamForwardSeekOperations; - private final MutableCounterLong streamBackwardSeekOperations; - private final MutableCounterLong streamBytesSkippedOnSeek; - private final MutableCounterLong streamBytesBackwardsOnSeek; - private final MutableCounterLong streamBytesRead; - private final MutableCounterLong streamReadOperations; - private final MutableCounterLong streamReadFullyOperations; - private final MutableCounterLong streamReadsIncomplete; - private final MutableCounterLong streamBytesReadInClose; - private final MutableCounterLong streamBytesDiscardedInAbort; private final MutableCounterLong ignoredErrors; private final MutableQuantiles putLatencyQuantile; private final MutableQuantiles throttleRateQuantile; @@ -136,6 +158,9 @@ public class S3AInstrumentation implements Closeable, MetricsSource { private final S3GuardInstrumentation s3GuardInstrumentation = new S3GuardInstrumentation(); + /** + * All the counters to create. + */ private static final Statistic[] COUNTERS_TO_CREATE = { INVOCATION_COPY_FROM_LOCAL_FILE, INVOCATION_CREATE, @@ -154,6 +179,7 @@ public class S3AInstrumentation implements Closeable, MetricsSource { INVOCATION_MKDIRS, INVOCATION_OPEN, INVOCATION_RENAME, + OBJECT_COPY_REQUESTS, OBJECT_DELETE_REQUESTS, OBJECT_LIST_REQUESTS, @@ -164,13 +190,35 @@ public class S3AInstrumentation implements Closeable, MetricsSource { OBJECT_PUT_REQUESTS, OBJECT_PUT_REQUESTS_COMPLETED, OBJECT_SELECT_REQUESTS, + + STREAM_READ_ABORTED, + STREAM_READ_BYTES_DISCARDED_ABORT, + STREAM_READ_CLOSED, + STREAM_READ_CLOSE_BYTES_READ, + STREAM_READ_CLOSE_OPERATIONS, + STREAM_READ_OPENED, + STREAM_READ_BYTES, + STREAM_READ_EXCEPTIONS, + STREAM_READ_FULLY_OPERATIONS, + STREAM_READ_OPERATIONS, + STREAM_READ_OPERATIONS_INCOMPLETE, + STREAM_READ_SEEK_OPERATIONS, + STREAM_READ_SEEK_POLICY_CHANGED, + STREAM_READ_SEEK_BACKWARD_OPERATIONS, + STREAM_READ_SEEK_FORWARD_OPERATIONS, + STREAM_READ_SEEK_BYTES_BACKWARDS, + STREAM_READ_SEEK_BYTES_READ, + STREAM_READ_SEEK_BYTES_SKIPPED, + STREAM_READ_TOTAL_BYTES, STREAM_READ_VERSION_MISMATCHES, + STREAM_WRITE_FAILURES, STREAM_WRITE_BLOCK_UPLOADS, STREAM_WRITE_BLOCK_UPLOADS_COMMITTED, STREAM_WRITE_BLOCK_UPLOADS_ABORTED, STREAM_WRITE_TOTAL_TIME, STREAM_WRITE_TOTAL_DATA, + COMMITTER_COMMITS_CREATED, COMMITTER_COMMITS_COMPLETED, COMMITTER_JOBS_SUCCEEDED, @@ -183,6 +231,7 @@ public class S3AInstrumentation implements Closeable, MetricsSource { COMMITTER_COMMITS_ABORTED, COMMITTER_COMMITS_REVERTED, COMMITTER_MAGIC_FILES_CREATED, + S3GUARD_METADATASTORE_PUT_PATH_REQUEST, S3GUARD_METADATASTORE_INITIALIZATION, S3GUARD_METADATASTORE_RECORD_DELETES, @@ -191,7 +240,11 @@ public class S3AInstrumentation implements Closeable, MetricsSource { S3GUARD_METADATASTORE_RETRY, S3GUARD_METADATASTORE_THROTTLED, S3GUARD_METADATASTORE_AUTHORITATIVE_DIRECTORIES_UPDATED, + STORE_IO_THROTTLED, + STORE_IO_REQUEST, + STORE_IO_RETRY, + DELEGATION_TOKENS_ISSUED, FILES_DELETE_REJECTED, MULTIPART_INSTANTIATED, @@ -217,27 +270,6 @@ public S3AInstrumentation(URI name) { "A unique identifier for the instance", fileSystemInstanceId.toString()); registry.tag(METRIC_TAG_BUCKET, "Hostname from the FS URL", name.getHost()); - streamOpenOperations = counter(STREAM_OPENED); - streamCloseOperations = counter(STREAM_CLOSE_OPERATIONS); - streamClosed = counter(STREAM_CLOSED); - streamAborted = counter(STREAM_ABORTED); - streamSeekOperations = counter(STREAM_SEEK_OPERATIONS); - streamReadExceptions = counter(STREAM_READ_EXCEPTIONS); - streamForwardSeekOperations = - counter(STREAM_FORWARD_SEEK_OPERATIONS); - streamBackwardSeekOperations = - counter(STREAM_BACKWARD_SEEK_OPERATIONS); - streamBytesSkippedOnSeek = counter(STREAM_SEEK_BYTES_SKIPPED); - streamBytesBackwardsOnSeek = - counter(STREAM_SEEK_BYTES_BACKWARDS); - streamBytesRead = counter(STREAM_SEEK_BYTES_READ); - streamReadOperations = counter(STREAM_READ_OPERATIONS); - streamReadFullyOperations = - counter(STREAM_READ_FULLY_OPERATIONS); - streamReadsIncomplete = - counter(STREAM_READ_OPERATIONS_INCOMPLETE); - streamBytesReadInClose = counter(STREAM_CLOSE_BYTES_READ); - streamBytesDiscardedInAbort = counter(STREAM_ABORT_BYTES_DISCARDED); numberOfFilesCreated = counter(FILES_CREATED); numberOfFilesCopied = counter(FILES_COPIED); bytesOfFilesCopied = counter(FILES_COPIED_BYTES); @@ -266,7 +298,7 @@ public S3AInstrumentation(URI name) { @VisibleForTesting public MetricsSystem getMetricsSystem() { - synchronized (metricsSystemLock) { + synchronized (METRICS_SYSTEM_LOCK) { if (metricsSystem == null) { metricsSystem = new MetricsSystemImpl(); metricsSystem.init(METRICS_SYSTEM_NAME); @@ -281,7 +313,7 @@ public MetricsSystem getMetricsSystem() { */ private void registerAsMetricsSource(URI name) { int number; - synchronized(metricsSystemLock) { + synchronized(METRICS_SYSTEM_LOCK) { getMetricsSystem(); metricsSourceActiveCounter++; @@ -503,7 +535,11 @@ public void errorIgnored() { * @param count increment value */ public void incrementCounter(Statistic op, long count) { - MutableCounterLong counter = lookupCounter(op.getSymbol()); + incrementNamedCounter(op.getSymbol(), count); + } + + private void incrementNamedCounter(final String name, final long count) { + MutableCounterLong counter = lookupCounter(name); if (counter != null) { counter.incr(count); } @@ -568,9 +604,11 @@ public void decrementGauge(Statistic op, long count) { /** * Create a stream input statistics instance. * @return the new instance + * @param filesystemStatistics FS Stats. */ - public InputStreamStatistics newInputStreamStatistics() { - return new InputStreamStatistics(); + public S3AInputStreamStatistics newInputStreamStatistics( + final FileSystem.Statistics filesystemStatistics) { + return new InputStreamStatisticsImpl(filesystemStatistics); } /** @@ -586,34 +624,8 @@ public MetastoreInstrumentation getS3GuardInstrumentation() { * Create a new instance of the committer statistics. * @return a new committer statistics instance */ - CommitterStatistics newCommitterStatistics() { - return new CommitterStatistics(); - } - - /** - * Merge in the statistics of a single input stream into - * the filesystem-wide statistics. - * @param statistics stream statistics - */ - private void mergeInputStreamStatistics(InputStreamStatistics statistics) { - streamOpenOperations.incr(statistics.openOperations); - streamCloseOperations.incr(statistics.closeOperations); - streamClosed.incr(statistics.closed); - streamAborted.incr(statistics.aborted); - streamSeekOperations.incr(statistics.seekOperations); - streamReadExceptions.incr(statistics.readExceptions); - streamForwardSeekOperations.incr(statistics.forwardSeekOperations); - streamBytesSkippedOnSeek.incr(statistics.bytesSkippedOnSeek); - streamBackwardSeekOperations.incr(statistics.backwardSeekOperations); - streamBytesBackwardsOnSeek.incr(statistics.bytesBackwardsOnSeek); - streamBytesRead.incr(statistics.bytesRead); - streamReadOperations.incr(statistics.readOperations); - streamReadFullyOperations.incr(statistics.readFullyOperations); - streamReadsIncomplete.incr(statistics.readsIncomplete); - streamBytesReadInClose.incr(statistics.bytesReadInClose); - streamBytesDiscardedInAbort.incr(statistics.bytesDiscardedInAbort); - incrementCounter(STREAM_READ_VERSION_MISMATCHES, - statistics.versionMismatches.get()); + public CommitterStatistics newCommitterStatistics() { + return new CommitterStatisticsImpl(); } @Override @@ -622,14 +634,15 @@ public void getMetrics(MetricsCollector collector, boolean all) { } public void close() { - synchronized (metricsSystemLock) { + synchronized (METRICS_SYSTEM_LOCK) { // it is critical to close each quantile, as they start a scheduled // task in a shared thread pool. putLatencyQuantile.stop(); throttleRateQuantile.stop(); s3GuardThrottleRateQuantile.stop(); metricsSystem.unregisterSource(metricsSourceName); - int activeSources = --metricsSourceActiveCounter; + metricsSourceActiveCounter--; + int activeSources = metricsSourceActiveCounter; if (activeSources == 0) { LOG.debug("Shutting down metrics publisher"); metricsSystem.publishMetricsNow(); @@ -640,36 +653,74 @@ public void close() { } /** - * Statistics updated by an input stream during its actual operation. - * These counters not thread-safe and are for use in a single instance - * of a stream. + * Statistics updated by an S3AInputStream during its actual operation. + *

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

+ * When {@code close()} is called, the final set of numbers are propagated + * to the S3AFileSystem metrics. + * The {@link FileSystem.Statistics} statistics passed in are also + * updated. This ensures that whichever thread calls close() gets the + * total count of bytes read, even if any work is done in other + * threads. + * */ - @InterfaceAudience.Private - @InterfaceStability.Unstable - public final class InputStreamStatistics implements AutoCloseable { - public long openOperations; - public long closeOperations; - public long closed; - public long aborted; - public long seekOperations; - public long readExceptions; - public long forwardSeekOperations; - public long backwardSeekOperations; - public long bytesRead; - public long bytesSkippedOnSeek; - public long bytesBackwardsOnSeek; - public long readOperations; - public long readFullyOperations; - public long readsIncomplete; - public long bytesReadInClose; - public long bytesDiscardedInAbort; - public long policySetCount; - public long inputPolicy; - /** This is atomic so that it can be passed as a reference. */ - private final AtomicLong versionMismatches = new AtomicLong(0); - private InputStreamStatistics mergedStats; - - private InputStreamStatistics() { + private final class InputStreamStatisticsImpl + extends AbstractS3AStatisticsSource + implements S3AInputStreamStatistics { + + /** + * Distance used when incrementing FS stats. + */ + private static final int DISTANCE = 5; + + private final FileSystem.Statistics filesystemStatistics; + + /** + * The statistics from the last merge. + */ + private IOStatisticsSnapshot mergedStats; + + private InputStreamStatisticsImpl( + FileSystem.Statistics filesystemStatistics) { + this.filesystemStatistics = filesystemStatistics; + IOStatisticsStore st = iostatisticsStore() + .withCounters( + StreamStatisticNames.STREAM_READ_ABORTED, + StreamStatisticNames.STREAM_READ_BYTES_DISCARDED_ABORT, + StreamStatisticNames.STREAM_READ_CLOSED, + StreamStatisticNames.STREAM_READ_CLOSE_BYTES_READ, + StreamStatisticNames.STREAM_READ_CLOSE_OPERATIONS, + StreamStatisticNames.STREAM_READ_OPENED, + StreamStatisticNames.STREAM_READ_BYTES, + StreamStatisticNames.STREAM_READ_EXCEPTIONS, + StreamStatisticNames.STREAM_READ_FULLY_OPERATIONS, + StreamStatisticNames.STREAM_READ_OPERATIONS, + StreamStatisticNames.STREAM_READ_OPERATIONS_INCOMPLETE, + StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS, + StreamStatisticNames.STREAM_READ_SEEK_POLICY_CHANGED, + StreamStatisticNames.STREAM_READ_SEEK_BACKWARD_OPERATIONS, + StreamStatisticNames.STREAM_READ_SEEK_FORWARD_OPERATIONS, + StreamStatisticNames.STREAM_READ_SEEK_BYTES_BACKWARDS, + StreamStatisticNames.STREAM_READ_SEEK_BYTES_READ, + StreamStatisticNames.STREAM_READ_SEEK_BYTES_SKIPPED, + StreamStatisticNames.STREAM_READ_TOTAL_BYTES, + StreamStatisticNames.STREAM_READ_VERSION_MISMATCHES) + .withGauges(STREAM_READ_GAUGE_INPUT_POLICY) + .withDurationTracking(ACTION_HTTP_GET_REQUEST) + .build(); + setIOStatistics(st); + // create initial snapshot of merged statistics + mergedStats = snapshotIOStatistics(st); + } + + private long increment(String name) { + return incCounter(name); + } + + private long increment(String name, long value) { + return incCounter(name, value); } /** @@ -677,23 +728,35 @@ private InputStreamStatistics() { * @param negativeOffset how far was the seek? * This is expected to be negative. */ + @Override public void seekBackwards(long negativeOffset) { - seekOperations++; - backwardSeekOperations++; - bytesBackwardsOnSeek -= negativeOffset; + increment(StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS); + increment(StreamStatisticNames.STREAM_READ_SEEK_BACKWARD_OPERATIONS); + increment(StreamStatisticNames.STREAM_READ_SEEK_BYTES_BACKWARDS, + -negativeOffset); } /** * Record a forward seek, adding a seek operation, a forward * seek operation, and any bytes skipped. - * @param skipped number of bytes skipped by reading from the stream. - * If the seek was implemented by a close + reopen, set this to zero. + * @param skipped + * @param bytesRead number of bytes skipped by reading from the stream. + * If the seek was implemented by a close + reopen, set this to zero. */ - public void seekForwards(long skipped) { - seekOperations++; - forwardSeekOperations++; + @Override + public void seekForwards(final long skipped, + long bytesRead) { + increment(StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS); + increment(StreamStatisticNames.STREAM_READ_SEEK_FORWARD_OPERATIONS); if (skipped > 0) { - bytesSkippedOnSeek += skipped; + increment(StreamStatisticNames.STREAM_READ_SEEK_BYTES_SKIPPED, + skipped); + } + if (bytesRead > 0) { + increment(StreamStatisticNames.STREAM_READ_SEEK_BYTES_READ, + bytesRead); + increment(StreamStatisticNames.STREAM_READ_TOTAL_BYTES, + bytesRead); } } @@ -701,10 +764,9 @@ public void seekForwards(long skipped) { * The inner stream was opened. * @return the previous count */ + @Override public long streamOpened() { - long count = openOperations; - openOperations++; - return count; + return increment(StreamStatisticNames.STREAM_READ_OPENED); } /** @@ -714,23 +776,28 @@ public long streamOpened() { * @param remainingInCurrentRequest the number of bytes remaining in * the current request. */ + @Override public void streamClose(boolean abortedConnection, long remainingInCurrentRequest) { - closeOperations++; if (abortedConnection) { - this.aborted++; - bytesDiscardedInAbort += remainingInCurrentRequest; + increment(StreamStatisticNames.STREAM_READ_ABORTED); + increment(StreamStatisticNames.STREAM_READ_BYTES_DISCARDED_ABORT, + remainingInCurrentRequest); } else { - closed++; - bytesReadInClose += remainingInCurrentRequest; + increment(StreamStatisticNames.STREAM_READ_CLOSED); + increment(StreamStatisticNames.STREAM_READ_CLOSE_BYTES_READ, + remainingInCurrentRequest); + increment(StreamStatisticNames.STREAM_READ_TOTAL_BYTES, + remainingInCurrentRequest); } } /** * An ignored stream read exception was received. */ + @Override public void readException() { - readExceptions++; + increment(StreamStatisticNames.STREAM_READ_EXCEPTIONS); } /** @@ -738,9 +805,11 @@ public void readException() { * no-op if the argument is negative. * @param bytes number of bytes read */ + @Override public void bytesRead(long bytes) { if (bytes > 0) { - bytesRead += bytes; + increment(StreamStatisticNames.STREAM_READ_BYTES, bytes); + increment(StreamStatisticNames.STREAM_READ_TOTAL_BYTES, bytes); } } @@ -749,8 +818,9 @@ public void bytesRead(long bytes) { * @param pos starting position of the read * @param len length of bytes to read */ + @Override public void readOperationStarted(long pos, long len) { - readOperations++; + increment(StreamStatisticNames.STREAM_READ_OPERATIONS); } /** @@ -759,8 +829,9 @@ public void readOperationStarted(long pos, long len) { * @param pos starting position of the read * @param len length of bytes to read */ + @Override public void readFullyOperationStarted(long pos, long len) { - readFullyOperations++; + increment(StreamStatisticNames.STREAM_READ_FULLY_OPERATIONS); } /** @@ -768,9 +839,10 @@ public void readFullyOperationStarted(long pos, long len) { * @param requested number of requested bytes * @param actual the actual number of bytes */ + @Override public void readOperationCompleted(int requested, int actual) { if (requested > actual) { - readsIncomplete++; + increment(StreamStatisticNames.STREAM_READ_OPERATIONS_INCOMPLETE); } } @@ -787,17 +859,22 @@ public void close() { * The input policy has been switched. * @param updatedPolicy enum value of new policy. */ + @Override public void inputPolicySet(int updatedPolicy) { - policySetCount++; - inputPolicy = updatedPolicy; + increment(StreamStatisticNames.STREAM_READ_SEEK_POLICY_CHANGED); + getIOStatistics().setGauge(STREAM_READ_GAUGE_INPUT_POLICY, updatedPolicy); } /** - * Get a reference to the version mismatch counter. - * @return a counter which can be incremented. + * The change tracker increments {@code versionMismatches} on any + * mismatch. + * @return change tracking. */ - public AtomicLong getVersionMismatchCounter() { - return versionMismatches; + @Override + public ChangeTrackerStatistics getChangeTrackerStatistics() { + return new CountingChangeTracker( + getIOStatistics().getCounterReference( + StreamStatisticNames.STREAM_READ_VERSION_MISMATCHES)); } /** @@ -811,204 +888,321 @@ public AtomicLong getVersionMismatchCounter() { public String toString() { final StringBuilder sb = new StringBuilder( "StreamStatistics{"); - sb.append("OpenOperations=").append(openOperations); - sb.append(", CloseOperations=").append(closeOperations); - sb.append(", Closed=").append(closed); - sb.append(", Aborted=").append(aborted); - sb.append(", SeekOperations=").append(seekOperations); - sb.append(", ReadExceptions=").append(readExceptions); - sb.append(", ForwardSeekOperations=") - .append(forwardSeekOperations); - sb.append(", BackwardSeekOperations=") - .append(backwardSeekOperations); - sb.append(", BytesSkippedOnSeek=").append(bytesSkippedOnSeek); - sb.append(", BytesBackwardsOnSeek=").append(bytesBackwardsOnSeek); - sb.append(", BytesRead=").append(bytesRead); - sb.append(", BytesRead excluding skipped=") - .append(bytesRead - bytesSkippedOnSeek); - sb.append(", ReadOperations=").append(readOperations); - sb.append(", ReadFullyOperations=").append(readFullyOperations); - sb.append(", ReadsIncomplete=").append(readsIncomplete); - sb.append(", BytesReadInClose=").append(bytesReadInClose); - sb.append(", BytesDiscardedInAbort=").append(bytesDiscardedInAbort); - sb.append(", InputPolicy=").append(inputPolicy); - sb.append(", InputPolicySetCount=").append(policySetCount); - sb.append(", versionMismatches=").append(versionMismatches.get()); + sb.append(IOStatisticsLogging.ioStatisticsToString( + getIOStatistics())); sb.append('}'); return sb.toString(); } + @Override + public void unbuffered() { + merge(false); + } + /** * Merge the statistics into the filesystem's instrumentation instance. * Takes a diff between the current version of the stats and the * version of the stats when merge was last called, and merges the diff * into the instrumentation instance. Used to periodically merge the - * stats into the fs-wide stats. Behavior is undefined if called on a - * closed instance. + * stats into the fs-wide stats. + *

+ * Behavior is undefined if called on a closed instance. */ - void merge(boolean isClosed) { - if (mergedStats != null) { - mergeInputStreamStatistics(diff(mergedStats)); - } else { - mergeInputStreamStatistics(this); - } - // If stats are closed, no need to create another copy - if (!isClosed) { - mergedStats = copy(); + private void merge(boolean isClosed) { + + LOG.debug("Merging statistics into FS statistics in {}: {}", + (isClosed ? "close()" : "unbuffer()"), + demandStringifyIOStatistics(getIOStatistics())); + mergeInputStreamStatistics(); + mergedStats = snapshotIOStatistics(getIOStatistics()); + + if (isClosed) { + // stream is being closed. + // increment the filesystem statistics for this thread. + if (filesystemStatistics != null) { + long t = getTotalBytesRead(); + filesystemStatistics.incrementBytesRead(t); + filesystemStatistics.incrementBytesReadByDistance(DISTANCE, t); + } } } /** - * Returns a diff between this {@link InputStreamStatistics} instance and - * the given {@link InputStreamStatistics} instance. + * Propagate a counter from the instance-level statistics + * to the S3A instrumentation, subtracting the previous marged value. + * @param name statistic to promote */ - private InputStreamStatistics diff(InputStreamStatistics inputStats) { - InputStreamStatistics diff = new InputStreamStatistics(); - diff.openOperations = openOperations - inputStats.openOperations; - diff.closeOperations = closeOperations - inputStats.closeOperations; - diff.closed = closed - inputStats.closed; - diff.aborted = aborted - inputStats.aborted; - diff.seekOperations = seekOperations - inputStats.seekOperations; - diff.readExceptions = readExceptions - inputStats.readExceptions; - diff.forwardSeekOperations = - forwardSeekOperations - inputStats.forwardSeekOperations; - diff.backwardSeekOperations = - backwardSeekOperations - inputStats.backwardSeekOperations; - diff.bytesRead = bytesRead - inputStats.bytesRead; - diff.bytesSkippedOnSeek = - bytesSkippedOnSeek - inputStats.bytesSkippedOnSeek; - diff.bytesBackwardsOnSeek = - bytesBackwardsOnSeek - inputStats.bytesBackwardsOnSeek; - diff.readOperations = readOperations - inputStats.readOperations; - diff.readFullyOperations = - readFullyOperations - inputStats.readFullyOperations; - diff.readsIncomplete = readsIncomplete - inputStats.readsIncomplete; - diff.bytesReadInClose = bytesReadInClose - inputStats.bytesReadInClose; - diff.bytesDiscardedInAbort = - bytesDiscardedInAbort - inputStats.bytesDiscardedInAbort; - diff.policySetCount = policySetCount - inputStats.policySetCount; - diff.inputPolicy = inputPolicy - inputStats.inputPolicy; - diff.versionMismatches.set(versionMismatches.longValue() - - inputStats.versionMismatches.longValue()); - return diff; + void promoteIOCounter(String name) { + incrementNamedCounter(name, + lookupCounterValue(name) + - mergedStats.counters().get(name)); } /** - * Returns a new {@link InputStreamStatistics} instance with all the same - * values as this {@link InputStreamStatistics}. + * Merge in the statistics of a single input stream into + * the filesystem-wide statistics. */ - private InputStreamStatistics copy() { - InputStreamStatistics copy = new InputStreamStatistics(); - copy.openOperations = openOperations; - copy.closeOperations = closeOperations; - copy.closed = closed; - copy.aborted = aborted; - copy.seekOperations = seekOperations; - copy.readExceptions = readExceptions; - copy.forwardSeekOperations = forwardSeekOperations; - copy.backwardSeekOperations = backwardSeekOperations; - copy.bytesRead = bytesRead; - copy.bytesSkippedOnSeek = bytesSkippedOnSeek; - copy.bytesBackwardsOnSeek = bytesBackwardsOnSeek; - copy.readOperations = readOperations; - copy.readFullyOperations = readFullyOperations; - copy.readsIncomplete = readsIncomplete; - copy.bytesReadInClose = bytesReadInClose; - copy.bytesDiscardedInAbort = bytesDiscardedInAbort; - copy.policySetCount = policySetCount; - copy.inputPolicy = inputPolicy; - return copy; + private void mergeInputStreamStatistics() { + // iterate through all the counters + getIOStatistics().counters() + .keySet().stream() + .forEach(e -> promoteIOCounter(e)); + } + + @Override + public long getCloseOperations() { + return lookupCounterValue( + StreamStatisticNames.STREAM_READ_CLOSE_OPERATIONS); + } + + @Override + public long getClosed() { + return lookupCounterValue(StreamStatisticNames.STREAM_READ_CLOSED); + } + + @Override + public long getAborted() { + return lookupCounterValue(StreamStatisticNames.STREAM_READ_ABORTED); + } + + @Override + public long getForwardSeekOperations() { + return lookupCounterValue( + StreamStatisticNames.STREAM_READ_SEEK_FORWARD_OPERATIONS); + } + + @Override + public long getBackwardSeekOperations() { + return lookupCounterValue( + StreamStatisticNames.STREAM_READ_SEEK_BACKWARD_OPERATIONS); + } + + @Override + public long getBytesRead() { + return lookupCounterValue(StreamStatisticNames.STREAM_READ_BYTES); + } + + @Override + public long getTotalBytesRead() { + return lookupCounterValue(StreamStatisticNames.STREAM_READ_TOTAL_BYTES); + } + + @Override + public long getBytesSkippedOnSeek() { + return lookupCounterValue( + StreamStatisticNames.STREAM_READ_SEEK_BYTES_SKIPPED); + } + + @Override + public long getBytesBackwardsOnSeek() { + return lookupCounterValue( + StreamStatisticNames.STREAM_READ_SEEK_BYTES_BACKWARDS); + } + + @Override + public long getBytesReadInClose() { + return lookupCounterValue( + StreamStatisticNames.STREAM_READ_CLOSE_BYTES_READ); + } + + @Override + public long getBytesDiscardedInAbort() { + return lookupCounterValue( + StreamStatisticNames.STREAM_READ_BYTES_DISCARDED_ABORT); } + + @Override + public long getOpenOperations() { + return lookupCounterValue(StreamStatisticNames.STREAM_READ_OPENED); + } + + @Override + public long getSeekOperations() { + return lookupCounterValue( + StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS); + } + + @Override + public long getReadExceptions() { + return lookupCounterValue( + StreamStatisticNames.STREAM_READ_EXCEPTIONS); + } + + @Override + public long getReadOperations() { + return lookupCounterValue(StreamStatisticNames.STREAM_READ_CLOSED); + } + + @Override + public long getReadFullyOperations() { + return lookupCounterValue( + StreamStatisticNames.STREAM_READ_FULLY_OPERATIONS); + } + + @Override + public long getReadsIncomplete() { + return lookupCounterValue( + StreamStatisticNames.STREAM_READ_OPERATIONS_INCOMPLETE); + } + + @Override + public long getPolicySetCount() { + return lookupCounterValue( + StreamStatisticNames.STREAM_READ_SEEK_POLICY_CHANGED); + } + + @Override + public long getVersionMismatches() { + return lookupCounterValue( + StreamStatisticNames.STREAM_READ_VERSION_MISMATCHES); + } + + @Override + public long getInputPolicy() { + return getIOStatistics().gauges().get(STREAM_READ_GAUGE_INPUT_POLICY); + } + + @Override + public DurationTracker initiateGetRequest() { + return trackDuration(ACTION_HTTP_GET_REQUEST); + } + } /** * Create a stream output statistics instance. * @return the new instance */ - OutputStreamStatistics newOutputStreamStatistics(Statistics statistics) { - return new OutputStreamStatistics(statistics); + public BlockOutputStreamStatistics newOutputStreamStatistics( + FileSystem.Statistics filesystemStatistics) { + return new BlockOutputStreamStatisticsImpl(filesystemStatistics); } /** * Merge in the statistics of a single output stream into * the filesystem-wide statistics. - * @param statistics stream statistics + * @param source stream statistics */ - private void mergeOutputStreamStatistics(OutputStreamStatistics statistics) { - incrementCounter(STREAM_WRITE_TOTAL_TIME, statistics.totalUploadDuration()); - incrementCounter(STREAM_WRITE_QUEUE_DURATION, statistics.queueDuration); - incrementCounter(STREAM_WRITE_TOTAL_DATA, statistics.bytesUploaded); + private void mergeOutputStreamStatistics( + BlockOutputStreamStatisticsImpl source) { + incrementCounter(STREAM_WRITE_TOTAL_TIME, source.totalUploadDuration()); + incrementCounter(STREAM_WRITE_QUEUE_DURATION, source.queueDuration); + incrementCounter(STREAM_WRITE_TOTAL_DATA, source.bytesUploaded); incrementCounter(STREAM_WRITE_BLOCK_UPLOADS, - statistics.blockUploadsCompleted); + source.blockUploadsCompleted); + incrementCounter(STREAM_WRITE_FAILURES, + source.lookupCounterValue( + StreamStatisticNames.STREAM_WRITE_EXCEPTIONS)); } /** * Statistics updated by an output stream during its actual operation. - * Some of these stats may be relayed. However, as block upload is - * spans multiple + *

+ * Some of these stats are propagated to any passed in + * {@link FileSystem.Statistics} instance; this is only done + * in close() for better cross-thread accounting. */ - @InterfaceAudience.Private - @InterfaceStability.Unstable - public final class OutputStreamStatistics implements Closeable { - private final AtomicLong blocksSubmitted = new AtomicLong(0); + private final class BlockOutputStreamStatisticsImpl + extends AbstractS3AStatisticsSource + implements BlockOutputStreamStatistics { + private final AtomicLong blocksInQueue = new AtomicLong(0); private final AtomicLong blocksActive = new AtomicLong(0); private final AtomicLong blockUploadsCompleted = new AtomicLong(0); - private final AtomicLong blockUploadsFailed = new AtomicLong(0); private final AtomicLong bytesPendingUpload = new AtomicLong(0); - private final AtomicLong bytesUploaded = new AtomicLong(0); + private final AtomicLong bytesWritten; + private final AtomicLong bytesUploaded; private final AtomicLong transferDuration = new AtomicLong(0); private final AtomicLong queueDuration = new AtomicLong(0); - private final AtomicLong exceptionsInMultipartFinalize = new AtomicLong(0); private final AtomicInteger blocksAllocated = new AtomicInteger(0); private final AtomicInteger blocksReleased = new AtomicInteger(0); - private Statistics statistics; + private final FileSystem.Statistics filesystemStatistics; + + private BlockOutputStreamStatisticsImpl( + @Nullable FileSystem.Statistics filesystemStatistics) { + this.filesystemStatistics = filesystemStatistics; + IOStatisticsStore st = iostatisticsStore() + .withCounters( + StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS, + StreamStatisticNames.STREAM_WRITE_BYTES, + StreamStatisticNames.STREAM_WRITE_EXCEPTIONS, + StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING, + STREAM_WRITE_TOTAL_TIME.getSymbol(), + STREAM_WRITE_QUEUE_DURATION.getSymbol(), + STREAM_WRITE_TOTAL_DATA.getSymbol(), + STREAM_WRITE_FAILURES.getSymbol(), + STREAM_WRITE_EXCEPTIONS_COMPLETING_UPLOADS.getSymbol() + ) + .withGauges( + STREAM_WRITE_BLOCK_UPLOADS_PENDING.getSymbol(), + STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING.getSymbol()) + .withDurationTracking(ACTION_EXECUTOR_ACQUIRED) + .build(); + setIOStatistics(st); + // these are extracted to avoid lookups on heavily used counters. + bytesUploaded = st.getCounterReference( + STREAM_WRITE_TOTAL_DATA.getSymbol()); + bytesWritten = st.getCounterReference( + StreamStatisticNames.STREAM_WRITE_BYTES); + } - public OutputStreamStatistics(Statistics statistics){ - this.statistics = statistics; + /** + * Increment the Statistic gauge and the local IOStats + * equivalent. + * @param statistic statistic + * @param v value. + * @return local IOStatistic value + */ + private long incAllGauges(Statistic statistic, long v) { + incrementGauge(statistic, v); + return incGauge(statistic.getSymbol(), v); } /** * A block has been allocated. */ - void blockAllocated() { + @Override + public void blockAllocated() { blocksAllocated.incrementAndGet(); } /** * A block has been released. */ - void blockReleased() { + @Override + public void blockReleased() { blocksReleased.incrementAndGet(); } /** * Block is queued for upload. */ - void blockUploadQueued(int blockSize) { - blocksSubmitted.incrementAndGet(); + @Override + public void blockUploadQueued(int blockSize) { + incCounter(StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS); + blocksInQueue.incrementAndGet(); bytesPendingUpload.addAndGet(blockSize); - incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_PENDING, 1); - incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING, blockSize); + incAllGauges(STREAM_WRITE_BLOCK_UPLOADS_PENDING, 1); + incAllGauges(STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING, blockSize); } /** Queued block has been scheduled for upload. */ - void blockUploadStarted(long duration, int blockSize) { + @Override + public void blockUploadStarted(long duration, int blockSize) { queueDuration.addAndGet(duration); blocksInQueue.decrementAndGet(); - blocksActive.incrementAndGet(); - incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_PENDING, -1); - incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_ACTIVE, 1); + incAllGauges(STREAM_WRITE_BLOCK_UPLOADS_PENDING, -1); + incAllGauges(STREAM_WRITE_BLOCK_UPLOADS_ACTIVE, 1); } /** A block upload has completed. */ - void blockUploadCompleted(long duration, int blockSize) { - this.transferDuration.addAndGet(duration); - incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_ACTIVE, -1); - blocksActive.decrementAndGet(); + @Override + public void blockUploadCompleted(long duration, int blockSize) { + transferDuration.addAndGet(duration); + incAllGauges(STREAM_WRITE_BLOCK_UPLOADS_ACTIVE, -1); blockUploadsCompleted.incrementAndGet(); } @@ -1017,14 +1211,15 @@ void blockUploadCompleted(long duration, int blockSize) { * A final transfer completed event is still expected, so this * does not decrement the active block counter. */ - void blockUploadFailed(long duration, int blockSize) { - blockUploadsFailed.incrementAndGet(); + @Override + public void blockUploadFailed(long duration, int blockSize) { + incCounter(StreamStatisticNames.STREAM_WRITE_EXCEPTIONS); } /** Intermediate report of bytes uploaded. */ - void bytesTransferred(long byteCount) { + @Override + public void bytesTransferred(long byteCount) { bytesUploaded.addAndGet(byteCount); - statistics.incrementBytesWritten(byteCount); bytesPendingUpload.addAndGet(-byteCount); incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING, -byteCount); } @@ -1033,23 +1228,29 @@ void bytesTransferred(long byteCount) { * Note exception in a multipart complete. * @param count count of exceptions */ - void exceptionInMultipartComplete(int count) { + @Override + public void exceptionInMultipartComplete(int count) { if (count > 0) { - exceptionsInMultipartFinalize.addAndGet(count); + incCounter( + STREAM_WRITE_EXCEPTIONS_COMPLETING_UPLOADS.getSymbol(), + count); } } /** * Note an exception in a multipart abort. */ - void exceptionInMultipartAbort() { - exceptionsInMultipartFinalize.incrementAndGet(); + @Override + public void exceptionInMultipartAbort() { + incCounter( + STREAM_WRITE_EXCEPTIONS_COMPLETING_UPLOADS.getSymbol()); } /** * Get the number of bytes pending upload. * @return the number of bytes in the pending upload state. */ + @Override public long getBytesPendingUpload() { return bytesPendingUpload.get(); } @@ -1059,6 +1260,7 @@ public long getBytesPendingUpload() { * to be called at the end of the write. * @param size size in bytes */ + @Override public void commitUploaded(long size) { incrementCounter(COMMITTER_BYTES_UPLOADED, size); } @@ -1074,11 +1276,18 @@ public void close() { " as pending upload in {}", this); } mergeOutputStreamStatistics(this); + // and patch the FS statistics. + // provided the stream is closed in the worker thread, this will + // ensure that the thread-specific worker stats are updated. + if (filesystemStatistics != null) { + filesystemStatistics.incrementBytesWritten(bytesUploaded.get()); + } } long averageQueueTime() { - return blocksSubmitted.get() > 0 ? - (queueDuration.get() / blocksSubmitted.get()) : 0; + long l = getCounterValue(StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS); + return l > 0 ? + (queueDuration.get() / l) : 0; } double effectiveBandwidth() { @@ -1091,40 +1300,59 @@ long totalUploadDuration() { return queueDuration.get() + transferDuration.get(); } - public int blocksAllocated() { + @Override + public int getBlocksAllocated() { return blocksAllocated.get(); } - public int blocksReleased() { + @Override + public int getBlocksReleased() { return blocksReleased.get(); } /** - * Get counters of blocks actively allocated; my be inaccurate + * Get counters of blocks actively allocated; may be inaccurate * if the numbers change during the (non-synchronized) calculation. * @return the number of actively allocated blocks. */ - public int blocksActivelyAllocated() { + @Override + public int getBlocksActivelyAllocated() { return blocksAllocated.get() - blocksReleased.get(); } + /** + * Record bytes written. + * @param count number of bytes + */ + @Override + public void writeBytes(long count) { + bytesWritten.addAndGet(count); + } + + /** + * Get the current count of bytes written. + * @return the counter value. + */ + @Override + public long getBytesWritten() { + return bytesWritten.get(); + } @Override public String toString() { final StringBuilder sb = new StringBuilder( "OutputStreamStatistics{"); - sb.append("blocksSubmitted=").append(blocksSubmitted); + sb.append(getIOStatistics().toString()); sb.append(", blocksInQueue=").append(blocksInQueue); sb.append(", blocksActive=").append(blocksActive); sb.append(", blockUploadsCompleted=").append(blockUploadsCompleted); - sb.append(", blockUploadsFailed=").append(blockUploadsFailed); sb.append(", bytesPendingUpload=").append(bytesPendingUpload); sb.append(", bytesUploaded=").append(bytesUploaded); + sb.append(", bytesWritten=").append(bytesWritten); sb.append(", blocksAllocated=").append(blocksAllocated); sb.append(", blocksReleased=").append(blocksReleased); - sb.append(", blocksActivelyAllocated=").append(blocksActivelyAllocated()); - sb.append(", exceptionsInMultipartFinalize=").append( - exceptionsInMultipartFinalize); + sb.append(", blocksActivelyAllocated=") + .append(getBlocksActivelyAllocated()); sb.append(", transferDuration=").append(transferDuration).append(" ms"); sb.append(", queueDuration=").append(queueDuration).append(" ms"); sb.append(", averageQueueTime=").append(averageQueueTime()).append(" ms"); @@ -1199,60 +1427,92 @@ public void entryAdded(final long durationNanos) { } /** - * Instrumentation exported to S3Guard Committers. + * Instrumentation exported to S3A Committers. */ - @InterfaceAudience.Private - @InterfaceStability.Unstable - public final class CommitterStatistics { + private final class CommitterStatisticsImpl + extends AbstractS3AStatisticsSource + implements CommitterStatistics { + + private CommitterStatisticsImpl() { + IOStatisticsStore st = iostatisticsStore() + .withCounters( + COMMITTER_BYTES_COMMITTED.getSymbol(), + COMMITTER_BYTES_UPLOADED.getSymbol(), + COMMITTER_COMMITS_CREATED.getSymbol(), + COMMITTER_COMMITS_ABORTED.getSymbol(), + COMMITTER_COMMITS_COMPLETED.getSymbol(), + COMMITTER_COMMITS_FAILED.getSymbol(), + COMMITTER_COMMITS_REVERTED.getSymbol(), + COMMITTER_JOBS_FAILED.getSymbol(), + COMMITTER_JOBS_SUCCEEDED.getSymbol(), + COMMITTER_TASKS_FAILED.getSymbol(), + COMMITTER_TASKS_SUCCEEDED.getSymbol()) + .build(); + setIOStatistics(st); + } + + private long increment(Statistic stat, long value) { + incrementCounter(stat, value); + return incCounter(stat.getSymbol(), value); + } /** A commit has been created. */ + @Override public void commitCreated() { - incrementCounter(COMMITTER_COMMITS_CREATED, 1); + increment(COMMITTER_COMMITS_CREATED, 1); } /** * Data has been uploaded to be committed in a subsequent operation. * @param size size in bytes */ + @Override public void commitUploaded(long size) { - incrementCounter(COMMITTER_BYTES_UPLOADED, size); + increment(COMMITTER_BYTES_UPLOADED, size); } /** * A commit has been completed. * @param size size in bytes */ + @Override public void commitCompleted(long size) { - incrementCounter(COMMITTER_COMMITS_COMPLETED, 1); - incrementCounter(COMMITTER_BYTES_COMMITTED, size); + increment(COMMITTER_COMMITS_COMPLETED, 1); + increment(COMMITTER_BYTES_COMMITTED, size); } /** A commit has been aborted. */ + @Override public void commitAborted() { - incrementCounter(COMMITTER_COMMITS_ABORTED, 1); + increment(COMMITTER_COMMITS_ABORTED, 1); } + @Override public void commitReverted() { - incrementCounter(COMMITTER_COMMITS_REVERTED, 1); + increment(COMMITTER_COMMITS_REVERTED, 1); } + @Override public void commitFailed() { - incrementCounter(COMMITTER_COMMITS_FAILED, 1); + increment(COMMITTER_COMMITS_FAILED, 1); } + @Override public void taskCompleted(boolean success) { - incrementCounter( + increment( success ? COMMITTER_TASKS_SUCCEEDED : COMMITTER_TASKS_FAILED, 1); } + @Override public void jobCompleted(boolean success) { - incrementCounter( + increment( success ? COMMITTER_JOBS_SUCCEEDED : COMMITTER_JOBS_FAILED, 1); } + } /** @@ -1260,23 +1520,28 @@ public void jobCompleted(boolean success) { * @return an instance of delegation token statistics */ public DelegationTokenStatistics newDelegationTokenStatistics() { - return new DelegationTokenStatistics(); + return new DelegationTokenStatisticsImpl(); } /** * Instrumentation exported to S3A Delegation Token support. */ - @InterfaceAudience.Private - @InterfaceStability.Unstable - public final class DelegationTokenStatistics { + private final class DelegationTokenStatisticsImpl implements + DelegationTokenStatistics { - private DelegationTokenStatistics() { + private DelegationTokenStatisticsImpl() { } /** A token has been issued. */ + @Override public void tokenIssued() { incrementCounter(DELEGATION_TOKENS_ISSUED, 1); } + + @Override + public DurationTracker trackDuration(final String key, final int count) { + return stubDurationTracker(); + } } /** @@ -1373,4 +1638,5 @@ public Map getMap() { return map; } } + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOpContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOpContext.java index 4e0aac5138eea..f39d4731f8689 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOpContext.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOpContext.java @@ -24,19 +24,23 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.s3a.impl.ActiveOperationContext; +import org.apache.hadoop.fs.s3a.impl.statistics.S3AStatisticsContext; /** - * Base class for operation context struct passed through codepaths for main + * Class for operation context struct passed through codepaths for main * S3AFileSystem operations. * Anything op-specific should be moved to a subclass of this. + * + * This was originally a base class, but {@link ActiveOperationContext} was + * created to be more minimal and cover many more operation type. */ -@SuppressWarnings("visibilitymodifier") // I want a struct of finals, for real. -public class S3AOpContext { +@SuppressWarnings("visibilitymodifier") +public class S3AOpContext extends ActiveOperationContext { final boolean isS3GuardEnabled; final Invoker invoker; @Nullable final FileSystem.Statistics stats; - final S3AInstrumentation instrumentation; @Nullable final Invoker s3guardInvoker; /** FileStatus for "destination" path being operated on. */ @@ -53,9 +57,14 @@ public class S3AOpContext { * @param dstFileStatus file status from existence check */ public S3AOpContext(boolean isS3GuardEnabled, Invoker invoker, - Invoker s3guardInvoker, @Nullable FileSystem.Statistics stats, - S3AInstrumentation instrumentation, FileStatus dstFileStatus) { + @Nullable Invoker s3guardInvoker, + @Nullable FileSystem.Statistics stats, + S3AStatisticsContext instrumentation, + FileStatus dstFileStatus) { + super(newOperationId(), + instrumentation, + null); Preconditions.checkNotNull(invoker, "Null invoker arg"); Preconditions.checkNotNull(instrumentation, "Null instrumentation arg"); Preconditions.checkNotNull(dstFileStatus, "Null dstFileStatus arg"); @@ -65,7 +74,6 @@ public S3AOpContext(boolean isS3GuardEnabled, Invoker invoker, this.invoker = invoker; this.s3guardInvoker = s3guardInvoker; this.stats = stats; - this.instrumentation = instrumentation; this.dstFileStatus = dstFileStatus; } @@ -77,8 +85,10 @@ public S3AOpContext(boolean isS3GuardEnabled, Invoker invoker, * @param instrumentation instrumentation to use * @param dstFileStatus file status from existence check */ - public S3AOpContext(boolean isS3GuardEnabled, Invoker invoker, - @Nullable FileSystem.Statistics stats, S3AInstrumentation instrumentation, + public S3AOpContext(boolean isS3GuardEnabled, + Invoker invoker, + @Nullable FileSystem.Statistics stats, + S3AStatisticsContext instrumentation, FileStatus dstFileStatus) { this(isS3GuardEnabled, invoker, null, stats, instrumentation, dstFileStatus); @@ -97,10 +107,6 @@ public FileSystem.Statistics getStats() { return stats; } - public S3AInstrumentation getInstrumentation() { - return instrumentation; - } - @Nullable public Invoker getS3guardInvoker() { return s3guardInvoker; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java index a7317c945127c..af92d61d713f1 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java @@ -22,6 +22,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; +import org.apache.hadoop.fs.s3a.impl.statistics.S3AStatisticsContext; import javax.annotation.Nullable; @@ -60,8 +61,8 @@ public class S3AReadOpContext extends S3AOpContext { * @param isS3GuardEnabled true iff S3Guard is enabled. * @param invoker invoker for normal retries. * @param s3guardInvoker S3Guard-specific retry invoker. - * @param stats statistics (may be null) - * @param instrumentation FS instrumentation + * @param stats Fileystem statistics (may be null) + * @param instrumentation statistics context * @param dstFileStatus target file status * @param inputPolicy the input policy * @param readahead readahead for GET operations/skip, etc. @@ -71,13 +72,14 @@ public S3AReadOpContext( final Path path, boolean isS3GuardEnabled, Invoker invoker, - Invoker s3guardInvoker, + @Nullable Invoker s3guardInvoker, @Nullable FileSystem.Statistics stats, - S3AInstrumentation instrumentation, + S3AStatisticsContext instrumentation, FileStatus dstFileStatus, S3AInputPolicy inputPolicy, ChangeDetectionPolicy changeDetectionPolicy, final long readahead) { + super(isS3GuardEnabled, invoker, s3guardInvoker, stats, instrumentation, dstFileStatus); this.path = checkNotNull(path); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java index 1d399505f5823..79a255bc80e8a 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java @@ -47,6 +47,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.util.functional.RemoteIterators; import org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider; import org.apache.hadoop.fs.s3a.impl.NetworkBinding; import org.apache.hadoop.fs.s3native.S3xLoginHelper; @@ -1445,12 +1446,7 @@ public interface LocatedFileStatusMap { public static long applyLocatedFiles( RemoteIterator iterator, CallOnLocatedFileStatus eval) throws IOException { - long count = 0; - while (iterator.hasNext()) { - count++; - eval.call(iterator.next()); - } - return count; + return RemoteIterators.foreach(iterator, eval::call); } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java index e0a1d780ccf5f..0514502c91580 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java @@ -26,6 +26,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.s3a.impl.statistics.StatisticsFromAwsSdk; /** * Factory for creation of {@link AmazonS3} client instances. @@ -41,12 +42,14 @@ public interface S3ClientFactory { * @param bucket Optional bucket to use to look up per-bucket proxy secrets * @param credentialSet credentials to use * @param userAgentSuffix optional suffix for the UA field. + * @param statisticsFromAwsSdk binding for AWS stats - may be null * @return S3 client * @throws IOException IO problem */ AmazonS3 createS3Client(URI name, String bucket, AWSCredentialsProvider credentialSet, - String userAgentSuffix) throws IOException; + String userAgentSuffix, + StatisticsFromAwsSdk statisticsFromAwsSdk) throws IOException; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java index 8153169054ba9..fbc7a981552cf 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java @@ -18,7 +18,9 @@ package org.apache.hadoop.fs.s3a; -import org.apache.hadoop.fs.StorageStatistics.CommonStatisticNames; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.statistics.StoreStatisticNames; +import org.apache.hadoop.fs.statistics.StreamStatisticNames; import java.util.HashMap; import java.util.Map; @@ -26,8 +28,13 @@ /** * Statistic which are collected in S3A. * These statistics are available at a low level in {@link S3AStorageStatistics} - * and as metrics in {@link S3AInstrumentation} + * and as metrics in {@link S3AInstrumentation}. + *

+ * Where possible, stream names come from {@link StreamStatisticNames} + * and {@link StoreStatisticNames} + * */ +@InterfaceStability.Unstable public enum Statistic { DIRECTORIES_CREATED("directories_created", @@ -49,117 +56,192 @@ public enum Statistic { FAKE_DIRECTORIES_DELETED("fake_directories_deleted", "Total number of fake directory deletes submitted to object store."), IGNORED_ERRORS("ignored_errors", "Errors caught and ignored"), - INVOCATION_COPY_FROM_LOCAL_FILE(CommonStatisticNames.OP_COPY_FROM_LOCAL_FILE, + INVOCATION_COPY_FROM_LOCAL_FILE( + StoreStatisticNames.OP_COPY_FROM_LOCAL_FILE, "Calls of copyFromLocalFile()"), - INVOCATION_CREATE(CommonStatisticNames.OP_CREATE, + INVOCATION_CREATE( + StoreStatisticNames.OP_CREATE, "Calls of create()"), - INVOCATION_CREATE_NON_RECURSIVE(CommonStatisticNames.OP_CREATE_NON_RECURSIVE, + INVOCATION_CREATE_NON_RECURSIVE( + StoreStatisticNames.OP_CREATE_NON_RECURSIVE, "Calls of createNonRecursive()"), - INVOCATION_DELETE(CommonStatisticNames.OP_DELETE, + INVOCATION_DELETE( + StoreStatisticNames.OP_DELETE, "Calls of delete()"), - INVOCATION_EXISTS(CommonStatisticNames.OP_EXISTS, + INVOCATION_EXISTS( + StoreStatisticNames.OP_EXISTS, "Calls of exists()"), - INVOCATION_GET_DELEGATION_TOKEN(CommonStatisticNames.OP_GET_DELEGATION_TOKEN, + INVOCATION_GET_DELEGATION_TOKEN( + StoreStatisticNames.OP_GET_DELEGATION_TOKEN, "Calls of getDelegationToken()"), - INVOCATION_GET_FILE_CHECKSUM(CommonStatisticNames.OP_GET_FILE_CHECKSUM, + INVOCATION_GET_FILE_CHECKSUM( + StoreStatisticNames.OP_GET_FILE_CHECKSUM, "Calls of getFileChecksum()"), - INVOCATION_GET_FILE_STATUS(CommonStatisticNames.OP_GET_FILE_STATUS, + INVOCATION_GET_FILE_STATUS( + StoreStatisticNames.OP_GET_FILE_STATUS, "Calls of getFileStatus()"), - INVOCATION_GLOB_STATUS(CommonStatisticNames.OP_GLOB_STATUS, + INVOCATION_GLOB_STATUS( + StoreStatisticNames.OP_GLOB_STATUS, "Calls of globStatus()"), - INVOCATION_IS_DIRECTORY(CommonStatisticNames.OP_IS_DIRECTORY, + INVOCATION_IS_DIRECTORY( + StoreStatisticNames.OP_IS_DIRECTORY, "Calls of isDirectory()"), - INVOCATION_IS_FILE(CommonStatisticNames.OP_IS_FILE, + INVOCATION_IS_FILE( + StoreStatisticNames.OP_IS_FILE, "Calls of isFile()"), - INVOCATION_LIST_FILES(CommonStatisticNames.OP_LIST_FILES, + INVOCATION_LIST_FILES( + StoreStatisticNames.OP_LIST_FILES, "Calls of listFiles()"), - INVOCATION_LIST_LOCATED_STATUS(CommonStatisticNames.OP_LIST_LOCATED_STATUS, + INVOCATION_LIST_LOCATED_STATUS( + StoreStatisticNames.OP_LIST_LOCATED_STATUS, "Calls of listLocatedStatus()"), - INVOCATION_LIST_STATUS(CommonStatisticNames.OP_LIST_STATUS, + INVOCATION_LIST_STATUS( + StoreStatisticNames.OP_LIST_STATUS, "Calls of listStatus()"), - INVOCATION_MKDIRS(CommonStatisticNames.OP_MKDIRS, + INVOCATION_MKDIRS( + StoreStatisticNames.OP_MKDIRS, "Calls of mkdirs()"), - INVOCATION_OPEN(CommonStatisticNames.OP_OPEN, + INVOCATION_OPEN( + StoreStatisticNames.OP_OPEN, "Calls of open()"), - INVOCATION_RENAME(CommonStatisticNames.OP_RENAME, + INVOCATION_RENAME( + StoreStatisticNames.OP_RENAME, "Calls of rename()"), OBJECT_COPY_REQUESTS("object_copy_requests", "Object copy requests"), OBJECT_DELETE_REQUESTS("object_delete_requests", "Object delete requests"), - OBJECT_LIST_REQUESTS("object_list_requests", + OBJECT_LIST_REQUESTS(StoreStatisticNames.OBJECT_LIST_REQUEST, "Number of object listings made"), - OBJECT_CONTINUE_LIST_REQUESTS("object_continue_list_requests", + OBJECT_CONTINUE_LIST_REQUESTS( + StoreStatisticNames.OBJECT_CONTINUE_LIST_REQUEST, "Number of continued object listings made"), - OBJECT_METADATA_REQUESTS("object_metadata_requests", + OBJECT_METADATA_REQUESTS( + StoreStatisticNames.ACTION_HTTP_HEAD_REQUEST, "Number of requests for object metadata"), - OBJECT_MULTIPART_UPLOAD_INITIATED("object_multipart_initiated", + OBJECT_MULTIPART_UPLOAD_INITIATED( + StoreStatisticNames.OBJECT_MULTIPART_UPLOAD_INITIATED, "Object multipart upload initiated"), - OBJECT_MULTIPART_UPLOAD_ABORTED("object_multipart_aborted", + OBJECT_MULTIPART_UPLOAD_ABORTED( + StoreStatisticNames.OBJECT_MULTIPART_UPLOAD_ABORTED, "Object multipart upload aborted"), - OBJECT_PUT_REQUESTS("object_put_requests", + OBJECT_PUT_REQUESTS( + StoreStatisticNames.OBJECT_PUT_REQUEST, "Object put/multipart upload count"), - OBJECT_PUT_REQUESTS_COMPLETED("object_put_requests_completed", + OBJECT_PUT_REQUESTS_COMPLETED( + StoreStatisticNames.OBJECT_PUT_REQUEST_COMPLETED, "Object put/multipart upload completed count"), - OBJECT_PUT_REQUESTS_ACTIVE("object_put_requests_active", + OBJECT_PUT_REQUESTS_ACTIVE( + StoreStatisticNames.OBJECT_PUT_REQUEST_ACTIVE, "Current number of active put requests"), - OBJECT_PUT_BYTES("object_put_bytes", "number of bytes uploaded"), - OBJECT_PUT_BYTES_PENDING("object_put_bytes_pending", + OBJECT_PUT_BYTES( + StoreStatisticNames.OBJECT_PUT_BYTES, + "number of bytes uploaded"), + OBJECT_PUT_BYTES_PENDING( + StoreStatisticNames.OBJECT_PUT_BYTES_PENDING, "number of bytes queued for upload/being actively uploaded"), - OBJECT_SELECT_REQUESTS("object_select_requests", + OBJECT_SELECT_REQUESTS( + StoreStatisticNames.OBJECT_SELECT_REQUESTS, "Count of S3 Select requests issued"), - STREAM_ABORTED("stream_aborted", + STREAM_READ_ABORTED( + StreamStatisticNames.STREAM_READ_ABORTED, "Count of times the TCP stream was aborted"), - STREAM_BACKWARD_SEEK_OPERATIONS("stream_backward_seek_operations", + STREAM_READ_BYTES( + StreamStatisticNames.STREAM_READ_BYTES, + "Bytes read from an input stream in read() calls"), + STREAM_READ_CLOSED( + StreamStatisticNames.STREAM_READ_CLOSED, + "Count of times the TCP stream was closed"), + STREAM_READ_CLOSE_OPERATIONS( + StreamStatisticNames.STREAM_READ_CLOSE_OPERATIONS, + "Total count of times an attempt to close an input stream was made"), + STREAM_READ_EXCEPTIONS( + StreamStatisticNames.STREAM_READ_EXCEPTIONS, + "Number of exceptions raised during input stream reads"), + STREAM_READ_FULLY_OPERATIONS( + StreamStatisticNames.STREAM_READ_FULLY_OPERATIONS, + "Count of readFully() operations in an input stream"), + STREAM_READ_OPENED( + StreamStatisticNames.STREAM_READ_OPENED, + "Total count of times an input stream to object store data was opened"), + STREAM_READ_OPERATIONS( + StreamStatisticNames.STREAM_READ_OPERATIONS, + "Count of read() operations in an input stream"), + STREAM_READ_OPERATIONS_INCOMPLETE( + StreamStatisticNames.STREAM_READ_OPERATIONS_INCOMPLETE, + "Count of incomplete read() operations in an input stream"), + STREAM_READ_VERSION_MISMATCHES( + StreamStatisticNames.STREAM_READ_VERSION_MISMATCHES, + "Count of version mismatches encountered while reading an input stream"), + STREAM_READ_SEEK_BACKWARD_OPERATIONS( + StreamStatisticNames.STREAM_READ_SEEK_BACKWARD_OPERATIONS, "Number of executed seek operations which went backwards in a stream"), - STREAM_CLOSED("stream_closed", "Count of times the TCP stream was closed"), - STREAM_CLOSE_OPERATIONS("stream_close_operations", - "Total count of times an attempt to close a data stream was made"), - STREAM_FORWARD_SEEK_OPERATIONS("stream_forward_seek_operations", - "Number of executed seek operations which went forward in a stream"), - STREAM_OPENED("stream_opened", - "Total count of times an input stream to object store was opened"), - STREAM_READ_EXCEPTIONS("stream_read_exceptions", - "Number of exceptions invoked on input streams"), - STREAM_READ_FULLY_OPERATIONS("stream_read_fully_operations", - "Count of readFully() operations in streams"), - STREAM_READ_OPERATIONS("stream_read_operations", - "Count of read() operations in streams"), - STREAM_READ_OPERATIONS_INCOMPLETE("stream_read_operations_incomplete", - "Count of incomplete read() operations in streams"), - STREAM_READ_VERSION_MISMATCHES("stream_read_version_mismatches", - "Count of version mismatches encountered while reading streams"), - STREAM_SEEK_BYTES_BACKWARDS("stream_bytes_backwards_on_seek", - "Count of bytes moved backwards during seek operations"), - STREAM_SEEK_BYTES_READ("stream_bytes_read", - "Count of bytes read during seek() in stream operations"), - STREAM_SEEK_BYTES_SKIPPED("stream_bytes_skipped_on_seek", - "Count of bytes skipped during forward seek operation"), - STREAM_SEEK_OPERATIONS("stream_seek_operations", - "Number of seek operations during stream IO."), - STREAM_CLOSE_BYTES_READ("stream_bytes_read_in_close", - "Count of bytes read when closing streams during seek operations."), - STREAM_ABORT_BYTES_DISCARDED("stream_bytes_discarded_in_abort", - "Count of bytes discarded by aborting the stream"), - STREAM_WRITE_FAILURES("stream_write_failures", + STREAM_READ_SEEK_BYTES_BACKWARDS( + StreamStatisticNames.STREAM_READ_SEEK_BYTES_BACKWARDS, + "Count of bytes moved backwards during seek operations" + + " in an input stream"), + STREAM_READ_SEEK_BYTES_READ( + StreamStatisticNames.STREAM_READ_SEEK_BYTES_READ, + "Count of bytes read during seek() in an input stream"), + STREAM_READ_SEEK_BYTES_SKIPPED( + StreamStatisticNames.STREAM_READ_SEEK_BYTES_SKIPPED, + "Count of bytes skipped during forward seek operations" + + " an input stream"), + STREAM_READ_SEEK_FORWARD_OPERATIONS( + StreamStatisticNames.STREAM_READ_SEEK_FORWARD_OPERATIONS, + "Number of executed seek operations which went forward in" + + " an input stream"), + STREAM_READ_SEEK_OPERATIONS( + StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS, + "Number of seek operations in an input stream"), + STREAM_READ_SEEK_POLICY_CHANGED( + StreamStatisticNames.STREAM_READ_SEEK_POLICY_CHANGED, + "Count of times the seek policy was dynamically changed" + + " in an input stream"), + STREAM_READ_CLOSE_BYTES_READ( + StreamStatisticNames.STREAM_READ_CLOSE_BYTES_READ, + "Count of bytes read when closing an input stream"), + STREAM_READ_BYTES_DISCARDED_ABORT( + StreamStatisticNames.STREAM_READ_BYTES_DISCARDED_ABORT, + "Count of bytes discarded by aborting in an input stream"), + STREAM_READ_TOTAL_BYTES( + StreamStatisticNames.STREAM_READ_TOTAL_BYTES, + "Total count of read from an input stream calls"), + + STREAM_WRITE_FAILURES( + StreamStatisticNames.STREAM_WRITE_EXCEPTIONS, "Count of stream write failures reported"), - STREAM_WRITE_BLOCK_UPLOADS("stream_write_block_uploads", + STREAM_WRITE_EXCEPTIONS_COMPLETING_UPLOADS( + StreamStatisticNames.STREAM_WRITE_EXCEPTIONS_COMPLETING_UPLOADS, + "failures when finalizing a multipart upload"), + STREAM_WRITE_BLOCK_UPLOADS( + StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS, "Count of block/partition uploads completed"), - STREAM_WRITE_BLOCK_UPLOADS_ACTIVE("stream_write_block_uploads_active", + STREAM_WRITE_BLOCK_UPLOADS_ACTIVE( + StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS_ACTIVE, "Count of block/partition uploads completed"), - STREAM_WRITE_BLOCK_UPLOADS_COMMITTED("stream_write_block_uploads_committed", + STREAM_WRITE_BLOCK_UPLOADS_COMMITTED( + StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS_COMMITTED, "Count of number of block uploads committed"), - STREAM_WRITE_BLOCK_UPLOADS_ABORTED("stream_write_block_uploads_aborted", + STREAM_WRITE_BLOCK_UPLOADS_ABORTED( + StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS_ABORTED, "Count of number of block uploads aborted"), - STREAM_WRITE_BLOCK_UPLOADS_PENDING("stream_write_block_uploads_pending", + STREAM_WRITE_BLOCK_UPLOADS_PENDING( + StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS_PENDING, "Gauge of block/partitions uploads queued to be written"), STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING( - "stream_write_block_uploads_data_pending", + StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING, "Gauge of block/partitions data uploads queued to be written"), - STREAM_WRITE_TOTAL_TIME("stream_write_total_time", + STREAM_WRITE_TOTAL_TIME( + StreamStatisticNames.STREAM_WRITE_TOTAL_TIME, "Count of total time taken for uploads to complete"), STREAM_WRITE_TOTAL_DATA("stream_write_total_data", - "Count of total data uploaded in block output"), - STREAM_WRITE_QUEUE_DURATION("stream_write_queue_duration", + "Count of total data uploaded"), + STREAM_WRITE_BYTES( + StreamStatisticNames.STREAM_WRITE_BYTES, + "Count of bytes written to output stream" + + " (including all not yet uploaded"), + STREAM_WRITE_QUEUE_DURATION( + StreamStatisticNames.STREAM_WRITE_QUEUE_DURATION, "Total queue duration of all block uploads"), // S3guard committer stats @@ -256,7 +338,13 @@ public enum Statistic { "Multipart Upload Completed"), MULTIPART_UPLOAD_STARTED( "multipart_upload_started", - "Multipart Upload Started"); + "Multipart Upload Started"), + + STORE_IO_REQUEST(StoreStatisticNames.STORE_IO_REQUEST, + "requests made of the remote store"), + + STORE_IO_RETRY(StoreStatisticNames.STORE_IO_RETRY, + "retried requests made of the remote store"); private static final Map SYMBOL_MAP = new HashMap<>(Statistic.values().length); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java index 26d0942db61d4..a79f228798b03 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java @@ -50,6 +50,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.s3a.impl.statistics.S3AStatisticsContext; import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; import org.apache.hadoop.fs.s3a.s3guard.S3Guard; import org.apache.hadoop.fs.s3a.select.SelectBinding; @@ -108,17 +109,26 @@ public class WriteOperationHelper implements WriteOperations { /** Bucket of the owner FS. */ private final String bucket; + /** + * statistics context. + */ + private final S3AStatisticsContext statisticsContext; + /** * Constructor. * @param owner owner FS creating the helper * @param conf Configuration object + * @param statisticsContext statistics context * */ - protected WriteOperationHelper(S3AFileSystem owner, Configuration conf) { + protected WriteOperationHelper(S3AFileSystem owner, + Configuration conf, + S3AStatisticsContext statisticsContext) { this.owner = owner; this.invoker = new Invoker(new S3ARetryPolicy(conf), this::operationRetried); this.conf = conf; + this.statisticsContext = statisticsContext; bucket = owner.getBucket(); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/S3ADelegationTokens.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/S3ADelegationTokens.java index ef0b129e6e0c2..5e0c1cd55d279 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/S3ADelegationTokens.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/S3ADelegationTokens.java @@ -34,9 +34,9 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.s3a.AWSCredentialProviderList; -import org.apache.hadoop.fs.s3a.S3AInstrumentation; import org.apache.hadoop.fs.s3a.auth.RoleModel; import org.apache.hadoop.fs.s3a.impl.StoreContext; +import org.apache.hadoop.fs.s3a.impl.statistics.DelegationTokenStatistics; import org.apache.hadoop.io.Text; import org.apache.hadoop.security.Credentials; import org.apache.hadoop.security.UserGroupInformation; @@ -135,9 +135,9 @@ public class S3ADelegationTokens extends AbstractDTService { AWSPolicyProvider.AccessLevel.WRITE); /** - * Statistics for the owner FS. + * Statistics for the operations. */ - private S3AInstrumentation.DelegationTokenStatistics stats; + private DelegationTokenStatistics stats; /** * Name of the token binding as extracted from token kind; used for @@ -161,7 +161,7 @@ public void bindToFileSystem(final URI uri, throws IOException { super.bindToFileSystem(uri, context, delegationOperations); service = getTokenService(getCanonicalUri()); - stats = context.getInstrumentation().newDelegationTokenStatistics(); + stats = context.getStatisticsContext().newDelegationTokenStatistics(); } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java index 32d00a4353e98..d6757d6445b5b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java @@ -44,6 +44,10 @@ import org.apache.hadoop.fs.s3a.commit.files.PendingSet; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; import org.apache.hadoop.fs.s3a.commit.files.SuccessData; +import org.apache.hadoop.fs.s3a.impl.statistics.CommitterStatistics; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.JobStatus; import org.apache.hadoop.mapreduce.TaskAttemptContext; @@ -85,7 +89,8 @@ * created by a few tasks, and the HTTP connection pool in the S3A * committer was large enough for more all the parallel POST requests. */ -public abstract class AbstractS3ACommitter extends PathOutputCommitter { +public abstract class AbstractS3ACommitter extends PathOutputCommitter + implements IOStatisticsSource { private static final Logger LOG = LoggerFactory.getLogger(AbstractS3ACommitter.class); @@ -128,6 +133,8 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter { /** Should a job marker be created? */ private final boolean createJobMarker; + private final CommitterStatistics committerStatistics; + /** * Create a committer. * This constructor binds the destination directory and configuration, but @@ -154,7 +161,9 @@ protected AbstractS3ACommitter( createJobMarker = context.getConfiguration().getBoolean( CREATE_SUCCESSFUL_JOB_OUTPUT_DIR_MARKER, DEFAULT_CREATE_SUCCESSFUL_JOB_DIR_MARKER); - commitOperations = new CommitOperations(fs); + // the statistics are shared between this committer and its operations. + committerStatistics = fs.newCommitterStatistics(); + commitOperations = new CommitOperations(fs, committerStatistics); } /** @@ -374,7 +383,12 @@ protected void maybeCreateSuccessMarkerFromCommits(JobContext context, // The list of committed objects in pending is size limited in // ActiveCommit.uploadCommitted. filenames.addAll(pending.committedObjects); - maybeCreateSuccessMarker(context, filenames); + // load in all the pending statistics + IOStatisticsSnapshot snapshot = new IOStatisticsSnapshot( + pending.getIOStatistics()); + snapshot.aggregate(getIOStatistics()); + + maybeCreateSuccessMarker(context, filenames, snapshot); } /** @@ -385,10 +399,12 @@ protected void maybeCreateSuccessMarkerFromCommits(JobContext context, * PUT up a the contents of a {@link SuccessData} file. * @param context job context * @param filenames list of filenames. + * @param ioStatistics any IO Statistics to include * @throws IOException IO failure */ protected void maybeCreateSuccessMarker(JobContext context, - List filenames) + List filenames, + final IOStatisticsSnapshot ioStatistics) throws IOException { if (createJobMarker) { // create a success data structure and then save it @@ -400,6 +416,7 @@ protected void maybeCreateSuccessMarker(JobContext context, successData.setTimestamp(now.getTime()); successData.setDate(now.toString()); successData.setFilenames(filenames); + successData.getIOStatistics().aggregate(ioStatistics); commitOperations.createSuccessMarker(getOutputPath(), successData, true); } } @@ -536,6 +553,7 @@ private void loadAndCommit( activeCommit.uploadCommitted( commit.getDestinationKey(), commit.getLength()); }); + activeCommit.pendingsetCommitted(pendingSet.getIOStatistics()); } } @@ -1044,6 +1062,11 @@ protected void abortPendingUploads( } } + @Override + public IOStatistics getIOStatistics() { + return committerStatistics.getIOStatistics(); + } + /** * State of the active commit operation. * @@ -1094,6 +1117,12 @@ public static class ActiveCommit { */ private long committedBytes; + /** + * Aggregate statistics of all supplied by + * committed uploads. + */ + private IOStatisticsSnapshot ioStatistics = new IOStatisticsSnapshot(); + /** * Construct from a source FS and list of files. * @param sourceFS filesystem containing the list of pending files @@ -1145,7 +1174,8 @@ public FileSystem getSourceFS() { * @param key key of the committed object. * @param size size in bytes. */ - public synchronized void uploadCommitted(String key, long size) { + public synchronized void uploadCommitted(String key, + long size) { if (committedObjects.size() < SUCCESS_MARKER_FILE_LIMIT) { committedObjects.add( key.startsWith("/") ? key : ("/" + key)); @@ -1154,6 +1184,19 @@ public synchronized void uploadCommitted(String key, long size) { committedBytes += size; } + /** + * Callback when a pendingset has been committed, + * including any source statistics. + * @param sourceStatistics any source statistics + */ + public void pendingsetCommitted(final IOStatistics sourceStatistics) { + ioStatistics.aggregate(sourceStatistics); + } + + public IOStatisticsSnapshot getIOStatistics() { + return ioStatistics; + } + public synchronized List getCommittedObjects() { return committedObjects; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java index 155e86a3d4688..d8470177b7c75 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java @@ -45,18 +45,21 @@ import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.s3a.S3AFileSystem; -import org.apache.hadoop.fs.s3a.S3AInstrumentation; import org.apache.hadoop.fs.s3a.S3AUtils; import org.apache.hadoop.fs.s3a.WriteOperationHelper; import org.apache.hadoop.fs.s3a.commit.files.PendingSet; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; import org.apache.hadoop.fs.s3a.commit.files.SuccessData; import org.apache.hadoop.fs.s3a.impl.InternalConstants; +import org.apache.hadoop.fs.s3a.impl.statistics.CommitterStatistics; import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.util.DurationInfo; import org.apache.hadoop.util.Progressable; +import static java.util.Objects.requireNonNull; import static org.apache.hadoop.fs.s3a.S3AUtils.*; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; import static org.apache.hadoop.fs.s3a.Constants.*; @@ -71,7 +74,7 @@ * duplicate that work. * */ -public class CommitOperations { +public class CommitOperations implements IOStatisticsSource { private static final Logger LOG = LoggerFactory.getLogger( CommitOperations.class); @@ -81,7 +84,7 @@ public class CommitOperations { private final S3AFileSystem fs; /** Statistics. */ - private final S3AInstrumentation.CommitterStatistics statistics; + private final CommitterStatistics statistics; /** * Write operations for the destination fs. @@ -105,9 +108,18 @@ public class CommitOperations { * @param fs FS to bind to */ public CommitOperations(S3AFileSystem fs) { - Preconditions.checkArgument(fs != null, "null fs"); - this.fs = fs; - statistics = fs.newCommitterStatistics(); + this(requireNonNull(fs), fs.newCommitterStatistics()); + } + + /** + * Instantiate. + * @param fs FS to bind to + * @param committerStatistics committer statistics + */ + public CommitOperations(S3AFileSystem fs, + CommitterStatistics committerStatistics) { + this.fs = requireNonNull(fs); + statistics = requireNonNull(committerStatistics); writeOperations = fs.getWriteOperationHelper(); } @@ -128,10 +140,15 @@ public String toString() { } /** @return statistics. */ - protected S3AInstrumentation.CommitterStatistics getStatistics() { + protected CommitterStatistics getStatistics() { return statistics; } + @Override + public IOStatistics getIOStatistics() { + return statistics.getIOStatistics(); + } + /** * Commit the operation, throwing an exception on any failure. * @param commit commit to execute diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java index 7f9dadf06f320..acb51a5925acc 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java @@ -27,6 +27,7 @@ import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.Statistic; import org.apache.hadoop.fs.s3a.commit.magic.MagicCommitTracker; +import org.apache.hadoop.fs.s3a.impl.StoreContext; import static org.apache.hadoop.fs.s3a.commit.MagicCommitPaths.*; @@ -50,6 +51,8 @@ public class MagicCommitIntegration { private final S3AFileSystem owner; private final boolean magicCommitEnabled; + private final StoreContext storeContext; + /** * Instantiate. * @param owner owner class @@ -59,6 +62,7 @@ public MagicCommitIntegration(S3AFileSystem owner, boolean magicCommitEnabled) { this.owner = owner; this.magicCommitEnabled = magicCommitEnabled; + storeContext = owner.createStoreContext(); } /** @@ -94,10 +98,10 @@ public PutTracker createTracker(Path path, String key) { if (isMagicCommitPath(elements)) { final String destKey = keyOfFinalDestination(elements, key); String pendingsetPath = key + CommitConstants.PENDING_SUFFIX; - owner.getInstrumentation() - .incrementCounter(Statistic.COMMITTER_MAGIC_FILES_CREATED, 1); + storeContext.incrementStatistic( + Statistic.COMMITTER_MAGIC_FILES_CREATED); tracker = new MagicCommitTracker(path, - owner.getBucket(), + storeContext.getBucket(), key, destKey, pendingsetPath, diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/PutTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/PutTracker.java index bbffef3ad221c..10440f77e7277 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/PutTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/PutTracker.java @@ -24,6 +24,7 @@ import com.amazonaws.services.s3.model.PartETag; import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.fs.statistics.IOStatistics; /** * Multipart put tracker. @@ -68,6 +69,7 @@ public boolean outputImmediatelyVisible() { * @param uploadId Upload ID * @param parts list of parts * @param bytesWritten bytes written + * @param iostatistics nullable IO statistics * @return true if the commit is to be initiated immediately. * False implies the output stream does not need to worry about * what happens. @@ -75,7 +77,8 @@ public boolean outputImmediatelyVisible() { */ public boolean aboutToComplete(String uploadId, List parts, - long bytesWritten) + long bytesWritten, + final IOStatistics iostatistics) throws IOException { return true; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PendingSet.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PendingSet.java index c0d7415fcb9e4..1fb7c8e91576e 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PendingSet.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PendingSet.java @@ -27,6 +27,7 @@ import java.util.Map; import java.util.Set; +import com.fasterxml.jackson.annotation.JsonProperty; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -35,6 +36,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.commit.ValidationFailure; +import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; import org.apache.hadoop.util.JsonSerialization; import static org.apache.hadoop.fs.s3a.commit.CommitUtils.validateCollectionClass; @@ -44,11 +47,20 @@ * Persistent format for multiple pending commits. * Contains 0 or more {@link SinglePendingCommit} entries; validation logic * checks those values on load. + *

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

+ * As single commits are added via {@link #add(SinglePendingCommit)}, + * any statistics from those commits are merged into the aggregate + * statistics, and those of the single commit cleared. */ @SuppressWarnings("unused") @InterfaceAudience.Private @InterfaceStability.Unstable -public class PendingSet extends PersistentCommitData { +public class PendingSet extends PersistentCommitData + implements IOStatisticsSource { private static final Logger LOG = LoggerFactory.getLogger(PendingSet.class); /** @@ -56,7 +68,7 @@ public class PendingSet extends PersistentCommitData { * If this is changed the value of {@link #serialVersionUID} will change, * to avoid deserialization problems. */ - public static final int VERSION = 1; + public static final int VERSION = 2; /** * Serialization ID: {@value}. @@ -77,6 +89,12 @@ public class PendingSet extends PersistentCommitData { */ private final Map extraData = new HashMap<>(0); + /** + * IOStatistics. + */ + @JsonProperty("iosstatistics") + private IOStatisticsSnapshot ioStatistics = new IOStatisticsSnapshot(); + public PendingSet() { this(0); } @@ -116,6 +134,12 @@ public static PendingSet load(FileSystem fs, Path path) */ public void add(SinglePendingCommit commit) { commits.add(commit); + // add any statistics. + IOStatisticsSnapshot st = commit.getIOStatistics(); + if (st != null) { + ioStatistics.aggregate(st); + st.clear(); + } } /** @@ -189,4 +213,13 @@ public List getCommits() { public void setCommits(List commits) { this.commits = commits; } + + @Override + public IOStatisticsSnapshot getIOStatistics() { + return ioStatistics; + } + + public void setIOStatistics(final IOStatisticsSnapshot ioStatistics) { + this.ioStatistics = ioStatistics; + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java index 596dd95685c8b..64eded97d5a17 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java @@ -31,6 +31,7 @@ import java.util.Map; import com.amazonaws.services.s3.model.PartETag; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import org.apache.commons.lang3.StringUtils; @@ -39,6 +40,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.commit.ValidationFailure; +import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; import org.apache.hadoop.util.JsonSerialization; import static org.apache.hadoop.fs.s3a.commit.CommitUtils.validateCollectionClass; @@ -47,19 +50,22 @@ /** * This is the serialization format for uploads yet to be committerd. - * + *

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

+ * The statistics published through the {@link IOStatisticsSource} + * interface are the static ones marshalled with the commit data; + * they may be empty. */ @SuppressWarnings("unused") @InterfaceAudience.Private @InterfaceStability.Unstable public class SinglePendingCommit extends PersistentCommitData - implements Iterable { + implements Iterable, IOStatisticsSource { /** * Serialization ID: {@value}. @@ -113,6 +119,12 @@ public class SinglePendingCommit extends PersistentCommitData */ private Map extraData = new HashMap<>(0); + /** + * IOStatistics. + */ + @JsonProperty("iosstatistics") + private IOStatisticsSnapshot ioStatistics = new IOStatisticsSnapshot(); + /** Destination file size. */ private long length; @@ -429,4 +441,13 @@ public long getLength() { public void setLength(long length) { this.length = length; } + + @Override + public IOStatisticsSnapshot getIOStatistics() { + return ioStatistics; + } + + public void setIOStatistics(final IOStatisticsSnapshot ioStatistics) { + this.ioStatistics = ioStatistics; + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SuccessData.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SuccessData.java index e0273fa11a584..4a7ec3563ed24 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SuccessData.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SuccessData.java @@ -25,6 +25,7 @@ import java.util.List; import java.util.Map; +import com.fasterxml.jackson.annotation.JsonProperty; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -34,6 +35,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.commit.ValidationFailure; +import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; import org.apache.hadoop.util.JsonSerialization; /** @@ -62,20 +65,29 @@ @SuppressWarnings("unused") @InterfaceAudience.Private @InterfaceStability.Unstable -public class SuccessData extends PersistentCommitData { +public class SuccessData extends PersistentCommitData + implements IOStatisticsSource { + private static final Logger LOG = LoggerFactory.getLogger(SuccessData.class); + /** + * Supported version value: {@value}. + * If this is changed the value of {@link #serialVersionUID} will change, + * to avoid deserialization problems. + */ + public static final int VERSION = 1; + /** * Serialization ID: {@value}. */ - private static final long serialVersionUID = 507133045258460084L; + private static final long serialVersionUID = 507133045258460084L + VERSION;; /** * Name to include in persisted data, so as to differentiate from * any other manifests: {@value}. */ public static final String NAME - = "org.apache.hadoop.fs.s3a.commit.files.SuccessData/1"; + = "org.apache.hadoop.fs.s3a.commit.files.SuccessData/" + VERSION; /** * Name of file; includes version marker. @@ -118,6 +130,12 @@ public class SuccessData extends PersistentCommitData { */ private List filenames = new ArrayList<>(0); + /** + * IOStatistics. + */ + @JsonProperty("iosstatistics") + private IOStatisticsSnapshot ioStatistics = new IOStatisticsSnapshot(); + @Override public void validate() throws ValidationFailure { ValidationFailure.verify(name != null, @@ -325,4 +343,13 @@ public void setDiagnostics(Map diagnostics) { public void addDiagnostic(String key, String value) { diagnostics.put(key, value); } + + @Override + public IOStatisticsSnapshot getIOStatistics() { + return ioStatistics; + } + + public void setIOStatistics(final IOStatisticsSnapshot ioStatistics) { + this.ioStatistics = ioStatistics; + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java index a619fc7b7d94e..565db09189335 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java @@ -34,6 +34,8 @@ import org.apache.hadoop.fs.s3a.WriteOperationHelper; import org.apache.hadoop.fs.s3a.commit.PutTracker; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; /** * Put tracker for Magic commits. @@ -102,6 +104,7 @@ public boolean outputImmediatelyVisible() { * @param uploadId Upload ID * @param parts list of parts * @param bytesWritten bytes written + * @param iostatistics nullable IO statistics * @return false, indicating that the commit must fail. * @throws IOException any IO problem. * @throws IllegalArgumentException bad argument @@ -109,7 +112,8 @@ public boolean outputImmediatelyVisible() { @Override public boolean aboutToComplete(String uploadId, List parts, - long bytesWritten) + long bytesWritten, + final IOStatistics iostatistics) throws IOException { Preconditions.checkArgument(StringUtils.isNotEmpty(uploadId), "empty/null upload ID: "+ uploadId); @@ -117,6 +121,15 @@ public boolean aboutToComplete(String uploadId, "No uploaded parts list"); Preconditions.checkArgument(!parts.isEmpty(), "No uploaded parts to save"); + + // put a 0-byte file with the name of the original under-magic path + PutObjectRequest originalDestPut = writer.createPutObjectRequest( + originalDestKey, + new ByteArrayInputStream(EMPTY), + 0); + writer.uploadObject(originalDestPut); + + // build the commit summary SinglePendingCommit commitData = new SinglePendingCommit(); commitData.touch(System.currentTimeMillis()); commitData.setDestinationKey(getDestKey()); @@ -126,6 +139,8 @@ public boolean aboutToComplete(String uploadId, commitData.setText(""); commitData.setLength(bytesWritten); commitData.bindCommitData(parts); + commitData.setIOStatistics( + new IOStatisticsSnapshot(iostatistics)); byte[] bytes = commitData.toBytes(); LOG.info("Uncommitted data pending to file {};" + " commit metadata for {} parts in {}. sixe: {} byte(s)", @@ -138,12 +153,6 @@ public boolean aboutToComplete(String uploadId, bytes.length); writer.uploadObject(put); - // now put a 0-byte file with the name of the original under-magic path - PutObjectRequest originalDestPut = writer.createPutObjectRequest( - originalDestKey, - new ByteArrayInputStream(EMPTY), - 0); - writer.uploadObject(originalDestPut); return false; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java index 99121730644b1..f567af30c964f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java @@ -37,6 +37,7 @@ import org.apache.hadoop.fs.s3a.commit.CommitUtilsWithMR; import org.apache.hadoop.fs.s3a.commit.files.PendingSet; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; +import org.apache.hadoop.fs.statistics.IOStatisticsLogging; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.TaskAttemptID; @@ -46,6 +47,7 @@ import static org.apache.hadoop.fs.s3a.commit.CommitUtils.*; import static org.apache.hadoop.fs.s3a.commit.MagicCommitPaths.*; import static org.apache.hadoop.fs.s3a.commit.CommitUtilsWithMR.*; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.demandStringifyIOStatistics; /** * This is a dedicated committer which requires the "magic" directory feature @@ -177,6 +179,8 @@ public void commitTask(TaskAttemptContext context) throws IOException { destroyThreadPool(); } getCommitOperations().taskCompleted(true); + LOG.debug("aggregate statistics\n{}", + demandStringifyIOStatistics(getIOStatistics())); } /** @@ -220,6 +224,8 @@ private PendingSet innerCommitTask( taskAttemptID.getTaskID().toString() + CommitConstants.PENDINGSET_SUFFIX); LOG.info("Saving work of {} to {}", taskAttemptID, taskOutcomePath); + LOG.debug("task statistics\n{}", + IOStatisticsLogging.demandStringifyIOStatisticsSource(pendingSet)); try { pendingSet.save(getDestFS(), taskOutcomePath, false); } catch (IOException e) { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ActiveOperationContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ActiveOperationContext.java new file mode 100644 index 0000000000000..4dba392e50064 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ActiveOperationContext.java @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl; + +import javax.annotation.Nullable; + +import java.util.Objects; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.hadoop.fs.s3a.impl.statistics.S3AStatisticsContext; +import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; + +/** + * Context for any active operation. + */ +public class ActiveOperationContext { + + /** + * An operation ID; currently just for logging...proper tracing needs more. + */ + private final long operationId; + + /** + * Statistics context. + */ + private final S3AStatisticsContext statisticsContext; + + /** + * S3Guard bulk operation state, if (currently) set. + */ + @Nullable private BulkOperationState bulkOperationState; + + public ActiveOperationContext( + final long operationId, + final S3AStatisticsContext statisticsContext, + @Nullable final BulkOperationState bulkOperationState) { + this.operationId = operationId; + this.statisticsContext = Objects.requireNonNull(statisticsContext, + "null statistics context"); + this.bulkOperationState = bulkOperationState; + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder( + "ActiveOperation{"); + sb.append("operationId=").append(operationId); + sb.append(", bulkOperationState=").append(bulkOperationState); + sb.append('}'); + return sb.toString(); + } + + @Nullable + public BulkOperationState getBulkOperationState() { + return bulkOperationState; + } + + public long getOperationId() { + return operationId; + } + + public S3AStatisticsContext getS3AStatisticsContext() { + return statisticsContext; + } + + private static final AtomicLong NEXT_OPERATION_ID = new AtomicLong(0); + + /** + * Create an operation ID. The nature of it should be opaque. + * @return an ID for the constructor. + */ + protected static long newOperationId() { + return NEXT_OPERATION_ID.incrementAndGet(); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteRetryHandler.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteRetryHandler.java index b2c1cc6271ec7..6eccd6a417ca5 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteRetryHandler.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteRetryHandler.java @@ -26,9 +26,9 @@ import org.slf4j.LoggerFactory; import org.apache.hadoop.fs.s3a.AWSClientIOException; -import org.apache.hadoop.fs.s3a.S3AInstrumentation; import org.apache.hadoop.fs.s3a.S3AStorageStatistics; import org.apache.hadoop.fs.s3a.Statistic; +import org.apache.hadoop.fs.s3a.impl.statistics.S3AStatisticsContext; import static org.apache.hadoop.fs.s3a.S3AUtils.isThrottleException; import static org.apache.hadoop.fs.s3a.Statistic.IGNORED_ERRORS; @@ -53,7 +53,7 @@ public class BulkDeleteRetryHandler extends AbstractStoreOperation { */ public static final String XML_PARSE_BROKEN = "Failed to parse XML document"; - private final S3AInstrumentation instrumentation; + private final S3AStatisticsContext instrumentation; private final S3AStorageStatistics storageStatistics; @@ -63,7 +63,7 @@ public class BulkDeleteRetryHandler extends AbstractStoreOperation { */ public BulkDeleteRetryHandler(final StoreContext storeContext) { super(storeContext); - instrumentation = storeContext.getInstrumentation(); + instrumentation = storeContext.getStatisticsContext(); storageStatistics = storeContext.getStorageStatistics(); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CallableSupplier.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CallableSupplier.java index 1c61d30b08a60..56c0eae45bf39 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CallableSupplier.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CallableSupplier.java @@ -31,7 +31,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.fs.impl.WrappedIOException; +import org.apache.hadoop.util.functional.RuntimeIOException; import org.apache.hadoop.util.DurationInfo; import static org.apache.hadoop.fs.impl.FutureIOSupport.raiseInnerCause; @@ -63,9 +63,9 @@ public Object get() { } catch (RuntimeException e) { throw e; } catch (IOException e) { - throw new WrappedIOException(e); + throw new RuntimeIOException(e); } catch (Exception e) { - throw new WrappedIOException(new IOException(e)); + throw new RuntimeIOException(new IOException(e)); } } @@ -73,7 +73,7 @@ public Object get() { * Submit a callable into a completable future. * RTEs are rethrown. * Non RTEs are caught and wrapped; IOExceptions to - * {@link WrappedIOException} instances. + * {@link RuntimeIOException} instances. * @param executor executor. * @param call call to invoke * @param type diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java index d34328ccbfe7c..649532cfac14b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java @@ -18,8 +18,6 @@ package org.apache.hadoop.fs.s3a.impl; -import java.util.concurrent.atomic.AtomicLong; - import com.amazonaws.AmazonServiceException; import com.amazonaws.SdkBaseException; import com.amazonaws.services.s3.model.CopyObjectRequest; @@ -39,6 +37,7 @@ import org.apache.hadoop.fs.s3a.NoVersionAttributeException; import org.apache.hadoop.fs.s3a.RemoteFileChangedException; import org.apache.hadoop.fs.s3a.S3ObjectAttributes; +import org.apache.hadoop.fs.s3a.impl.statistics.ChangeTrackerStatistics; import static com.google.common.base.Preconditions.checkNotNull; @@ -72,7 +71,7 @@ public class ChangeTracker { * Mismatch counter; expected to be wired up to StreamStatistics except * during testing. */ - private final AtomicLong versionMismatches; + private final ChangeTrackerStatistics versionMismatches; /** * Revision identifier (e.g. eTag or versionId, depending on change @@ -90,7 +89,7 @@ public class ChangeTracker { */ public ChangeTracker(final String uri, final ChangeDetectionPolicy policy, - final AtomicLong versionMismatches, + final ChangeTrackerStatistics versionMismatches, final S3ObjectAttributes s3ObjectAttributes) { this.policy = checkNotNull(policy); this.uri = uri; @@ -111,8 +110,8 @@ public ChangeDetectionPolicy.Source getSource() { } @VisibleForTesting - public AtomicLong getVersionMismatches() { - return versionMismatches; + public long getVersionMismatches() { + return versionMismatches.getVersionMismatches(); } /** @@ -177,7 +176,7 @@ public void processResponse(final S3Object object, if (revisionId != null) { // the requirements of the change detection policy wasn't met: the // object was not returned. - versionMismatches.incrementAndGet(); + versionMismatches.versionMismatchError(); throw new RemoteFileChangedException(uri, operation, String.format(CHANGE_REPORTED_BY_S3 + " during %s" @@ -235,7 +234,7 @@ public void processException(SdkBaseException e, String operation) throws // This isn't really going to be hit due to // https://github.com/aws/aws-sdk-java/issues/1644 if (serviceException.getStatusCode() == SC_PRECONDITION_FAILED) { - versionMismatches.incrementAndGet(); + versionMismatches.versionMismatchError(); throw new RemoteFileChangedException(uri, operation, String.format( RemoteFileChangedException.PRECONDITIONS_FAILED + " on %s." @@ -292,10 +291,10 @@ private void processNewRevision(final String newRevisionId, uri, pos, operation, - versionMismatches.get()); + versionMismatches.getVersionMismatches()); if (pair.left) { // an mismatch has occurred: note it. - versionMismatches.incrementAndGet(); + versionMismatches.versionMismatchError(); } if (pair.right != null) { // there's an exception to raise: do it diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java index 4d944a1721303..87e31df24a4a0 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java @@ -110,4 +110,11 @@ private InternalConstants() { * Value: {@value}. */ public static final int DEFAULT_UPLOAD_PART_COUNT_LIMIT = 10000; + + /** + * Flag to enable AWS Statistics binding. As this is triggering + * problems related to region/endpoint setup, it is currently + * disabled. + */ + public static final boolean AWS_SDK_METRICS_ENABLED = false; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ListingOperationCallbacks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ListingOperationCallbacks.java index e5f9f7d9808ea..48f00928e7167 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ListingOperationCallbacks.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ListingOperationCallbacks.java @@ -29,6 +29,9 @@ import org.apache.hadoop.fs.s3a.S3ListRequest; import org.apache.hadoop.fs.s3a.S3ListResult; import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider; +import org.apache.hadoop.fs.statistics.DurationTrackerFactory; + +import static java.util.Objects.requireNonNull; /** * These are all the callbacks which @@ -44,12 +47,14 @@ public interface ListingOperationCallbacks { * * Retry policy: retry untranslated. * @param request request to initiate + * @param listingContext context with statistics to update * @return the results * @throws IOException if the retry invocation raises one (it shouldn't). */ @Retries.RetryRaw CompletableFuture listObjectsAsync( - S3ListRequest request) + S3ListRequest request, + ListingContext listingContext) throws IOException; /** @@ -57,13 +62,15 @@ CompletableFuture listObjectsAsync( * Retry policy: retry untranslated. * @param request last list objects request to continue * @param prevResult last paged result to continue from + * @param listingContext context with statistics to update * @return the next result object * @throws IOException none, just there for retryUntranslated. */ @Retries.RetryRaw CompletableFuture continueListObjectsAsync( S3ListRequest request, - S3ListResult prevResult) + S3ListResult prevResult, + ListingContext listingContext) throws IOException; /** @@ -117,4 +124,27 @@ S3ListRequest createListObjectsRequest( * @return true iff the path is authoritative on the client. */ boolean allowAuthoritative(Path p); + + /** + * This is a context to pass down to the asynchronous + * statistic calls; it initially contains a reference + * to the IOStatisticsStore to update with statistics + * on the list requests. + *

+ * If/when a telemetry context reference is to be + * passed across threads, this is where it can be + * passed around. + */ + final class ListingContext { + + private final DurationTrackerFactory durationTrackerFactory; + + public ListingContext(final DurationTrackerFactory durationTrackerFactory) { + this.durationTrackerFactory = requireNonNull(durationTrackerFactory); + } + + public DurationTrackerFactory getDurationTrackerFactory() { + return durationTrackerFactory; + } + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/NetworkBinding.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/NetworkBinding.java index ca1b09e9bdc93..ebc1fcbb48afd 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/NetworkBinding.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/NetworkBinding.java @@ -51,13 +51,10 @@ public class NetworkBinding { ".thirdparty.apache.http.conn.ssl.SSLConnectionSocketFactory"; /** - * Configures the {@link com.amazonaws.thirdparty.apache.http.conn.ssl - * .SSLConnectionSocketFactory} used by the AWS SDK. A custom - * SSLConnectionSocketFactory can be set using the method - * {@link com.amazonaws.ApacheHttpClientConfig#setSslSocketFactory( - * com.amazonaws.thirdparty.apache.http.conn.socket.ConnectionSocketFactory)}. - * If {@link com.amazonaws.thirdparty.apache.http.conn.ssl - * .SSLConnectionSocketFactory} cannot be found on the classpath, the value + * Configures the {@code SSLConnectionSocketFactory} used by the AWS SDK. + * A custom Socket Factory can be set using the method + * {@code setSslSocketFactory()}. + * If {@code SSLConnectionSocketFactory} cannot be found on the classpath, the value * of {@link org.apache.hadoop.fs.s3a.Constants#SSL_CHANNEL_MODE} is ignored. * * @param conf the {@link Configuration} used to get the client specified diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploader.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploader.java index 9f131dd3989d5..14a9f1dd2ba17 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploader.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploader.java @@ -30,6 +30,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -55,6 +56,9 @@ import org.apache.hadoop.fs.s3a.WriteOperations; import org.apache.hadoop.fs.s3a.impl.statistics.S3AMultipartUploaderStatistics; import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; +import org.apache.hadoop.fs.statistics.IOStatistics; + +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToString; /** * MultipartUploader for S3AFileSystem. This uses the S3 multipart @@ -102,7 +106,7 @@ class S3AMultipartUploader extends AbstractMultipartUploader { this.builder = builder; this.writeOperations = writeOperations; this.context = context; - this.statistics = statistics; + this.statistics = Objects.requireNonNull(statistics); } @Override @@ -113,6 +117,22 @@ public void close() throws IOException { super.close(); } + @Override + public IOStatistics getIOStatistics() { + return statistics.getIOStatistics(); + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder( + "S3AMultipartUploader{"); + sb.append("base=").append(getBasePath()); + sb.append("; statistics=").append( + ioStatisticsToString(statistics.getIOStatistics())); + sb.append('}'); + return sb.toString(); + } + /** * Retrieve the operation state; create one on demand if needed * and there has been no unsuccessful attempt to create one. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java index cafa22fdec80c..e3deb36228a2c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java @@ -33,9 +33,9 @@ import org.apache.hadoop.fs.s3a.Invoker; import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.S3AInputPolicy; -import org.apache.hadoop.fs.s3a.S3AInstrumentation; import org.apache.hadoop.fs.s3a.S3AStorageStatistics; import org.apache.hadoop.fs.s3a.Statistic; +import org.apache.hadoop.fs.s3a.impl.statistics.S3AStatisticsContext; import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider; import org.apache.hadoop.fs.s3a.s3guard.MetadataStore; import org.apache.hadoop.security.UserGroupInformation; @@ -88,7 +88,7 @@ public class StoreContext { private final Invoker invoker; /** Instrumentation and statistics. */ - private final S3AInstrumentation instrumentation; + private final S3AStatisticsContext statisticsContext; private final S3AStorageStatistics storageStatistics; /** Seek policy. */ @@ -129,7 +129,7 @@ public StoreContext( final ListeningExecutorService executor, final int executorCapacity, final Invoker invoker, - final S3AInstrumentation instrumentation, + final S3AStatisticsContext statisticsContext, final S3AStorageStatistics storageStatistics, final S3AInputPolicy inputPolicy, final ChangeDetectionPolicy changeDetectionPolicy, @@ -146,7 +146,7 @@ public StoreContext( this.executor = executor; this.executorCapacity = executorCapacity; this.invoker = invoker; - this.instrumentation = instrumentation; + this.statisticsContext = statisticsContext; this.storageStatistics = storageStatistics; this.inputPolicy = inputPolicy; this.changeDetectionPolicy = changeDetectionPolicy; @@ -186,8 +186,13 @@ public Invoker getInvoker() { return invoker; } - public S3AInstrumentation getInstrumentation() { - return instrumentation; + /** + * Get the statistics context for this StoreContext. + * @return the statistics context this store context was created + * with. + */ + public S3AStatisticsContext getStatisticsContext() { + return statisticsContext; } public S3AInputPolicy getInputPolicy() { @@ -267,7 +272,7 @@ public void incrementStatistic(Statistic statistic) { * @param count the count to increment */ public void incrementStatistic(Statistic statistic, long count) { - instrumentation.incrementCounter(statistic, count); + statisticsContext.incrementCounter(statistic, count); storageStatistics.incrementCounter(statistic, count); } @@ -277,7 +282,7 @@ public void incrementStatistic(Statistic statistic, long count) { * @param count the count to decrement */ public void decrementGauge(Statistic statistic, long count) { - instrumentation.decrementGauge(statistic, count); + statisticsContext.decrementGauge(statistic, count); } /** @@ -286,7 +291,7 @@ public void decrementGauge(Statistic statistic, long count) { * @param count the count to increment */ public void incrementGauge(Statistic statistic, long count) { - instrumentation.incrementGauge(statistic, count); + statisticsContext.incrementGauge(statistic, count); } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContextBuilder.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContextBuilder.java index a5e0dadf379f5..b9c63dff8d1a8 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContextBuilder.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContextBuilder.java @@ -25,8 +25,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.s3a.Invoker; import org.apache.hadoop.fs.s3a.S3AInputPolicy; -import org.apache.hadoop.fs.s3a.S3AInstrumentation; import org.apache.hadoop.fs.s3a.S3AStorageStatistics; +import org.apache.hadoop.fs.s3a.impl.statistics.S3AStatisticsContext; import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider; import org.apache.hadoop.fs.s3a.s3guard.MetadataStore; import org.apache.hadoop.security.UserGroupInformation; @@ -52,7 +52,7 @@ public class StoreContextBuilder { private Invoker invoker; - private S3AInstrumentation instrumentation; + private S3AStatisticsContext statisticsContext; private S3AStorageStatistics storageStatistics; @@ -112,9 +112,9 @@ public StoreContextBuilder setInvoker(final Invoker invoke) { return this; } - public StoreContextBuilder setInstrumentation( - final S3AInstrumentation instr) { - this.instrumentation = instr; + public StoreContextBuilder setStatisticsContext( + final S3AStatisticsContext statisticsContext) { + this.statisticsContext = statisticsContext; return this; } @@ -176,7 +176,7 @@ public StoreContext build() { executor, executorCapacity, invoker, - instrumentation, + statisticsContext, storageStatistics, inputPolicy, changeDetectionPolicy, diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/AbstractS3AStatisticsSource.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/AbstractS3AStatisticsSource.java new file mode 100644 index 0000000000000..29f5617da07de --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/AbstractS3AStatisticsSource.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl.statistics; + +import org.apache.hadoop.fs.statistics.DurationTracker; +import org.apache.hadoop.fs.statistics.DurationTrackerFactory; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; + +/** + * Base class for implementing IOStatistics sources in the S3 module. + *

+ * A lot of the methods are very terse, because S3AInstrumentation has + * verbose methods of similar names; the short ones always + * refer to the inner class and not any superclass method. + */ +public abstract class AbstractS3AStatisticsSource implements + IOStatisticsSource, DurationTrackerFactory { + + private IOStatisticsStore ioStatistics; + + protected AbstractS3AStatisticsSource() { + } + + @Override + public IOStatisticsStore getIOStatistics() { + return ioStatistics; + } + + /** + * Setter. + * this must be called in the subclass constructor with + * whatever + * @param statistics statistics to set + */ + protected void setIOStatistics(final IOStatisticsStore statistics) { + this.ioStatistics = statistics; + } + + public long incCounter(String name) { + return incCounter(name, 1); + } + + public long incCounter(String name, long value) { + return ioStatistics.incrementCounter(name, value); + } + + public Long lookupCounterValue(final String name) { + return ioStatistics.counters().get(name); + } + + public Long lookupGaugeValue(final String name) { + return 0L; + } + + public Long getGaugeValue(final String name) { + return ioStatistics.gauges().get(name); + } + + public long incGauge(String name, long v) { + return ioStatistics.incrementGauge(name, v); + } + + public long incGauge(String name) { + return incGauge(name, 1); + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder( + "AbstractS3AStatisticsSource{"); + sb.append(ioStatistics); + sb.append('}'); + return sb.toString(); + } + + + @Override + public DurationTracker trackDuration(final String key, final int count) { + return getIOStatistics().trackDuration(key, count); + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/AwsStatisticsCollector.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/AwsStatisticsCollector.java new file mode 100644 index 0000000000000..8826a62abcdcc --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/AwsStatisticsCollector.java @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl.statistics; + +import java.time.Duration; +import java.util.function.Consumer; +import java.util.function.LongConsumer; + +import com.amazonaws.Request; +import com.amazonaws.Response; +import com.amazonaws.metrics.RequestMetricCollector; +import com.amazonaws.util.TimingInfo; + +import static com.amazonaws.util.AWSRequestMetrics.Field.ClientExecuteTime; +import static com.amazonaws.util.AWSRequestMetrics.Field.HttpClientRetryCount; +import static com.amazonaws.util.AWSRequestMetrics.Field.HttpRequestTime; +import static com.amazonaws.util.AWSRequestMetrics.Field.RequestCount; +import static com.amazonaws.util.AWSRequestMetrics.Field.RequestMarshallTime; +import static com.amazonaws.util.AWSRequestMetrics.Field.RequestSigningTime; +import static com.amazonaws.util.AWSRequestMetrics.Field.ResponseProcessingTime; +import static com.amazonaws.util.AWSRequestMetrics.Field.ThrottleException; + +/** + * Collect statistics from the AWS SDK and forward to an instance of + * {@link StatisticsFromAwsSdk} and thence into the S3A statistics. + *

+ * See {@code com.facebook.presto.hive.s3.PrestoS3FileSystemMetricCollector} + * for the inspiration for this. + *

+ * See {@code com.amazonaws.util.AWSRequestMetrics} for metric names. + */ +public class AwsStatisticsCollector extends RequestMetricCollector { + + /** + * final destination of updates. + */ + private final StatisticsFromAwsSdk collector; + + /** + * Instantiate. + * @param collector final destination of updates + */ + public AwsStatisticsCollector(final StatisticsFromAwsSdk collector) { + this.collector = collector; + } + + /** + * This is the callback from the AWS SDK where metrics + * can be collected. + * @param request AWS request + * @param response AWS response + */ + @Override + public void collectMetrics( + final Request request, + final Response response) { + + TimingInfo timingInfo = request.getAWSRequestMetrics().getTimingInfo(); + + counter(timingInfo, HttpClientRetryCount.name(), + collector::updateAwsRetryCount); + counter(timingInfo, RequestCount.name(), + collector::updateAwsRequestCount); + counter(timingInfo, ThrottleException.name(), + collector::updateAwsThrottleExceptionsCount); + + timing(timingInfo, ClientExecuteTime.name(), + collector::noteAwsClientExecuteTime); + timing(timingInfo, HttpRequestTime.name(), + collector::noteAwsRequestTime); + timing(timingInfo, RequestMarshallTime.name(), + collector::noteRequestMarshallTime); + timing(timingInfo, RequestSigningTime.name(), + collector::noteRequestSigningTime); + timing(timingInfo, ResponseProcessingTime.name(), + collector::noteResponseProcessingTime); + } + + /** + * Process a timing. + * @param timingInfo timing info + * @param subMeasurementName sub measurement + * @param durationConsumer consumer + */ + private void timing( + TimingInfo timingInfo, + String subMeasurementName, + Consumer durationConsumer) { + TimingInfo t1 = timingInfo.getSubMeasurement(subMeasurementName); + if (t1 != null && t1.getTimeTakenMillisIfKnown() != null) { + durationConsumer.accept(Duration.ofMillis( + t1.getTimeTakenMillisIfKnown().longValue())); + } + } + + /** + * Process a counter. + * @param timingInfo timing info + * @param subMeasurementName sub measurement + * @param consumer consumer + */ + private void counter( + TimingInfo timingInfo, + String subMeasurementName, + LongConsumer consumer) { + Number n = timingInfo.getCounter(subMeasurementName); + if (n != null) { + consumer.accept(n.longValue()); + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/BlockOutputStreamStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/BlockOutputStreamStatistics.java new file mode 100644 index 0000000000000..a294b4476cc2c --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/BlockOutputStreamStatistics.java @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl.statistics; + +import java.io.Closeable; + +/** + * Block output stream statistics. + */ +public interface BlockOutputStreamStatistics extends Closeable, + S3AStatisticInterface { + + /** + * Block is queued for upload. + */ + void blockUploadQueued(int blockSize); + + /** Queued block has been scheduled for upload. */ + void blockUploadStarted(long duration, int blockSize); + + /** A block upload has completed. */ + void blockUploadCompleted(long duration, int blockSize); + + /** + * A block upload has failed. + *

+ * A final transfer completed event is still expected, so this + * does not decrement the active block counter. + */ + void blockUploadFailed(long duration, int blockSize); + + /** Intermediate report of bytes uploaded. */ + void bytesTransferred(long byteCount); + + /** + * Note exception in a multipart complete. + * @param count count of exceptions + */ + void exceptionInMultipartComplete(int count); + + /** + * Note an exception in a multipart abort. + */ + void exceptionInMultipartAbort(); + + /** + * Get the number of bytes pending upload. + * @return the number of bytes in the pending upload state. + */ + long getBytesPendingUpload(); + + /** + * Data has been uploaded to be committed in a subsequent operation; + * to be called at the end of the write. + * @param size size in bytes + */ + void commitUploaded(long size); + + int getBlocksAllocated(); + + int getBlocksReleased(); + + /** + * Get counters of blocks actively allocated; may be inaccurate + * if the numbers change during the (non-synchronized) calculation. + * @return the number of actively allocated blocks. + */ + int getBlocksActivelyAllocated(); + + /** + * Record bytes written. + * @param count number of bytes + */ + void writeBytes(long count); + + /** + * Get the current count of bytes written. + * @return the counter value. + */ + long getBytesWritten(); + + /** + * A block has been allocated. + */ + void blockAllocated(); + + /** + * A block has been released. + */ + void blockReleased(); + + + /** + * Get the value of a counter. + * @param name counter name + * @return the value or null if no matching counter was found. + */ + Long lookupCounterValue(String name); + + /** + * Get the value of a gauge. + * @param name gauge name + * @return the value or null if no matching gauge was found. + */ + Long lookupGaugeValue(String name); +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/BondedS3AStatisticsContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/BondedS3AStatisticsContext.java new file mode 100644 index 0000000000000..d74cf1b50ad1b --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/BondedS3AStatisticsContext.java @@ -0,0 +1,209 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl.statistics; + +import javax.annotation.Nullable; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.s3a.S3AInstrumentation; +import org.apache.hadoop.fs.s3a.Statistic; +import org.apache.hadoop.fs.s3a.s3guard.MetastoreInstrumentation; + +/** + * An S3A statistics context which is bonded to a + * S3AInstrumentation instance -inevitably that of an S3AFileSystem + * instance. + *

+ * An interface is used to bind to the relevant fields, rather + * than have them passed in the constructor because some + * production code, specifically, DelegateToFileSystem, + * patches the protected field after initialization. + *

+ * All operations are passed through directly to that class. + *

+ * + * If an instance of FileSystem.Statistics is passed in, it + * will be used whenever input stream statistics are created - + * However, Internally always increments the statistics in the + * current thread. + *

+ * As a result, cross-thread IO will under-report. + *

+ * + * This is addressed through the stream statistics classes + * only updating the stats in the close() call. Provided + * they are closed in the worker thread, all stats collected in + * helper threads will be included. + */ +public class BondedS3AStatisticsContext implements S3AStatisticsContext { + + /** Source of statistics services. */ + private final S3AFSStatisticsSource statisticsSource; + + /** + * Instantiate. + * @param statisticsSource integration binding + */ + public BondedS3AStatisticsContext( + final S3AFSStatisticsSource statisticsSource) { + this.statisticsSource = statisticsSource; + } + + + /** + * Get the instrumentation from the FS integration. + * @return instrumentation instance. + */ + private S3AInstrumentation getInstrumentation() { + return statisticsSource.getInstrumentation(); + } + + /** + * The filesystem statistics: know this is thread-local. + * @return FS statistics. + */ + private FileSystem.Statistics getInstanceStatistics() { + return statisticsSource.getInstanceStatistics(); + } + + /** + * Get a MetastoreInstrumentation getInstrumentation() instance for this + * context. + * @return the S3Guard getInstrumentation() point. + */ + @Override + public MetastoreInstrumentation getS3GuardInstrumentation() { + return getInstrumentation().getS3GuardInstrumentation(); + } + + /** + * Create a stream input statistics instance. + * @return the new instance + */ + @Override + public S3AInputStreamStatistics newInputStreamStatistics() { + return getInstrumentation().newInputStreamStatistics( + statisticsSource.getInstanceStatistics()); + } + + /** + * Create a new instance of the committer statistics. + * @return a new committer statistics instance + */ + @Override + public CommitterStatistics newCommitterStatistics() { + return getInstrumentation().newCommitterStatistics(); + } + + /** + * Create a stream output statistics instance. + * @return the new instance + */ + @Override + public BlockOutputStreamStatistics newOutputStreamStatistics() { + return getInstrumentation() + .newOutputStreamStatistics(getInstanceStatistics()); + } + + /** + * Increment a specific counter. + *

+ * No-op if not defined. + * @param op operation + * @param count increment value + */ + @Override + public void incrementCounter(Statistic op, long count) { + getInstrumentation().incrementCounter(op, count); + } + + /** + * Increment a specific gauge. + *

+ * No-op if not defined. + * @param op operation + * @param count increment value + * @throws ClassCastException if the metric is of the wrong type + */ + @Override + public void incrementGauge(Statistic op, long count) { + getInstrumentation().incrementGauge(op, count); + } + + /** + * Decrement a specific gauge. + *

+ * No-op if not defined. + * @param op operation + * @param count increment value + * @throws ClassCastException if the metric is of the wrong type + */ + @Override + public void decrementGauge(Statistic op, long count) { + getInstrumentation().decrementGauge(op, count); + } + + /** + * Add a value to a quantiles statistic. No-op if the quantile + * isn't found. + * @param op operation to look up. + * @param value value to add. + * @throws ClassCastException if the metric is not a Quantiles. + */ + @Override + public void addValueToQuantiles(Statistic op, long value) { + getInstrumentation().addValueToQuantiles(op, value); + } + + /** + * Create a delegation token statistics instance. + * @return an instance of delegation token statistics + */ + @Override + public DelegationTokenStatistics newDelegationTokenStatistics() { + return getInstrumentation().newDelegationTokenStatistics(); + } + + @Override + public StatisticsFromAwsSdk newStatisticsFromAwsSdk() { + return new StatisticsFromAwsSdkImpl(getInstrumentation()); + } + + /** + * This is the interface which an integration source must implement + * for the integration. + * Note that the FileSystem.statistics field may be null for a class; + */ + public interface S3AFSStatisticsSource { + + /** + * Get the S3A Instrumentation. + * @return a non-null instrumentation instance + */ + S3AInstrumentation getInstrumentation(); + + /** + * Get the statistics of the FS instance, shared across all threads. + * @return filesystem statistics + */ + @Nullable + FileSystem.Statistics getInstanceStatistics(); + + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/ChangeTrackerStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/ChangeTrackerStatistics.java new file mode 100644 index 0000000000000..c4aa1bab69c9e --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/ChangeTrackerStatistics.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl.statistics; + +/** + * Interface for change tracking statistics. + */ +public interface ChangeTrackerStatistics { + + /** + * A version mismatch was detected. + */ + void versionMismatchError(); + + /** + * How many version mismatches have occurred. + */ + long getVersionMismatches(); +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/CommitterStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/CommitterStatistics.java new file mode 100644 index 0000000000000..9739ef98c9a42 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/CommitterStatistics.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl.statistics; + +import org.apache.hadoop.fs.statistics.IOStatisticsSource; + +/** + * Statistics for committed work. + */ +public interface CommitterStatistics + extends S3AStatisticInterface { + + /** A commit has been created. */ + void commitCreated(); + + /** + * Data has been uploaded to be committed in a subsequent operation. + * @param size size in bytes + */ + void commitUploaded(long size); + + /** + * A commit has been completed. + * @param size size in bytes + */ + void commitCompleted(long size); + + /** A commit has been aborted. */ + void commitAborted(); + + /** + * A commit was reverted. + */ + void commitReverted(); + + /** + * A commit failed. + */ + void commitFailed(); + + /** + * Note that a task has completed. + * @param success success flag + */ + void taskCompleted(boolean success); + + /** + * Note that a job has completed. + * @param success success flag + */ + void jobCompleted(boolean success); +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/CountersAndGauges.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/CountersAndGauges.java new file mode 100644 index 0000000000000..5a0651f0878bf --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/CountersAndGauges.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl.statistics; + +import org.apache.hadoop.fs.s3a.Statistic; + +/** + * This is the foundational API for collecting S3A statistics. + */ +public interface CountersAndGauges { + + /** + * Increment a specific counter. + * No-op if not defined. + * @param op operation + * @param count increment value + */ + void incrementCounter(Statistic op, long count); + + /** + * Increment a specific gauge. + * No-op if not defined. + * @param op operation + * @param count increment value + * @throws ClassCastException if the metric is of the wrong type + */ + void incrementGauge(Statistic op, long count); + + /** + * Decrement a specific gauge. + * No-op if not defined. + * @param op operation + * @param count increment value + * @throws ClassCastException if the metric is of the wrong type + */ + void decrementGauge(Statistic op, long count); + + /** + * Add a value to a quantiles statistic. No-op if the quantile + * isn't found. + * @param op operation to look up. + * @param value value to add. + * @throws ClassCastException if the metric is not a Quantiles. + */ + void addValueToQuantiles(Statistic op, long value); +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/CountingChangeTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/CountingChangeTracker.java new file mode 100644 index 0000000000000..3581267b354ef --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/CountingChangeTracker.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl.statistics; + +import java.util.concurrent.atomic.AtomicLong; + +/** + * A change tracker which increments an atomic long. + */ +public class CountingChangeTracker implements + ChangeTrackerStatistics { + + /** + * The counter which is updated on every mismatch. + */ + private final AtomicLong counter; + + public CountingChangeTracker(final AtomicLong counter) { + this.counter = counter; + } + + public CountingChangeTracker() { + this(new AtomicLong()); + } + + @Override + public void versionMismatchError() { + counter.incrementAndGet(); + } + + @Override + public long getVersionMismatches() { + return counter.get(); + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/DelegationTokenStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/DelegationTokenStatistics.java new file mode 100644 index 0000000000000..3b460993a8121 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/DelegationTokenStatistics.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl.statistics; + +/** + * Instrumentation exported to for S3A Delegation Token support. + */ +public interface DelegationTokenStatistics extends S3AStatisticInterface { + + /** A token has been issued. */ + void tokenIssued(); +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/EmptyS3AStatisticsContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/EmptyS3AStatisticsContext.java new file mode 100644 index 0000000000000..6758006bcf844 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/EmptyS3AStatisticsContext.java @@ -0,0 +1,527 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl.statistics; + +import java.io.IOException; +import java.time.Duration; + +import org.apache.hadoop.fs.s3a.Statistic; +import org.apache.hadoop.fs.s3a.s3guard.MetastoreInstrumentation; +import org.apache.hadoop.fs.s3a.s3guard.MetastoreInstrumentationImpl; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.DurationTracker; + +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.emptyStatistics; +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.stubDurationTracker; + +/** + * Special statistics context, all of whose context operations are no-ops. + * All statistics instances it returns are also empty. + *

+ * This class is here primarily to aid in testing, but it also allows for + * classes to require a non-empty statistics context in their constructor -yet + * still be instantiated without one bound to any filesystem. + */ +public final class EmptyS3AStatisticsContext implements S3AStatisticsContext { + + public static final MetastoreInstrumentation + METASTORE_INSTRUMENTATION = new MetastoreInstrumentationImpl(); + + public static final S3AInputStreamStatistics + EMPTY_INPUT_STREAM_STATISTICS = new EmptyInputStreamStatistics(); + + public static final CommitterStatistics + EMPTY_COMMITTER_STATISTICS = new EmptyCommitterStatistics(); + + @SuppressWarnings("IOResourceOpenedButNotSafelyClosed") + public static final BlockOutputStreamStatistics + EMPTY_BLOCK_OUTPUT_STREAM_STATISTICS + = new EmptyBlockOutputStreamStatistics(); + + public static final DelegationTokenStatistics + EMPTY_DELEGATION_TOKEN_STATISTICS = new EmptyDelegationTokenStatistics(); + + public static final StatisticsFromAwsSdk + EMPTY_STATISTICS_FROM_AWS_SDK = new EmptyStatisticsFromAwsSdk(); + + @Override + public MetastoreInstrumentation getS3GuardInstrumentation() { + return METASTORE_INSTRUMENTATION; + } + + @Override + public S3AInputStreamStatistics newInputStreamStatistics() { + return EMPTY_INPUT_STREAM_STATISTICS; + } + + @Override + public CommitterStatistics newCommitterStatistics() { + return EMPTY_COMMITTER_STATISTICS; + } + + @Override + public BlockOutputStreamStatistics newOutputStreamStatistics() { + return EMPTY_BLOCK_OUTPUT_STREAM_STATISTICS; + } + + @Override + public DelegationTokenStatistics newDelegationTokenStatistics() { + return EMPTY_DELEGATION_TOKEN_STATISTICS; + } + + @Override + public StatisticsFromAwsSdk newStatisticsFromAwsSdk() { + return EMPTY_STATISTICS_FROM_AWS_SDK; + } + + @Override + public void incrementCounter(final Statistic op, final long count) { + + } + + @Override + public void incrementGauge(final Statistic op, final long count) { + + } + + @Override + public void decrementGauge(final Statistic op, final long count) { + + } + + @Override + public void addValueToQuantiles(final Statistic op, final long value) { + + } + + /** + * Base class for all the empty implementations. + */ + private static class EmptyS3AStatisticImpl implements + S3AStatisticInterface { + + /** + * Always return the stub duration tracker. + * @param key statistic key prefix + * @param count #of times to increment the matching counter in this + * operation. + * @return stub tracker. + */ + public DurationTracker trackDuration(String key, int count) { + return stubDurationTracker(); + } + } + + /** + * Input Stream statistics callbacks. + */ + private static final class EmptyInputStreamStatistics + extends EmptyS3AStatisticImpl + implements S3AInputStreamStatistics { + + @Override + public void seekBackwards(final long negativeOffset) { + + } + + @Override + public void seekForwards(final long skipped, + final long bytesRead) { + + } + + @Override + public long streamOpened() { + return 0; + } + + @Override + public void streamClose(final boolean abortedConnection, + final long remainingInCurrentRequest) { + + } + + @Override + public void readException() { + + } + + @Override + public void bytesRead(final long bytes) { + + } + + @Override + public void readOperationStarted(final long pos, final long len) { + + } + + @Override + public void readFullyOperationStarted(final long pos, final long len) { + + } + + @Override + public void readOperationCompleted(final int requested, final int actual) { + + } + + @Override + public void close() { + + } + + @Override + public void inputPolicySet(final int updatedPolicy) { + + } + + @Override + public void unbuffered() { + + } + + /** + * Return an IO statistics instance. + * @return an empty IO statistics instance. + */ + @Override + public IOStatistics getIOStatistics() { + return emptyStatistics(); + } + + @Override + public long getCloseOperations() { + return 0; + } + + @Override + public long getClosed() { + return 0; + } + + @Override + public long getAborted() { + return 0; + } + + @Override + public long getForwardSeekOperations() { + return 0; + } + + @Override + public long getBackwardSeekOperations() { + return 0; + } + + @Override + public long getBytesRead() { + return 0; + } + + @Override + public long getTotalBytesRead() { + return 0; + } + + @Override + public long getBytesSkippedOnSeek() { + return 0; + } + + @Override + public long getBytesBackwardsOnSeek() { + return 0; + } + + @Override + public long getBytesReadInClose() { + return 0; + } + + @Override + public long getBytesDiscardedInAbort() { + return 0; + } + + @Override + public long getOpenOperations() { + return 0; + } + + @Override + public long getSeekOperations() { + return 0; + } + + @Override + public long getReadExceptions() { + return 0; + } + + @Override + public long getReadOperations() { + return 0; + } + + @Override + public long getReadFullyOperations() { + return 0; + } + + @Override + public long getReadsIncomplete() { + return 0; + } + + @Override + public long getPolicySetCount() { + return 0; + } + + @Override + public long getVersionMismatches() { + return 0; + } + + @Override + public long getInputPolicy() { + return 0; + } + + @Override + public Long lookupCounterValue(final String name) { + return 0L; + } + + @Override + public Long lookupGaugeValue(final String name) { + return 0L; + } + + @Override + public ChangeTrackerStatistics getChangeTrackerStatistics() { + return new CountingChangeTracker(); + } + + @Override + public DurationTracker initiateGetRequest() { + return stubDurationTracker(); + } + + } + + /** + * Committer statistics. + */ + private static final class EmptyCommitterStatistics + extends EmptyS3AStatisticImpl + implements CommitterStatistics { + + @Override + public void commitCreated() { + } + + @Override + public void commitUploaded(final long size) { + } + + @Override + public void commitCompleted(final long size) { + } + + @Override + public void commitAborted() { + } + + @Override + public void commitReverted() { + } + + @Override + public void commitFailed() { + } + + @Override + public void taskCompleted(final boolean success) { + } + + @Override + public void jobCompleted(final boolean success) { + } + } + + private static final class EmptyBlockOutputStreamStatistics + extends EmptyS3AStatisticImpl + implements BlockOutputStreamStatistics { + + @Override + public void blockUploadQueued(final int blockSize) { + } + + @Override + public void blockUploadStarted(final long duration, final int blockSize) { + } + + @Override + public void blockUploadCompleted(final long duration, final int blockSize) { + } + + @Override + public void blockUploadFailed(final long duration, final int blockSize) { + } + + @Override + public void bytesTransferred(final long byteCount) { + } + + @Override + public void exceptionInMultipartComplete(final int count) { + + } + + @Override + public void exceptionInMultipartAbort() { + } + + @Override + public long getBytesPendingUpload() { + return 0; + } + + @Override + public void commitUploaded(final long size) { + + } + + @Override + public int getBlocksAllocated() { + return 0; + } + + @Override + public int getBlocksReleased() { + return 0; + } + + @Override + public int getBlocksActivelyAllocated() { + return 0; + } + + @Override + public IOStatistics getIOStatistics() { + return emptyStatistics(); + } + + @Override + public void blockAllocated() { + } + + @Override + public void blockReleased() { + } + + @Override + public void writeBytes(final long count) { + } + + @Override + public long getBytesWritten() { + return 0; + } + + @Override + public Long lookupCounterValue(final String name) { + return 0L; + } + + @Override + public Long lookupGaugeValue(final String name) { + return 0L; + } + + @Override + public void close() throws IOException { + } + + } + + /** + * Delegation Token Statistics. + */ + private static final class EmptyDelegationTokenStatistics + extends EmptyS3AStatisticImpl + implements DelegationTokenStatistics { + + @Override + public void tokenIssued() { + + } + } + + /** + * AWS SDK Callbacks. + */ + private static final class EmptyStatisticsFromAwsSdk + implements StatisticsFromAwsSdk { + + @Override + public void updateAwsRequestCount(final long longValue) { + + } + + @Override + public void updateAwsRetryCount(final long longValue) { + + } + + @Override + public void updateAwsThrottleExceptionsCount(final long longValue) { + + } + + @Override + public void noteAwsRequestTime(final Duration ofMillis) { + + } + + @Override + public void noteAwsClientExecuteTime(final Duration ofMillis) { + + } + + @Override + public void noteRequestMarshallTime(final Duration duration) { + + } + + @Override + public void noteRequestSigningTime(final Duration duration) { + + } + + @Override + public void noteResponseProcessingTime(final Duration duration) { + + } + } + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AInputStreamStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AInputStreamStatistics.java new file mode 100644 index 0000000000000..47e88454598bd --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AInputStreamStatistics.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.s3a.impl.statistics; + +import org.apache.hadoop.fs.statistics.DurationTracker; + +/** + * Statistics updated by an input stream during its actual operation. + * It also contains getters for tests. + */ +public interface S3AInputStreamStatistics extends AutoCloseable, + S3AStatisticInterface { + + /** + * Seek backwards, incrementing the seek and backward seek counters. + * @param negativeOffset how far was the seek? + * This is expected to be negative. + */ + void seekBackwards(long negativeOffset); + + /** + * Record a forward seek, adding a seek operation, a forward + * seek operation, and any bytes skipped. + * @param skipped bytes moved forward in stream + * @param bytesRead number of bytes skipped by reading from the stream. + * If the seek was implemented by a close + reopen, set this to zero. + */ + void seekForwards(long skipped, long bytesRead); + + /** + * The inner stream was opened. + * @return the previous count + */ + long streamOpened(); + + /** + * The inner stream was closed. + * @param abortedConnection flag to indicate the stream was aborted, + * rather than closed cleanly + * @param remainingInCurrentRequest the number of bytes remaining in + * the current request. + */ + void streamClose(boolean abortedConnection, + long remainingInCurrentRequest); + + /** + * An ignored stream read exception was received. + */ + void readException(); + + /** + * Increment the bytes read counter by the number of bytes; + * no-op if the argument is negative. + * @param bytes number of bytes read + */ + void bytesRead(long bytes); + + /** + * A {@code read(byte[] buf, int off, int len)} operation has started. + * @param pos starting position of the read + * @param len length of bytes to read + */ + void readOperationStarted(long pos, long len); + + /** + * A {@code PositionedRead.read(position, buffer, offset, length)} + * operation has just started. + * @param pos starting position of the read + * @param len length of bytes to read + */ + void readFullyOperationStarted(long pos, long len); + + /** + * A read operation has completed. + * @param requested number of requested bytes + * @param actual the actual number of bytes + */ + void readOperationCompleted(int requested, int actual); + + @Override + void close(); + + /** + * The input policy has been switched. + * @param updatedPolicy enum value of new policy. + */ + void inputPolicySet(int updatedPolicy); + + /** + * Get a reference to the change tracker statistics for this + * stream. + * @return a reference to the change tracker statistics + */ + ChangeTrackerStatistics getChangeTrackerStatistics(); + + /** + * A stream unbuffer() call has been made. + */ + void unbuffered(); + + long getCloseOperations(); + + long getClosed(); + + long getAborted(); + + long getForwardSeekOperations(); + + long getBackwardSeekOperations(); + + /** + * The bytes read in read() operations. + * @return the number of bytes returned to the caller. + */ + long getBytesRead(); + + /** + * The total number of bytes read, including + * all read and discarded when closing streams + * or skipped during seek calls. + * @return the total number of bytes read from + * S3. + */ + long getTotalBytesRead(); + + long getBytesSkippedOnSeek(); + + long getBytesBackwardsOnSeek(); + + long getBytesReadInClose(); + + long getBytesDiscardedInAbort(); + + long getOpenOperations(); + + long getSeekOperations(); + + long getReadExceptions(); + + long getReadOperations(); + + long getReadFullyOperations(); + + long getReadsIncomplete(); + + long getPolicySetCount(); + + long getVersionMismatches(); + + long getInputPolicy(); + + /** + * Get the value of a counter. + * @param name counter name + * @return the value or null if no matching counter was found. + */ + Long lookupCounterValue(String name); + + /** + * Get the value of a gauge. + * @param name gauge name + * @return the value or null if no matching gauge was found. + */ + Long lookupGaugeValue(String name); + + /** + * Initiate a GET request. + * @return duration tracker; + */ + DurationTracker initiateGetRequest(); + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatistics.java index 2cd74ff9cf5da..c009504516eef 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatistics.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatistics.java @@ -22,8 +22,11 @@ /** * Statistics for the S3A multipart uploader. + * It is expected to act as the statistics source for + * the uploader. */ -public interface S3AMultipartUploaderStatistics extends Closeable { +public interface S3AMultipartUploaderStatistics extends Closeable, + S3AStatisticInterface { void instantiated(); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatisticsImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatisticsImpl.java index 70e4785f62e19..3e92f07611e62 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatisticsImpl.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatisticsImpl.java @@ -20,9 +20,11 @@ import java.io.IOException; +import java.util.Objects; import java.util.function.BiConsumer; import org.apache.hadoop.fs.s3a.Statistic; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_ABORT_UNDER_PATH_INVOKED; import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_INSTANTIATED; @@ -31,15 +33,22 @@ import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_ABORTED; import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_COMPLETED; import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_STARTED; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore; /** * Implementation of the uploader statistics. + *

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

+ * Internally it builds a map of the relevant multipart statistics, + * increments as appropriate and serves this data back through + * the {@code IOStatisticsSource} API. */ -public final class S3AMultipartUploaderStatisticsImpl implements - S3AMultipartUploaderStatistics { +public final class S3AMultipartUploaderStatisticsImpl + extends AbstractS3AStatisticsSource + implements S3AMultipartUploaderStatistics { /** * The operation to increment a counter/statistic by a value. @@ -53,11 +62,23 @@ public final class S3AMultipartUploaderStatisticsImpl implements */ public S3AMultipartUploaderStatisticsImpl( final BiConsumer incrementCallback) { - this.incrementCallback = incrementCallback; + this.incrementCallback = Objects.requireNonNull(incrementCallback); + IOStatisticsStore st = iostatisticsStore() + .withCounters( + MULTIPART_INSTANTIATED.getSymbol(), + MULTIPART_PART_PUT.getSymbol(), + MULTIPART_PART_PUT_BYTES.getSymbol(), + MULTIPART_UPLOAD_ABORTED.getSymbol(), + MULTIPART_UPLOAD_ABORT_UNDER_PATH_INVOKED.getSymbol(), + MULTIPART_UPLOAD_COMPLETED.getSymbol(), + MULTIPART_UPLOAD_STARTED.getSymbol()) + .build(); + setIOStatistics(st); } private void inc(Statistic op, long count) { incrementCallback.accept(op, count); + incCounter(op.getSymbol(), count); } @Override diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AStatisticInterface.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AStatisticInterface.java new file mode 100644 index 0000000000000..41eef8b272f53 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AStatisticInterface.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl.statistics; + +import org.apache.hadoop.fs.statistics.DurationTrackerFactory; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; + +/** + * Base Interface for S3A Statistics, as a way of managing + * them. + */ +public interface S3AStatisticInterface extends IOStatisticsSource, + DurationTrackerFactory { + + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AStatisticsContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AStatisticsContext.java new file mode 100644 index 0000000000000..96c081a5c585d --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AStatisticsContext.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl.statistics; + +import org.apache.hadoop.fs.s3a.s3guard.MetastoreInstrumentation; + +/** + * This is the statistics context for ongoing operations in S3A. + */ +public interface S3AStatisticsContext extends CountersAndGauges { + + /** + * Get the metastore instrumentation. + * @return an instance of the metastore statistics tracking. + */ + MetastoreInstrumentation getS3GuardInstrumentation(); + + /** + * Create a stream input statistics instance. + * @return the new instance + */ + S3AInputStreamStatistics newInputStreamStatistics(); + + /** + * Create a new instance of the committer statistics. + * @return a new committer statistics instance + */ + CommitterStatistics newCommitterStatistics(); + + /** + * Create a stream output statistics instance. + * @return the new instance + */ + BlockOutputStreamStatistics newOutputStreamStatistics(); + + /** + * Create a delegation token statistics instance. + * @return an instance of delegation token statistics + */ + DelegationTokenStatistics newDelegationTokenStatistics(); + + /** + * Create a StatisticsFromAwsSdk instance. + * @return an instance of StatisticsFromAwsSdk + */ + StatisticsFromAwsSdk newStatisticsFromAwsSdk(); + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/StatisticsFromAwsSdk.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/StatisticsFromAwsSdk.java new file mode 100644 index 0000000000000..d04de7a0d5603 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/StatisticsFromAwsSdk.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl.statistics; + +import java.time.Duration; + +/** + * interface to receive statistics events from the AWS SDK + * by way of {@link AwsStatisticsCollector}. + */ +public interface StatisticsFromAwsSdk { + + /** + * Record a number of AWS requests. + * @param count number of events. + */ + void updateAwsRequestCount(long count); + + /** + * Record a number of AWS request retries. + * @param count number of events. + */ + void updateAwsRetryCount(long count); + + /** + * Record a number of throttle exceptions received. + * @param count number of events. + */ + void updateAwsThrottleExceptionsCount(long count); + + /** + * Record how long a request took overall. + * @param duration duration of operation. + */ + void noteAwsRequestTime(Duration duration); + + /** + * Record how long a request took to execute on the + * client. + * @param duration duration of operation. + */ + void noteAwsClientExecuteTime(Duration duration); + + /** + * Record how long a request took to marshall into + * XML. + * @param duration duration of operation. + */ + void noteRequestMarshallTime(Duration duration); + + /** + * Record how long a request took to sign, including + * any calls to EC2 credential endpoints. + * @param duration duration of operation. + */ + void noteRequestSigningTime(Duration duration); + + /** + * Record how long it took to process the response. + * @param duration duration of operation. + */ + void noteResponseProcessingTime(Duration duration); +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/StatisticsFromAwsSdkImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/StatisticsFromAwsSdkImpl.java new file mode 100644 index 0000000000000..07024cd8976a3 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/StatisticsFromAwsSdkImpl.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl.statistics; + +import java.time.Duration; + +import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_REQUEST; +import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_RETRY; +import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_THROTTLED; +import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_THROTTLE_RATE; + +/** + * Hook up AWS SDK Statistics to the S3 counters. + *

+ * Durations are not currently being used; that could be + * changed in future once an effective strategy for reporting + * them is determined. + */ +public final class StatisticsFromAwsSdkImpl implements + StatisticsFromAwsSdk { + + private final CountersAndGauges countersAndGauges; + + public StatisticsFromAwsSdkImpl( + final CountersAndGauges countersAndGauges) { + this.countersAndGauges = countersAndGauges; + } + + @Override + public void updateAwsRequestCount(final long count) { + countersAndGauges.incrementCounter(STORE_IO_REQUEST, count); + } + + @Override + public void updateAwsRetryCount(final long count) { + countersAndGauges.incrementCounter(STORE_IO_RETRY, count); + } + + @Override + public void updateAwsThrottleExceptionsCount(final long count) { + countersAndGauges.incrementCounter(STORE_IO_THROTTLED, count); + countersAndGauges.addValueToQuantiles(STORE_IO_THROTTLE_RATE, count); + } + + @Override + public void noteAwsRequestTime(final Duration duration) { + + } + + @Override + public void noteAwsClientExecuteTime(final Duration duration) { + + } + + @Override + public void noteRequestMarshallTime(final Duration duration) { + + } + + @Override + public void noteRequestSigningTime(final Duration duration) { + + } + + @Override + public void noteResponseProcessingTime(final Duration duration) { + + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/package-info.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/package-info.java new file mode 100644 index 0000000000000..bb9da296e1107 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/package-info.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Statistics collection for the S3A connector. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.fs.s3a.impl.statistics; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java index b131320a1b7ab..dab6a0f09c89c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java @@ -79,8 +79,9 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.RemoteIterator; -import org.apache.hadoop.fs.impl.FunctionsRaisingIOE; -import org.apache.hadoop.fs.impl.WrappedIOException; +import org.apache.hadoop.util.functional.CallableRaisingIOE; +import org.apache.hadoop.util.functional.RemoteIterators; +import org.apache.hadoop.util.functional.RuntimeIOException; import org.apache.hadoop.fs.s3a.AWSCredentialProviderList; import org.apache.hadoop.fs.s3a.AWSServiceThrottledException; import org.apache.hadoop.fs.s3a.Constants; @@ -450,7 +451,8 @@ void bindToOwnerFilesystem(final S3AFileSystem fs) { owner = fs; conf = owner.getConf(); StoreContext context = owner.createStoreContext(); - instrumentation = context.getInstrumentation().getS3GuardInstrumentation(); + instrumentation = context.getStatisticsContext() + .getS3GuardInstrumentation(); username = context.getUsername(); executor = context.createThrottledExecutor(); ttlTimeProvider = Preconditions.checkNotNull( @@ -638,8 +640,9 @@ public void deleteSubtree(Path path, LOG.debug("Subtree path {} is deleted; this will be a no-op", path); return; } - deleteEntries(new InternalIterators.PathFromRemoteStatusIterator( - new DescendantsIterator(this, meta)), + deleteEntries(RemoteIterators.mappingRemoteIterator( + new DescendantsIterator(this, meta), + FileStatus::getPath), operationState); } @@ -648,8 +651,7 @@ public void deleteSubtree(Path path, public void deletePaths(Collection paths, final BulkOperationState operationState) throws IOException { - deleteEntries( - new InternalIterators.RemoteIteratorFromIterator<>(paths.iterator()), + deleteEntries(RemoteIterators.remoteIteratorFromIterable(paths), operationState); } @@ -826,7 +828,7 @@ public DirListingMetadata listChildren(final Path path) throws IOException { for (Item item : wrapWithRetries(items)) { metas.add(itemToPathMetadata(item, username)); } - } catch (WrappedIOException e) { + } catch (RuntimeIOException e) { // failure in the iterators; unwrap. throw e.getCause(); } @@ -1634,7 +1636,7 @@ private int innerPrune( Set clearedParentPathSet = new HashSet<>(); // declare the operation to delete a batch as a function so // as to keep the code consistent across multiple uses. - FunctionsRaisingIOE.CallableRaisingIOE deleteBatchOperation = + CallableRaisingIOE deleteBatchOperation = () -> { // lowest path entries get deleted first. deletionBatch.sort(PathOrderComparators.TOPMOST_PATH_LAST); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/InternalIterators.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/InternalIterators.java deleted file mode 100644 index dd6fb5ff11544..0000000000000 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/InternalIterators.java +++ /dev/null @@ -1,91 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.s3a.s3guard; - -import java.io.IOException; -import java.util.Iterator; - -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.RemoteIterator; -import org.apache.hadoop.fs.s3a.S3AFileStatus; - -/** - * Internal iterators. - */ -final class InternalIterators { - - private InternalIterators() { - } - - /** - * From a remote status iterator, build a path iterator. - */ - static final class PathFromRemoteStatusIterator implements - RemoteIterator { - - private final RemoteIterator source; - - /** - * Construct. - * @param source source iterator. - */ - PathFromRemoteStatusIterator(final RemoteIterator source) { - this.source = source; - } - - @Override - public boolean hasNext() throws IOException { - return source.hasNext(); - } - - @Override - public Path next() throws IOException { - return source.next().getPath(); - } - } - - /** - * From a classic java.util.Iterator, build a Hadoop remote iterator. - * @param type of iterated value. - */ - static final class RemoteIteratorFromIterator implements - RemoteIterator { - - private final Iterator source; - - /** - * Construct. - * @param source source iterator. - */ - RemoteIteratorFromIterator(final Iterator source) { - this.source = source; - } - - @Override - public boolean hasNext() { - return source.hasNext(); - } - - @Override - public T next() { - return source.next(); - } - } - -} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RetryingCollection.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RetryingCollection.java index 394f393c57ec5..bbc25db53c03f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RetryingCollection.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RetryingCollection.java @@ -21,7 +21,7 @@ import java.io.IOException; import java.util.Iterator; -import org.apache.hadoop.fs.impl.WrappedIOException; +import org.apache.hadoop.util.functional.RuntimeIOException; import org.apache.hadoop.fs.s3a.Invoker; import org.apache.hadoop.fs.s3a.Retries; @@ -88,7 +88,7 @@ private RetryingIterator(final Iterator iterator) { /** * {@inheritDoc}. - * @throws WrappedIOException for IO failure, including throttling. + * @throws RuntimeIOException for IO failure, including throttling. */ @Override @Retries.RetryTranslated @@ -100,13 +100,13 @@ public boolean hasNext() { true, iterator::hasNext); } catch (IOException e) { - throw new WrappedIOException(e); + throw new RuntimeIOException(e); } } /** * {@inheritDoc}. - * @throws WrappedIOException for IO failure, including throttling. + * @throws RuntimeIOException for IO failure, including throttling. */ @Override @Retries.RetryTranslated @@ -118,7 +118,7 @@ public T next() { true, iterator::next); } catch (IOException e) { - throw new WrappedIOException(e); + throw new RuntimeIOException(e); } } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectInputStream.java index f4bd8d11708ef..5ee2b92df245e 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectInputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectInputStream.java @@ -39,9 +39,9 @@ import org.apache.hadoop.fs.FSInputStream; import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.s3a.Retries; -import org.apache.hadoop.fs.s3a.S3AInstrumentation; import org.apache.hadoop.fs.s3a.S3AReadOpContext; import org.apache.hadoop.fs.s3a.S3ObjectAttributes; +import org.apache.hadoop.fs.s3a.impl.statistics.S3AInputStreamStatistics; import org.apache.hadoop.io.IOUtils; import static com.google.common.base.Preconditions.checkNotNull; @@ -103,7 +103,7 @@ public class SelectInputStream extends FSInputStream implements private final S3AReadOpContext readContext; - private final S3AInstrumentation.InputStreamStatistics streamStatistics; + private final S3AInputStreamStatistics streamStatistics; private long readahead; @@ -130,7 +130,7 @@ public SelectInputStream( this.uri = "s3a://" + this.bucket + "/" + this.key; this.readContext = readContext; this.readahead = readContext.getReadahead(); - this.streamStatistics = readContext.getInstrumentation() + this.streamStatistics = readContext.getS3AStatisticsContext() .newInputStreamStatistics(); SelectRecordsInputStream stream = once( "S3 Select", @@ -204,7 +204,7 @@ public synchronized long skip(final long n) throws IOException { long skipped = once("skip", uri, () -> wrappedStream.skip(n)); pos.addAndGet(skipped); // treat as a forward skip for stats - streamStatistics.seekForwards(skipped); + streamStatistics.seekForwards(skipped, skipped); return skipped; } @@ -331,7 +331,7 @@ public synchronized void seek(long newPos) throws IOException { bytesSkipped++; } // read has finished. - streamStatistics.seekForwards(bytesSkipped); + streamStatistics.seekForwards(bytesSkipped, bytesSkipped); } } @@ -428,7 +428,7 @@ private void incrementBytesRead(long bytesRead) { */ @InterfaceAudience.Private @InterfaceStability.Unstable - public S3AInstrumentation.InputStreamStatistics getS3AStreamStatistics() { + public S3AInputStreamStatistics getS3AStreamStatistics() { return streamStatistics; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java index f8172233b6a35..24f721ad2b0d3 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java @@ -64,6 +64,7 @@ import static org.apache.hadoop.fs.s3a.Constants.BULK_DELETE_PAGE_SIZE; import static org.apache.hadoop.fs.s3a.Constants.BULK_DELETE_PAGE_SIZE_DEFAULT; import static org.apache.hadoop.fs.s3a.Invoker.once; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsSourceToString; import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_INTERRUPTED; import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_NOT_ACCEPTABLE; import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_NOT_FOUND; @@ -646,6 +647,7 @@ private boolean scanDirectoryTree( final int limit) throws IOException { int count = 0; + boolean result = true; RemoteIterator listing = operations .listObjects(path, storeContext.pathToKey(path)); while (listing.hasNext()) { @@ -674,10 +676,16 @@ private boolean scanDirectoryTree( if (limit > 0 && count >= limit) { println(out, "Limit of scan reached - %,d object%s", limit, suffix(limit)); - return false; + result = false; + break; } } - return true; + LOG.debug("Listing summary {}", listing); + if (verbose) { + println(out, "%nListing statistics:%n %s%n", + ioStatisticsSourceToString(listing)); + } + return result; } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestLocatedFileStatusFetcher.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestLocatedFileStatusFetcher.java index bd6bf2f6cdbc3..50be4a58d6035 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestLocatedFileStatusFetcher.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestLocatedFileStatusFetcher.java @@ -18,23 +18,287 @@ package org.apache.hadoop.fs.s3a; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.Collection; + +import org.assertj.core.api.Assertions; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; +import org.apache.hadoop.fs.s3a.auth.ITestRestrictedReadAccess; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.mapred.LocatedFileStatusFetcher; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile; +import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; +import static org.apache.hadoop.fs.s3a.Constants.AUTHORITATIVE_PATH; +import static org.apache.hadoop.fs.s3a.Constants.METADATASTORE_AUTHORITATIVE; +import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBucketOverrides; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatCounterStatistic; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.extractStatistics; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_LIST_REQUEST; +import static org.apache.hadoop.mapreduce.lib.input.FileInputFormat.LIST_STATUS_NUM_THREADS; + /** * Test the LocatedFileStatusFetcher can do. * This is related to HADOOP-16458. * There's basic tests in ITestS3AFSMainOperations; this * is see if we can create better corner cases. + *

+ * Much of the class is based on tests in {@link ITestRestrictedReadAccess}, + * but whereas that tests failure paths, this looks at the performance + * of successful invocations. */ +@RunWith(Parameterized.class) public class ITestLocatedFileStatusFetcher extends AbstractS3ATestBase { private static final Logger LOG = LoggerFactory.getLogger(ITestLocatedFileStatusFetcher.class); + + /** + * Parameterization. + */ + @Parameterized.Parameters(name = "{0}") + public static Collection params() { + return Arrays.asList(new Object[][]{ + {"raw", false}, + {"nonauth", true} + }); + } + + /** Filter to select everything. */ + private static final PathFilter EVERYTHING = t -> true; + + /** Filter to select .txt files. */ + private static final PathFilter TEXT_FILE = + path -> path.toUri().toString().endsWith(".txt"); + + /** The same path filter used in FileInputFormat. */ + private static final PathFilter HIDDEN_FILE_FILTER = + (p) -> { + String n = p.getName(); + return !n.startsWith("_") && !n.startsWith("."); + }; + + /** + * Text found in LocatedFileStatusFetcher exception when the glob + * returned "null". + */ + private static final String DOES_NOT_EXIST = "does not exist"; + + /** + * Text found in LocatedFileStatusFetcher exception when + * the glob returned an empty list. + */ + private static final String MATCHES_0_FILES = "matches 0 files"; + + /** + * Text used in files. + */ + public static final byte[] HELLO = "hello".getBytes(StandardCharsets.UTF_8); + + /** + * How many list calls are expected in a run which collects them: {@value}. + */ + private static final int EXPECTED_LIST_COUNT = 4; + + private final String name; + + private final boolean s3guard; + + private Path basePath; + + private Path emptyDir; + + private Path emptyFile; + + private Path subDir; + + private Path subdirFile; + + private Path subDir2; + + private Path subdir2File1; + + private Path subdir2File2; + + private Configuration listConfig; + + public ITestLocatedFileStatusFetcher(final String name, + final boolean s3guard) { + this.name = name; + this.s3guard = s3guard; + } + + @Override + public Configuration createConfiguration() { + Configuration conf = super.createConfiguration(); + String bucketName = getTestBucketName(conf); + + removeBaseAndBucketOverrides(bucketName, conf, + METADATASTORE_AUTHORITATIVE, + AUTHORITATIVE_PATH); + removeBucketOverrides(bucketName, conf, + S3_METADATA_STORE_IMPL); + if (!s3guard) { + removeBaseAndBucketOverrides(bucketName, conf, + S3_METADATA_STORE_IMPL); + } + conf.setBoolean(METADATASTORE_AUTHORITATIVE, false); + disableFilesystemCaching(conf); + return conf; + } + @Override + public void setup() throws Exception { + super.setup(); + S3AFileSystem fs + = getFileSystem(); + // avoiding the parameterization to steer clear of accidentally creating + // patterns; a timestamp is used to ensure tombstones from previous runs + // do not interfere + basePath = path("ITestLocatedFileStatusFetcher-" + name + + "-" + System.currentTimeMillis() / 1000); + + // define the paths and create them. + describe("Creating test directories and files"); + + // an empty directory directory under the noReadDir + emptyDir = new Path(basePath, "emptyDir"); + fs.mkdirs(emptyDir); + + // an empty file directory under the noReadDir + emptyFile = new Path(basePath, "emptyFile.txt"); + touch(fs, emptyFile); + + // a subdirectory + subDir = new Path(basePath, "subDir"); + + // and a file in that subdirectory + subdirFile = new Path(subDir, "subdirFile.txt"); + createFile(fs, subdirFile, true, HELLO); + subDir2 = new Path(subDir, "subDir2"); + subdir2File1 = new Path(subDir2, "subdir2File1.txt"); + subdir2File2 = new Path(subDir2, "subdir2File2.txt"); + createFile(fs, subdir2File1, true, HELLO); + createFile(fs, subdir2File2, true, HELLO); + listConfig = new Configuration(getConfiguration()); + } + + + /** + * Assert that the fetcher stats logs the expected number of calls. + * @param fetcher fetcher + * @param expectedListCount expected number of list calls + */ + private void assertListCount(final LocatedFileStatusFetcher fetcher, + final int expectedListCount) { + IOStatistics iostats = extractStatistics(fetcher); + LOG.info("Statistics of fetcher: {}", iostats); + assertThatCounterStatistic(iostats, + OBJECT_LIST_REQUEST) + .describedAs("stats of %s", iostats) + .isEqualTo(expectedListCount); + } + + /** + * Run a located file status fetcher against the directory tree. + */ @Test - public void testGlobScan() throws Throwable { + public void testSingleThreadedLocatedFileStatus() throws Throwable { + + describe("LocatedFileStatusFetcher operations"); + // use the same filter as FileInputFormat; single thread. + listConfig.setInt(LIST_STATUS_NUM_THREADS, 1); + LocatedFileStatusFetcher fetcher = + new LocatedFileStatusFetcher( + listConfig, + new Path[]{basePath}, + true, + HIDDEN_FILE_FILTER, + true); + Iterable stats = fetcher.getFileStatuses(); + Assertions.assertThat(stats) + .describedAs("result of located scan") + .flatExtracting(FileStatus::getPath) + .containsExactlyInAnyOrder( + emptyFile, + subdirFile, + subdir2File1, + subdir2File2); + assertListCount(fetcher, EXPECTED_LIST_COUNT); } + + /** + * Run a located file status fetcher against the directory tree. + */ + @Test + public void testLocatedFileStatusFourThreads() throws Throwable { + + // four threads and the text filter. + int threads = 4; + describe("LocatedFileStatusFetcher with %d", threads); + listConfig.setInt(LIST_STATUS_NUM_THREADS, threads); + LocatedFileStatusFetcher fetcher = + new LocatedFileStatusFetcher( + listConfig, + new Path[]{basePath}, + true, + EVERYTHING, + true); + Iterable stats = fetcher.getFileStatuses(); + IOStatistics iostats = extractStatistics(fetcher); + LOG.info("Statistics of fetcher: {}", iostats); + Assertions.assertThat(stats) + .describedAs("result of located scan") + .isNotNull() + .flatExtracting(FileStatus::getPath) + .containsExactlyInAnyOrder( + emptyFile, + subdirFile, + subdir2File1, + subdir2File2); + assertListCount(fetcher, EXPECTED_LIST_COUNT); + } + + /** + * Run a located file status fetcher against a file. + */ + @Test + public void testLocatedFileStatusScanFile() throws Throwable { + // pass in a file as the base of the scan. + describe("LocatedFileStatusFetcher with file %s", subdirFile); + listConfig.setInt(LIST_STATUS_NUM_THREADS, 16); + LocatedFileStatusFetcher fetcher + = new LocatedFileStatusFetcher( + listConfig, + new Path[]{subdirFile}, + true, + TEXT_FILE, + true); + Iterable stats = fetcher.getFileStatuses(); + Assertions.assertThat(stats) + .describedAs("result of located scan") + .isNotNull() + .flatExtracting(FileStatus::getPath) + .containsExactly(subdirFile); + IOStatistics ioStatistics = fetcher.getIOStatistics(); + Assertions.assertThat(ioStatistics) + .describedAs("IO statistics of %s", fetcher) + .isNull(); + } + } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java index afd3ec2bd3473..3e2786dbe475f 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java @@ -22,6 +22,7 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.s3a.impl.statistics.BlockOutputStreamStatistics; import org.apache.hadoop.io.IOUtils; import org.junit.BeforeClass; @@ -94,16 +95,16 @@ public void testBlocksClosed() throws Throwable { Path dest = path("testBlocksClosed"); describe(" testBlocksClosed"); FSDataOutputStream stream = getFileSystem().create(dest, true); - S3AInstrumentation.OutputStreamStatistics statistics + BlockOutputStreamStatistics statistics = S3ATestUtils.getOutputStreamStatistics(stream); byte[] data = ContractTestUtils.dataset(16, 'a', 26); stream.write(data); LOG.info("closing output stream"); stream.close(); assertEquals("total allocated blocks in " + statistics, - 1, statistics.blocksAllocated()); + 1, statistics.getBlocksAllocated()); assertEquals("actively allocated blocks in " + statistics, - 0, statistics.blocksActivelyAllocated()); + 0, statistics.getBlocksActivelyAllocated()); LOG.info("end of test case"); } @@ -129,7 +130,7 @@ private void markAndResetDatablock(S3ADataBlocks.BlockFactory factory) throws Exception { S3AInstrumentation instrumentation = new S3AInstrumentation(new URI("s3a://example")); - S3AInstrumentation.OutputStreamStatistics outstats + BlockOutputStreamStatistics outstats = instrumentation.newOutputStreamStatistics(null); S3ADataBlocks.DataBlock block = factory.create(1, BLOCK_SIZE, outstats); block.write(dataset, 0, dataset.length); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetrics.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetrics.java index 972c665438e9a..3bfe69c2bca91 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetrics.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetrics.java @@ -21,11 +21,15 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.metrics2.lib.MutableCounterLong; + +import org.assertj.core.api.Assertions; import org.junit.Test; import java.io.IOException; import java.io.InputStream; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsSourceToString; + /** * Test s3a performance metrics register and output. */ @@ -51,17 +55,34 @@ public void testStreamStatistics() throws IOException { Path file = path("testStreamStatistics"); byte[] data = "abcdefghijklmnopqrstuvwxyz".getBytes(); ContractTestUtils.createFile(fs, file, false, data); - - try (InputStream inputStream = fs.open(file)) { + InputStream inputStream = fs.open(file); + try { while (inputStream.read(data) != -1) { LOG.debug("Read batch of data from input stream..."); } + LOG.info("Final stream statistics: {}", + ioStatisticsSourceToString(inputStream)); + } finally { + // this is not try-with-resources only to aid debugging + inputStream.close(); } + final String statName = Statistic.STREAM_READ_BYTES.getSymbol(); + + final S3AInstrumentation instrumentation = fs.getInstrumentation(); + + final long counterValue = instrumentation.getCounterValue(statName); + + final int expectedBytesRead = 26; + Assertions.assertThat(counterValue) + .describedAs("Counter %s from instrumentation %s", + statName, instrumentation) + .isEqualTo(expectedBytesRead); MutableCounterLong read = (MutableCounterLong) - fs.getInstrumentation().getRegistry() - .get(Statistic.STREAM_SEEK_BYTES_READ.getSymbol()); - assertEquals("Stream statistics were not merged", 26, read.value()); + instrumentation.getRegistry() + .get(statName); + assertEquals("Stream statistics were not merged", expectedBytesRead, + read.value()); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AUnbuffer.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AUnbuffer.java index 2ba3fd7a65cde..8a7cec98e5a08 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AUnbuffer.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AUnbuffer.java @@ -21,13 +21,18 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.s3a.impl.statistics.S3AInputStreamStatistics; import org.apache.hadoop.io.IOUtils; +import org.assertj.core.api.Assertions; import org.junit.Test; import java.io.IOException; -import static org.apache.hadoop.fs.s3a.Statistic.STREAM_SEEK_BYTES_READ; +import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_BYTES; +import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_CLOSE_BYTES_READ; +import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_TOTAL_BYTES; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.demandStringifyIOStatisticsSource; /** * Integration test for calling @@ -40,6 +45,8 @@ */ public class ITestS3AUnbuffer extends AbstractS3ATestBase { + public static final int FILE_LENGTH = 16; + private Path dest; @Override @@ -48,7 +55,7 @@ public void setup() throws Exception { dest = path("ITestS3AUnbuffer"); describe("ITestS3AUnbuffer"); - byte[] data = ContractTestUtils.dataset(16, 'a', 26); + byte[] data = ContractTestUtils.dataset(FILE_LENGTH, 'a', 26); ContractTestUtils.writeDataset(getFileSystem(), dest, data, data.length, 16, true); } @@ -71,7 +78,7 @@ public void testUnbuffer() throws IOException { /** * Test that calling {@link S3AInputStream#unbuffer()} merges a stream's - * {@link org.apache.hadoop.fs.s3a.S3AInstrumentation.InputStreamStatistics} + * {@code InputStreamStatistics} * into the {@link S3AFileSystem}'s {@link S3AInstrumentation} instance. */ @Test @@ -79,36 +86,73 @@ public void testUnbufferStreamStatistics() throws IOException { describe("testUnbufferStreamStatistics"); // Validate bytesRead is updated correctly + S3AFileSystem fs = getFileSystem(); S3ATestUtils.MetricDiff bytesRead = new S3ATestUtils.MetricDiff( - getFileSystem(), STREAM_SEEK_BYTES_READ); + fs, STREAM_READ_BYTES); + S3ATestUtils.MetricDiff totalBytesRead = new S3ATestUtils.MetricDiff( + fs, STREAM_READ_TOTAL_BYTES); + S3ATestUtils.MetricDiff bytesReadInClose = new S3ATestUtils.MetricDiff( + fs, STREAM_READ_CLOSE_BYTES_READ); // Open file, read half the data, and then call unbuffer FSDataInputStream inputStream = null; + int firstBytesToRead = 8; + + int secondBytesToRead = 1; + long expectedFinalBytesRead; + long expectedTotalBytesRead; + + Object streamStatsStr; try { - inputStream = getFileSystem().open(dest); + inputStream = fs.open(dest); + streamStatsStr = demandStringifyIOStatisticsSource(inputStream); - readAndAssertBytesRead(inputStream, 8); + LOG.info("initial stream statistics {}", streamStatsStr); + readAndAssertBytesRead(inputStream, firstBytesToRead); + LOG.info("stream statistics after read {}", streamStatsStr); inputStream.unbuffer(); // Validate that calling unbuffer updates the input stream statistics - bytesRead.assertDiffEquals(8); + bytesRead.assertDiffEquals(firstBytesToRead); + final long bytesInUnbuffer = bytesReadInClose.diff(); + totalBytesRead.assertDiffEquals(firstBytesToRead + bytesInUnbuffer); // Validate that calling unbuffer twice in a row updates the statistics // correctly - readAndAssertBytesRead(inputStream, 4); + bytesReadInClose.reset(); + bytesRead.reset(); + readAndAssertBytesRead(inputStream, secondBytesToRead); inputStream.unbuffer(); - bytesRead.assertDiffEquals(12); + LOG.info("stream statistics after second read {}", streamStatsStr); + bytesRead.assertDiffEquals(secondBytesToRead); + final long bytesInClose = bytesReadInClose.diff(); + expectedFinalBytesRead = firstBytesToRead + secondBytesToRead; + expectedTotalBytesRead = expectedFinalBytesRead + + bytesInUnbuffer + bytesInClose; + + totalBytesRead.assertDiffEquals(expectedTotalBytesRead); } finally { + LOG.info("Closing stream"); IOUtils.closeStream(inputStream); } + LOG.info("stream statistics after close {}", streamStatsStr); // Validate that closing the file does not further change the statistics - bytesRead.assertDiffEquals(12); + totalBytesRead.assertDiffEquals(expectedTotalBytesRead); // Validate that the input stream stats are correct when the file is closed - assertEquals("S3AInputStream statistics were not updated properly", 12, - ((S3AInputStream) inputStream.getWrappedStream()) - .getS3AStreamStatistics().bytesRead); + S3AInputStreamStatistics streamStatistics = ((S3AInputStream) inputStream + .getWrappedStream()) + .getS3AStreamStatistics(); + Assertions.assertThat(streamStatistics) + .describedAs("Stream statistics %s", streamStatistics) + .hasFieldOrPropertyWithValue("bytesRead", + expectedFinalBytesRead) + .hasFieldOrPropertyWithValue("totalBytesRead", expectedTotalBytesRead); + assertEquals("S3AInputStream statistics were not updated properly in " + + streamStatsStr, + expectedFinalBytesRead, + streamStatistics.getBytesRead()); } private boolean isObjectStreamOpen(FSDataInputStream inputStream) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java index e30269298111d..96202af9efaae 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java @@ -39,11 +39,12 @@ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets; import org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase; +import org.apache.hadoop.fs.s3a.impl.statistics.CommitterStatistics; +import org.apache.hadoop.fs.s3a.impl.statistics.EmptyS3AStatisticsContext; import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; import org.apache.hadoop.util.Progressable; import static com.google.common.base.Preconditions.checkNotNull; -import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; /** * Relays FS calls to the mocked FS, allows for some extra logging with @@ -83,8 +84,6 @@ public class MockS3AFileSystem extends S3AFileSystem { * mock FS. */ private int logEvents = LOG_NAME; - private final S3AInstrumentation instrumentation = - new S3AInstrumentation(FS_URI); private Configuration conf; private WriteOperationHelper writeHelper; @@ -146,12 +145,12 @@ public Path qualify(final Path path) { public void initialize(URI name, Configuration originalConf) throws IOException { conf = originalConf; - writeHelper = new WriteOperationHelper(this, conf); + writeHelper = new WriteOperationHelper(this, conf, + new EmptyS3AStatisticsContext()); } @Override public void close() { - cleanupWithLogger(LOG, instrumentation); } @Override @@ -359,12 +358,13 @@ public String toString() { } @Override - public S3AInstrumentation.CommitterStatistics newCommitterStatistics() { - return instrumentation.newCommitterStatistics(); + public CommitterStatistics newCommitterStatistics() { + return EmptyS3AStatisticsContext.EMPTY_COMMITTER_STATISTICS; } @Override public void operationRetried(Exception ex) { /** no-op */ } + } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java index 4644cf24764ae..86a1675fae2f5 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java @@ -28,6 +28,8 @@ import com.amazonaws.services.s3.model.MultipartUploadListing; import com.amazonaws.services.s3.model.Region; +import org.apache.hadoop.fs.s3a.impl.statistics.StatisticsFromAwsSdk; + /** * An {@link S3ClientFactory} that returns Mockito mocks of the {@link AmazonS3} * interface suitable for unit testing. @@ -38,7 +40,8 @@ public class MockS3ClientFactory implements S3ClientFactory { public AmazonS3 createS3Client(URI name, final String bucket, final AWSCredentialsProvider credentialSet, - final String userAgentSuffix) { + final String userAgentSuffix, + final StatisticsFromAwsSdk statisticsFromAwsSdks) { AmazonS3 s3 = mock(AmazonS3.class); when(s3.doesBucketExist(bucket)).thenReturn(true); when(s3.doesBucketExistV2(bucket)).thenReturn(true); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java index f225800b872f3..4f0e8c32ef691 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java @@ -34,8 +34,8 @@ import org.apache.hadoop.fs.s3a.auth.MarshalledCredentialBinding; import org.apache.hadoop.fs.s3a.auth.MarshalledCredentials; import org.apache.hadoop.fs.s3a.commit.CommitConstants; - import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum; +import org.apache.hadoop.fs.s3a.impl.statistics.BlockOutputStreamStatistics; import org.apache.hadoop.fs.s3a.s3guard.MetadataStore; import org.apache.hadoop.fs.s3a.s3guard.MetadataStoreCapabilities; import org.apache.hadoop.fs.s3native.S3xLoginHelper; @@ -1171,7 +1171,7 @@ public static void assume(String message, boolean condition) { * @param out output stream * @return the (active) stats of the write */ - public static S3AInstrumentation.OutputStreamStatistics + public static BlockOutputStreamStatistics getOutputStreamStatistics(FSDataOutputStream out) { S3ABlockOutputStream blockOutputStream = (S3ABlockOutputStream) out.getWrappedStream(); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java index 60904d7ae83f1..c96a137ef1834 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java @@ -21,6 +21,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.s3a.commit.PutTracker; +import org.apache.hadoop.fs.s3a.impl.statistics.EmptyS3AStatisticsContext; import org.apache.hadoop.util.Progressable; import org.junit.Before; import org.junit.Test; @@ -50,11 +51,10 @@ public void setUp() throws Exception { S3ADataBlocks.BlockFactory blockFactory = mock(S3ADataBlocks.BlockFactory.class); long blockSize = Constants.DEFAULT_MULTIPART_SIZE; - S3AInstrumentation.OutputStreamStatistics statistics = null; WriteOperationHelper oHelper = mock(WriteOperationHelper.class); PutTracker putTracker = mock(PutTracker.class); stream = spy(new S3ABlockOutputStream(fs, "", executorService, - progressable, blockSize, blockFactory, statistics, oHelper, + progressable, blockSize, blockFactory, null, oHelper, putTracker)); } @@ -70,7 +70,8 @@ public void testWriteOperationHelperPartLimits() throws Throwable { S3AFileSystem s3a = mock(S3AFileSystem.class); when(s3a.getBucket()).thenReturn("bucket"); WriteOperationHelper woh = new WriteOperationHelper(s3a, - new Configuration()); + new Configuration(), + new EmptyS3AStatisticsContext()); ByteArrayInputStream inputStream = new ByteArrayInputStream( "a".getBytes()); // first one works diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java index 3d7cdfc08dec4..41ff4c2fbf3fd 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java @@ -18,8 +18,6 @@ package org.apache.hadoop.fs.s3a; -import java.util.concurrent.atomic.AtomicLong; - import com.amazonaws.AmazonServiceException; import com.amazonaws.SdkBaseException; import com.amazonaws.services.s3.Headers; @@ -36,6 +34,7 @@ import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; import org.apache.hadoop.fs.s3a.impl.ChangeTracker; +import org.apache.hadoop.fs.s3a.impl.statistics.CountingChangeTracker; import org.apache.hadoop.test.HadoopTestBase; import static org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy.CHANGE_DETECTED; @@ -359,7 +358,7 @@ protected void assertTrackerMismatchCount( final ChangeTracker tracker, final int expectedCount) { assertEquals("counter in tracker " + tracker, - expectedCount, tracker.getVersionMismatches().get()); + expectedCount, tracker.getVersionMismatches()); } /** @@ -386,7 +385,7 @@ protected ChangeTracker newTracker(final ChangeDetectionPolicy.Mode mode, source, requireVersion); ChangeTracker tracker = new ChangeTracker(URI, policy, - new AtomicLong(0), objectAttributes); + new CountingChangeTracker(), objectAttributes); if (objectAttributes.getVersionId() == null && objectAttributes.getETag() == null) { assertFalse("Tracker should not have applied constraints " + tracker, diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java index 4f6a1ff417873..ce22064a6c91c 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java @@ -50,6 +50,7 @@ import org.apache.hadoop.fs.s3a.commit.CommitOperations; import org.apache.hadoop.fs.s3a.commit.files.PendingSet; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; +import org.apache.hadoop.fs.s3a.impl.statistics.CommitterStatistics; import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; @@ -62,6 +63,7 @@ import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.forbidden; import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.newAssumedRoleConfig; import static org.apache.hadoop.fs.s3a.s3guard.S3GuardToolTestHelper.exec; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsSourceToString; import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains; import static org.apache.hadoop.test.LambdaTestUtils.*; @@ -573,8 +575,11 @@ public void testRestrictedCommitActions() throws Throwable { .addResources(directory(writeableDir)) ); roleFS = (S3AFileSystem) writeableDir.getFileSystem(conf); - CommitOperations fullOperations = new CommitOperations(fs); - CommitOperations operations = new CommitOperations(roleFS); + CommitterStatistics committerStatistics = fs.newCommitterStatistics(); + CommitOperations fullOperations = new CommitOperations(fs, + committerStatistics); + CommitOperations operations = new CommitOperations(roleFS, + committerStatistics); File localSrc = File.createTempFile("source", ""); writeCSVData(localSrc); @@ -608,7 +613,7 @@ public void testRestrictedCommitActions() throws Throwable { name + CommitConstants.PENDING_SUFFIX), true); assertTrue(src.delete()); })); - progress.assertCount("Process counter is not expected", + progress.assertCount("progress counter is not expected", range); try { @@ -652,6 +657,8 @@ public void testRestrictedCommitActions() throws Throwable { } finally { LOG.info("Cleanup"); fullOperations.abortPendingUploadsUnderPath(readOnlyDir); + LOG.info("Committer statistics {}", + ioStatisticsSourceToString(committerStatistics)); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ProgressCounter.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ProgressCounter.java index 15a57152091b7..362e674e13a54 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ProgressCounter.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ProgressCounter.java @@ -18,6 +18,8 @@ package org.apache.hadoop.fs.s3a.auth; +import java.util.concurrent.atomic.AtomicLong; + import org.apache.hadoop.util.Progressable; import static org.junit.Assert.assertEquals; @@ -27,14 +29,14 @@ */ public class ProgressCounter implements Progressable { - private long count; + private final AtomicLong count = new AtomicLong(); public void progress() { - count++; + count.incrementAndGet(); } public long getCount() { - return count; + return count.get(); } public void assertCount(String message, int expected) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationInFileystem.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationInFileystem.java index f6c908ea2f7e6..1910ed91fe98d 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationInFileystem.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationInFileystem.java @@ -44,6 +44,8 @@ import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.Statistic; +import org.apache.hadoop.fs.s3a.impl.statistics.EmptyS3AStatisticsContext; +import org.apache.hadoop.fs.s3a.impl.statistics.StatisticsFromAwsSdk; import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.security.TokenCache; @@ -70,6 +72,7 @@ import static org.apache.hadoop.fs.s3a.auth.delegation.MiniKerberizedHadoopCluster.ALICE; import static org.apache.hadoop.fs.s3a.auth.delegation.MiniKerberizedHadoopCluster.assertSecurityEnabled; import static org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens.lookupS3ADelegationToken; +import static org.apache.hadoop.fs.s3a.impl.InternalConstants.AWS_SDK_METRICS_ENABLED; import static org.apache.hadoop.test.LambdaTestUtils.doAs; import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.hamcrest.Matchers.containsString; @@ -305,6 +308,9 @@ public void testDelegatedFileSystem() throws Throwable { describe("Delegation tokens can be passed to a new filesystem;" + " if role restricted, permissions are tightened."); S3AFileSystem fs = getFileSystem(); + // force a probe of the remote FS to make sure its endpoint is valid + // (this always hits S3, even when S3Guard is enabled) + fs.getObjectMetadata(new Path("/")); readLandsatMetadata(fs); URI uri = fs.getUri(); @@ -561,8 +567,13 @@ protected ObjectMetadata readLandsatMetadata(final S3AFileSystem delegatedFS) conf.set(ENDPOINT, ""); factory.setConf(conf); String host = landsat.getHost(); + StatisticsFromAwsSdk awsStats = null; + if (AWS_SDK_METRICS_ENABLED) { + awsStats = new EmptyS3AStatisticsContext() + .newStatisticsFromAwsSdk(); + } AmazonS3 s3 = factory.createS3Client(landsat, host, testing, - "ITestSessionDelegationInFileystem"); + "ITestSessionDelegationInFileystem", awsStats); return Invoker.once("HEAD", host, () -> s3.getObjectMetadata(host, landsat.getPath().substring(1))); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java index 1cf3fb4a3f65f..c81ecb93255c6 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java @@ -53,6 +53,7 @@ import static org.apache.hadoop.fs.s3a.MultipartTestUtils.listMultipartUploads; import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToString; /** * Base test suite for committer operations. @@ -456,6 +457,8 @@ public static SuccessData validateSuccessFile(final Path outputPath, commitDetails); LOG.info("Committer statistics: \n{}", successData.dumpMetrics(" ", " = ", "\n")); + LOG.info("Job IOStatistics: \n{}", + ioStatisticsToString(successData.getIOStatistics())); LOG.info("Diagnostics\n{}", successData.dumpDiagnostics(" ", " = ", "\n")); if (!committerName.isEmpty()) { @@ -477,7 +480,7 @@ public static SuccessData validateSuccessFile(final Path outputPath, * @throws IOException failure to find/load the file * @throws AssertionError file is 0-bytes long, */ - public static SuccessData loadSuccessFile(final S3AFileSystem fs, + public static SuccessData loadSuccessFile(final FileSystem fs, final Path outputPath, final String origin) throws IOException { ContractTestUtils.assertPathExists(fs, "Output directory " + outputPath @@ -496,7 +499,9 @@ public static SuccessData loadSuccessFile(final S3AFileSystem fs, + success + " from " + origin + "; an S3A committer was not used", status.getLen() > 0); - LOG.info("Loading committer success file {}", success); + String body = ContractTestUtils.readUTF8(fs, success, -1); + LOG.info("Loading committer success file {}. Actual contents=\n{}", success, + body); return SuccessData.load(fs, success); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java index cacd54d12e90c..f5ad2c4e3654b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java @@ -40,6 +40,8 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.fs.s3a.commit.files.SuccessData; +import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.MapFile; @@ -68,7 +70,10 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.*; import static org.apache.hadoop.fs.s3a.S3AUtils.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; +import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_TASKS_SUCCEEDED; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatCounterStatistic; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsSourceToString; import static org.apache.hadoop.test.LambdaTestUtils.*; /** @@ -1245,9 +1250,28 @@ public void testOutputFormatIntegration() throws Throwable { assertTrue("Committer does not have data to commit " + committer, committer.needsTaskCommit(tContext)); commitTask(committer, tContext); + // at this point the committer tasks stats should be current. + IOStatisticsSnapshot snapshot = new IOStatisticsSnapshot( + committer.getIOStatistics()); + String commitsCompleted = COMMITTER_TASKS_SUCCEEDED.getSymbol(); + assertThatCounterStatistic(snapshot, commitsCompleted) + .describedAs("task commit count") + .isEqualTo(1L); + + commitJob(committer, jContext); + LOG.info("committer iostatistics {}", + ioStatisticsSourceToString(committer)); + // validate output - verifySuccessMarker(outDir); + SuccessData successData = verifySuccessMarker(outDir); + + // the task commit count should get through the job commit + IOStatisticsSnapshot successStats = successData.getIOStatistics(); + LOG.info("loaded statistics {}", successStats); + assertThatCounterStatistic(successStats, commitsCompleted) + .describedAs("task commit count") + .isEqualTo(1L); } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/integration/ITestS3ACommitterMRJob.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/integration/ITestS3ACommitterMRJob.java index caf54d1c36099..959477e1168ba 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/integration/ITestS3ACommitterMRJob.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/integration/ITestS3ACommitterMRJob.java @@ -80,7 +80,6 @@ import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.FS_S3A_COMMITTER_STAGING_UUID; import static org.apache.hadoop.fs.s3a.commit.staging.Paths.getMultipartUploadCommitsDirectory; import static org.apache.hadoop.fs.s3a.commit.staging.StagingCommitterConstants.STAGING_UPLOADS; -import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** * Test an MR Job with all the different committers. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/MockedStagingCommitter.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/MockedStagingCommitter.java index 47383b78a4d3b..d3da8185c8d65 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/MockedStagingCommitter.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/MockedStagingCommitter.java @@ -28,6 +28,7 @@ import org.apache.hadoop.fs.s3a.MockS3AFileSystem; import org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase.ClientErrors; import org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase.ClientResults; +import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.TaskAttemptContext; @@ -76,7 +77,8 @@ public void commitJob(JobContext context) throws IOException { @Override protected void maybeCreateSuccessMarker(JobContext context, - List filenames) + List filenames, + final IOStatisticsSnapshot ioStatistics) throws IOException { //skipped } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNetworkBinding.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNetworkBinding.java new file mode 100644 index 0000000000000..10fe339174fc5 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNetworkBinding.java @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl; + +import com.amazonaws.ClientConfiguration; +import com.amazonaws.client.builder.AwsClientBuilder; +import org.junit.Ignore; +import org.junit.Test; + +import org.apache.hadoop.test.AbstractHadoopTestBase; + +import static org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.createEndpointConfiguration; +import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.fixBucketRegion; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Unit tests related to the {@link NetworkBinding} class. + */ +public class TestNetworkBinding extends AbstractHadoopTestBase { + + private static final String US_EAST_1 = "us-east-1"; + + private static final String US_WEST_2 = "us-west-2"; + + @Test + public void testUSEast() { + assertRegionFixup(US_EAST_1, US_EAST_1); + } + + @Test + public void testUSWest() { + assertRegionFixup(US_WEST_2, US_WEST_2); + } + + @Test + public void testRegionUStoUSEast() { + assertRegionFixup("US", US_EAST_1); + } + + @Test + public void testRegionNullToUSEast() { + assertRegionFixup(null, US_EAST_1); + } + + private static void assertRegionFixup(String region, String expected) { + assertThat(fixBucketRegion(region)) + .describedAs("Fixup of %s", region) + .isEqualTo(expected); + } + + @Test + public void testNull() throws Throwable { + expectEndpoint("", true, "unused"); + } + + @Test + @Ignore("disabled until endpoint logic works for S3 client builder API") + public void testUSEastEndpoint() throws Throwable { + expectEndpoint(US_EAST_1, false, US_EAST_1); + } + + @Test + @Ignore("disabled until endpoint logic works for S3 client builder API") + public void testUSWestEndpoint() throws Throwable { + expectEndpoint(US_WEST_2, false, US_WEST_2); + } + + public void expectEndpoint(final String src, + final boolean expectNull, + final String expectRegion) { + AwsClientBuilder.EndpointConfiguration epr = + createEndpointConfiguration(src, new ClientConfiguration()); + String eprStr = epr == null + ? "(empty)" + : ("(" + epr.getServiceEndpoint() + " " + epr.getSigningRegion()); + if (expectNull) { + assertThat(epr) + .describedAs("Endpoint configuration of %s =", + src, eprStr) + .isNull(); + } else { + assertThat(epr) + .describedAs("Endpoint configuration of %s =", + src, eprStr) + .hasFieldOrPropertyWithValue("serviceEndpoint", src) + .hasFieldOrPropertyWithValue("signingRegion", expectRegion); + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNeworkBinding.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNeworkBinding.java deleted file mode 100644 index eebc3bfdf2fe4..0000000000000 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNeworkBinding.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.s3a.impl; - -import org.assertj.core.api.Assertions; -import org.junit.Test; - -import org.apache.hadoop.test.HadoopTestBase; - -import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.fixBucketRegion; - -/** - * Unit tests related to the {@link NetworkBinding} class. - */ -public class TestNeworkBinding extends HadoopTestBase { - - private static final String US_EAST_1 = "us-east-1"; - private static final String US_WEST_2 = "us-west-2"; - - @Test - public void testUSEast() { - assertRegionFixup(US_EAST_1, US_EAST_1); - } - - @Test - public void testUSWest() { - assertRegionFixup(US_WEST_2, US_WEST_2); - } - - @Test - public void testRegionUStoUSEast() { - assertRegionFixup("US", US_EAST_1); - } - - @Test - public void testRegionNullToUSEast() { - assertRegionFixup(null, US_EAST_1); - } - - private static void assertRegionFixup(String region, String expected) { - Assertions.assertThat(fixBucketRegion(region)) - .describedAs("Fixup of %s", region) - .isEqualTo(expected); - } -} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java index 0729f2ac289db..47f2c62834640 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java @@ -54,13 +54,13 @@ import org.apache.hadoop.fs.s3a.Invoker; import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.S3AInputPolicy; -import org.apache.hadoop.fs.s3a.S3AInstrumentation; import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus; import org.apache.hadoop.fs.s3a.S3AReadOpContext; import org.apache.hadoop.fs.s3a.S3AStorageStatistics; import org.apache.hadoop.fs.s3a.S3ListRequest; import org.apache.hadoop.fs.s3a.S3ListResult; import org.apache.hadoop.fs.s3a.S3ObjectAttributes; +import org.apache.hadoop.fs.s3a.impl.statistics.EmptyS3AStatisticsContext; import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; import org.apache.hadoop.fs.s3a.s3guard.DirListingMetadata; import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider; @@ -227,7 +227,7 @@ private StoreContext createMockStoreContext(boolean multiDelete, .setExecutorCapacity(Constants.DEFAULT_EXECUTOR_CAPACITY) .setInvoker( new Invoker(RetryPolicies.TRY_ONCE_THEN_FAIL, Invoker.LOG_EVENT)) - .setInstrumentation(new S3AInstrumentation(name)) + .setStatisticsContext(new EmptyS3AStatisticsContext()) .setStorageStatistics(new S3AStorageStatistics()) .setInputPolicy(S3AInputPolicy.Normal) .setChangeDetectionPolicy( @@ -245,7 +245,8 @@ private static class MinimalListingOperationCallbacks implements ListingOperationCallbacks { @Override public CompletableFuture listObjectsAsync( - S3ListRequest request) + S3ListRequest request, + ListingContext listingContext) throws IOException { return null; } @@ -253,7 +254,8 @@ public CompletableFuture listObjectsAsync( @Override public CompletableFuture continueListObjectsAsync( S3ListRequest request, - S3ListResult prevResult) + S3ListResult prevResult, + ListingContext listingContext) throws IOException { return null; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/statistics/ITestS3AContractStreamIOStatistics.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/statistics/ITestS3AContractStreamIOStatistics.java new file mode 100644 index 0000000000000..697dd195e8b22 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/statistics/ITestS3AContractStreamIOStatistics.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl.statistics; + +import java.util.Arrays; +import java.util.List; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractStreamIOStatisticsTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.fs.contract.s3a.S3AContract; +import org.apache.hadoop.fs.statistics.StreamStatisticNames; + +import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeEnableS3Guard; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.*; + +/** + * Test the S3A Streams IOStatistics support. + */ +public class ITestS3AContractStreamIOStatistics extends + AbstractContractStreamIOStatisticsTest { + + @Override + protected Configuration createConfiguration() { + Configuration conf = super.createConfiguration(); + // patch in S3Guard options + maybeEnableS3Guard(conf); + return conf; + } + + @Override + protected AbstractFSContract createContract(Configuration conf) { + return new S3AContract(conf); + } + + /** + * Keys which the input stream must support. + * @return a list of keys + */ + public List inputStreamStatisticKeys() { + return Arrays.asList( + StreamStatisticNames.STREAM_READ_ABORTED, + StreamStatisticNames.STREAM_READ_BYTES_DISCARDED_ABORT, + StreamStatisticNames.STREAM_READ_CLOSED, + StreamStatisticNames.STREAM_READ_CLOSE_BYTES_READ, + StreamStatisticNames.STREAM_READ_CLOSE_OPERATIONS, + StreamStatisticNames.STREAM_READ_OPENED, + StreamStatisticNames.STREAM_READ_BYTES, + StreamStatisticNames.STREAM_READ_EXCEPTIONS, + StreamStatisticNames.STREAM_READ_FULLY_OPERATIONS, + StreamStatisticNames.STREAM_READ_OPERATIONS, + StreamStatisticNames.STREAM_READ_OPERATIONS_INCOMPLETE, + StreamStatisticNames.STREAM_READ_VERSION_MISMATCHES, + StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS, + StreamStatisticNames.STREAM_READ_SEEK_BACKWARD_OPERATIONS, + StreamStatisticNames.STREAM_READ_SEEK_FORWARD_OPERATIONS, + StreamStatisticNames.STREAM_READ_SEEK_BYTES_BACKWARDS, + StreamStatisticNames.STREAM_READ_SEEK_BYTES_READ, + StreamStatisticNames.STREAM_READ_SEEK_BYTES_SKIPPED + ); + } + + /** + * Keys which the output stream must support. + * @return a list of keys + */ + @Override + public List outputStreamStatisticKeys() { + return Arrays.asList(STREAM_WRITE_BYTES, + STREAM_WRITE_BLOCK_UPLOADS, + STREAM_WRITE_EXCEPTIONS); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java index 75b630ae22a4b..99f6e55de5e87 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java @@ -49,7 +49,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.StorageStatistics; import org.apache.hadoop.fs.contract.ContractTestUtils; -import org.apache.hadoop.fs.impl.WrappedIOException; +import org.apache.hadoop.util.functional.RuntimeIOException; import org.apache.hadoop.fs.s3a.AWSServiceThrottledException; import org.apache.hadoop.fs.s3a.Invoker; import org.apache.hadoop.fs.s3a.S3AFileStatus; @@ -532,7 +532,7 @@ public void test_999_delete_all_entries() throws Throwable { LOG.info("Deleting {}", p); list.add(p); }); - } catch (WrappedIOException e) { + } catch (RuntimeIOException e) { // the iterator may have overloaded; swallow if so. if (!(e.getCause() instanceof AWSServiceThrottledException)) { throw e; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java index 315d1fe7285be..2549a2ba1a551 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java @@ -25,8 +25,7 @@ import com.amazonaws.event.ProgressEvent; import com.amazonaws.event.ProgressEventType; import com.amazonaws.event.ProgressListener; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.s3a.S3ATestUtils; +import org.assertj.core.api.Assertions; import org.junit.FixMethodOrder; import org.junit.Test; import org.junit.runners.MethodSorters; @@ -36,17 +35,23 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.StorageStatistics; import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.s3a.S3AFileSystem; -import org.apache.hadoop.fs.s3a.S3AInstrumentation; +import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.Statistic; +import org.apache.hadoop.fs.s3a.impl.statistics.BlockOutputStreamStatistics; +import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.util.Progressable; import static org.apache.hadoop.fs.contract.ContractTestUtils.*; import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.demandStringifyIOStatistics; +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics; +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.snapshotIOStatistics; /** * Scale test which creates a huge file. @@ -103,7 +108,7 @@ protected Configuration createScaleConfiguration() { KEY_HUGE_PARTITION_SIZE, DEFAULT_HUGE_PARTITION_SIZE); assertTrue("Partition size too small: " + partitionSize, - partitionSize > MULTIPART_MIN_SIZE); + partitionSize >= MULTIPART_MIN_SIZE); conf.setLong(SOCKET_SEND_BUFFER, _1MB); conf.setLong(SOCKET_RECV_BUFFER, _1MB); conf.setLong(MIN_MULTIPART_THRESHOLD, partitionSize); @@ -170,7 +175,7 @@ public void test_010_CreateHugeFile() throws IOException { Statistic putBytesPending = Statistic.OBJECT_PUT_BYTES_PENDING; ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer(); - S3AInstrumentation.OutputStreamStatistics streamStatistics; + BlockOutputStreamStatistics streamStatistics; long blocksPer10MB = blocksPerMB * 10; ProgressCallback progress = new ProgressCallback(timer); try (FSDataOutputStream out = fs.create(fileToCreate, @@ -221,8 +226,20 @@ public void test_010_CreateHugeFile() throws IOException { logFSState(); bandwidth(timer, filesize); LOG.info("Statistics after stream closed: {}", streamStatistics); + IOStatistics iostats = snapshotIOStatistics( + retrieveIOStatistics(getFileSystem())); + LOG.info("IOStatistics after upload: {}", + demandStringifyIOStatistics(iostats)); long putRequestCount = storageStatistics.getLong(putRequests); Long putByteCount = storageStatistics.getLong(putBytes); + Assertions.assertThat(putRequestCount) + .describedAs("Put request count from filesystem stats %s", + iostats) + .isGreaterThan(0); + Assertions.assertThat(putByteCount) + .describedAs("putByteCount count from filesystem stats %s", + iostats) + .isGreaterThan(0); LOG.info("PUT {} bytes in {} operations; {} MB/operation", putByteCount, putRequestCount, putByteCount / (putRequestCount * _1MB)); @@ -234,7 +251,7 @@ public void test_010_CreateHugeFile() throws IOException { "Put file " + fileToCreate + " of size " + filesize); if (streamStatistics != null) { assertEquals("actively allocated blocks in " + streamStatistics, - 0, streamStatistics.blocksActivelyAllocated()); + 0, streamStatistics.getBlocksActivelyAllocated()); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ILoadTestS3ABulkDeleteThrottling.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ILoadTestS3ABulkDeleteThrottling.java index a1d5c46159aaa..10405334a2066 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ILoadTestS3ABulkDeleteThrottling.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ILoadTestS3ABulkDeleteThrottling.java @@ -45,8 +45,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.ContractTestUtils; -import org.apache.hadoop.fs.impl.FunctionsRaisingIOE; -import org.apache.hadoop.fs.impl.WrappedIOException; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.auth.delegation.Csvout; @@ -316,15 +314,6 @@ private List buildDeleteRequest( return request; } - - private R wrap(FunctionsRaisingIOE.CallableRaisingIOE callable) { - try { - return callable.apply(); - } catch (IOException e) { - throw new WrappedIOException(e); - } - } - /** * Outcome of one of the load operations. */ diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java index efd96c4e7387e..5e325fa3f61d8 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java @@ -27,13 +27,20 @@ import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3AInputPolicy; import org.apache.hadoop.fs.s3a.S3AInputStream; -import org.apache.hadoop.fs.s3a.S3AInstrumentation; +import org.apache.hadoop.fs.s3a.impl.statistics.S3AInputStreamStatistics; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; +import org.apache.hadoop.fs.statistics.MeanStatistic; +import org.apache.hadoop.fs.statistics.StreamStatisticNames; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.CompressionCodecFactory; import org.apache.hadoop.util.LineReader; + +import org.assertj.core.api.Assertions; import org.junit.After; +import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -46,6 +53,17 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.*; import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatMinimumStatistic; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.lookupMaximumStatistic; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.lookupMeanStatistic; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyCounterStatisticValue; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsSourceToString; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToString; +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.snapshotIOStatistics; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.ACTION_HTTP_GET_REQUEST; +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; /** * Look at the performance of S3a operations. @@ -58,10 +76,13 @@ public class ITestS3AInputStreamPerformance extends S3AScaleTestBase { private Path testData; private FileStatus testDataStatus; private FSDataInputStream in; - private S3AInstrumentation.InputStreamStatistics streamStatistics; + private S3AInputStreamStatistics streamStatistics; public static final int BLOCK_SIZE = 32 * 1024; public static final int BIG_BLOCK_SIZE = 256 * 1024; + private static final IOStatisticsSnapshot IOSTATS = snapshotIOStatistics(); + + /** Tests only run if the there is a named test file that can be read. */ private boolean testDataAvailable = true; private String assumptionMessage = "test file"; @@ -106,9 +127,19 @@ private void bindS3aFS(Path path) throws IOException { public void cleanup() { describe("cleanup"); IOUtils.closeStream(in); + if (in != null) { + LOG.info("Stream statistics {}", + ioStatisticsSourceToString(in)); + IOSTATS.aggregate(in.getIOStatistics()); + } IOUtils.closeStream(s3aFS); } + @AfterClass + public static void dumpIOStatistics() { + LOG.info("Aggregate Stream Statistics {}", IOSTATS); + } + /** * Declare that the test requires the CSV test dataset. */ @@ -187,7 +218,7 @@ protected void assertStreamOpenedExactlyOnce() { */ private void assertOpenOperationCount(long expected) { assertEquals("open operations in\n" + in, - expected, streamStatistics.openOperations); + expected, streamStatistics.getOpenOperations()); } /** @@ -295,7 +326,7 @@ public void testLazySeekEnabled() throws Throwable { logTimePerIOP("seek()", timer, blockCount); logStreamStatistics(); assertOpenOperationCount(0); - assertEquals("bytes read", 0, streamStatistics.bytesRead); + assertEquals("bytes read", 0, streamStatistics.getBytesRead()); } @Test @@ -320,7 +351,7 @@ public void testReadWithNormalPolicy() throws Throwable { @Test public void testDecompressionSequential128K() throws Throwable { describe("Decompress with a 128K readahead"); - executeDecompression(128 * 1024, S3AInputPolicy.Sequential); + executeDecompression(128 * _1KB, S3AInputPolicy.Sequential); assertStreamOpenedExactlyOnce(); } @@ -339,9 +370,11 @@ private void executeDecompression(long readahead, int lines = 0; FSDataInputStream objectIn = openTestFile(inputPolicy, readahead); + IOStatistics readerStatistics = null; ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer(); try (LineReader lineReader = new LineReader( codec.createInputStream(objectIn), getConf())) { + readerStatistics = lineReader.getIOStatistics(); Text line = new Text(); int read; while ((read = lineReader.readLine(line)) > 0) { @@ -359,6 +392,9 @@ private void executeDecompression(long readahead, readahead); logTimePerIOP("line read", timer, lines); logStreamStatistics(); + assertNotNull("No IOStatistics through line reader", readerStatistics); + LOG.info("statistics from reader {}", + ioStatisticsToString(readerStatistics)); } private void logStreamStatistics() { @@ -391,8 +427,8 @@ protected void executeSeekReadSequence(long blockSize, readahead); logTimePerIOP("seek(pos + " + blockCount+"); read()", timer, blockCount); LOG.info("Effective bandwidth {} MB/S", - timer.bandwidthDescription(streamStatistics.bytesRead - - streamStatistics.bytesSkippedOnSeek)); + timer.bandwidthDescription(streamStatistics.getBytesRead() - + streamStatistics.getBytesSkippedOnSeek())); logStreamStatistics(); } @@ -419,7 +455,7 @@ protected void executeSeekReadSequence(long blockSize, public void testRandomIORandomPolicy() throws Throwable { executeRandomIO(S3AInputPolicy.Random, (long) RANDOM_IO_SEQUENCE.length); assertEquals("streams aborted in " + streamStatistics, - 0, streamStatistics.aborted); + 0, streamStatistics.getAborted()); } @Test @@ -427,11 +463,21 @@ public void testRandomIONormalPolicy() throws Throwable { long expectedOpenCount = RANDOM_IO_SEQUENCE.length; executeRandomIO(S3AInputPolicy.Normal, expectedOpenCount); assertEquals("streams aborted in " + streamStatistics, - 1, streamStatistics.aborted); + 1, streamStatistics.getAborted()); assertEquals("policy changes in " + streamStatistics, - 2, streamStatistics.policySetCount); + 2, streamStatistics.getPolicySetCount()); assertEquals("input policy in " + streamStatistics, - S3AInputPolicy.Random.ordinal(), streamStatistics.inputPolicy); + S3AInputPolicy.Random.ordinal(), + streamStatistics.getInputPolicy()); + IOStatistics ioStatistics = streamStatistics.getIOStatistics(); + verifyCounterStatisticValue( + ioStatistics, + StreamStatisticNames.STREAM_READ_ABORTED, + 1); + verifyCounterStatisticValue( + ioStatistics, + StreamStatisticNames.STREAM_READ_SEEK_POLICY_CHANGED, + 2); } /** @@ -466,9 +512,22 @@ private ContractTestUtils.NanoTimer executeRandomIO(S3AInputPolicy policy, assertOpenOperationCount(expectedOpenCount); logTimePerIOP("byte read", timer, totalBytesRead); LOG.info("Effective bandwidth {} MB/S", - timer.bandwidthDescription(streamStatistics.bytesRead - - streamStatistics.bytesSkippedOnSeek)); + timer.bandwidthDescription(streamStatistics.getBytesRead() - + streamStatistics.getBytesSkippedOnSeek())); logStreamStatistics(); + IOStatistics iostats = in.getIOStatistics(); + long maxHttpGet = lookupMaximumStatistic(iostats, + ACTION_HTTP_GET_REQUEST + SUFFIX_MAX); + assertThatMinimumStatistic(iostats, + ACTION_HTTP_GET_REQUEST + SUFFIX_MIN) + .isGreaterThan(0) + .isLessThan(maxHttpGet); + MeanStatistic getMeanStat = lookupMeanStatistic(iostats, + ACTION_HTTP_GET_REQUEST + SUFFIX_MEAN); + Assertions.assertThat(getMeanStat.getSamples()) + .describedAs("sample count of %s", getMeanStat) + .isEqualTo(expectedOpenCount); + return timer; } @@ -525,7 +584,7 @@ public void testRandomReadOverBuffer() throws Throwable { + " current position in stream " + currentPos + " in\n" + fs + "\n " + in, - 1, streamStatistics.openOperations); + 1, streamStatistics.getOpenOperations()); for (int i = currentPos; i < currentPos + read; i++) { assertEquals("Wrong value from byte " + i, sourceData[i], buffer[i]); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java index 42b73d3d88c09..eb8dc5cedf296 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java @@ -26,6 +26,7 @@ import org.apache.hadoop.fs.s3a.S3AInstrumentation; import org.apache.hadoop.fs.s3a.S3ATestConstants; import org.apache.hadoop.fs.s3a.Statistic; +import org.apache.hadoop.fs.s3a.impl.statistics.S3AInputStreamStatistics; import org.apache.hadoop.metrics2.lib.MutableGaugeLong; import org.slf4j.Logger; @@ -160,7 +161,7 @@ protected int getTestTimeoutMillis() { * @param in wrapper * @return the statistics for the inner stream */ - protected S3AInstrumentation.InputStreamStatistics getInputStreamStatistics( + protected S3AInputStreamStatistics getInputStreamStatistics( FSDataInputStream in) { return getS3AInputStream(in).getS3AStreamStatistics(); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3Select.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3Select.java index 64974db5a466c..29dff6e722b5c 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3Select.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3Select.java @@ -47,9 +47,9 @@ import org.apache.hadoop.fs.s3a.Constants; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3AInputStream; -import org.apache.hadoop.fs.s3a.S3AInstrumentation; import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.Statistic; +import org.apache.hadoop.fs.s3a.impl.statistics.S3AInputStreamStatistics; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapred.JobConf; @@ -302,8 +302,8 @@ public void testSelectSeek() throws Throwable { SELECT_EVERYTHING)) { SelectInputStream sis = (SelectInputStream) seekStream.getWrappedStream(); - S3AInstrumentation.InputStreamStatistics streamStats - = sis.getS3AStreamStatistics(); + S3AInputStreamStatistics streamStats = + sis.getS3AStreamStatistics(); // lazy seek doesn't raise a problem here seekStream.seek(0); assertEquals("first byte read", fullData[0], seekStream.read()); @@ -344,7 +344,7 @@ public void testSelectSeek() throws Throwable { assertEquals("byte at seek position", fullData[(int)seekStream.getPos()], seekStream.read()); assertEquals("Seek bytes skipped in " + streamStats, - seekRange, streamStats.bytesSkippedOnSeek); + seekRange, streamStats.getBytesSkippedOnSeek()); // try an invalid readahead range intercept(IllegalArgumentException.class, @@ -395,8 +395,9 @@ public void testSelectOddLinesNoHeader() throws Throwable { "SELECT * FROM S3OBJECT s WHERE s._5 = `TRUE`"); // and do a quick check on the instrumentation long bytesRead = getFileSystem().getInstrumentation() - .getCounterValue(Statistic.STREAM_SEEK_BYTES_READ); - assertNotEquals("No bytes read count", 0, bytesRead); + .getCounterValue(Statistic.STREAM_READ_BYTES); + assertNotEquals("No bytes read count in filesystem instrumentation counter", + 0, bytesRead); } @Test @@ -588,13 +589,14 @@ public void testCloseWithAbort() throws Throwable { stream.setReadahead(1L); assertEquals("Readahead on " + sis, 1, sis.getReadahead()); stream.read(); - S3AInstrumentation.InputStreamStatistics stats - = sis.getS3AStreamStatistics(); + S3AInputStreamStatistics stats + = (S3AInputStreamStatistics) + sis.getS3AStreamStatistics(); assertEquals("Read count in " + sis, - 1, stats.bytesRead); + 1, stats.getBytesRead()); stream.close(); assertEquals("Abort count in " + sis, - 1, stats.aborted); + 1, stats.getAborted()); readOps.assertDiffEquals("Read operations are still considered active", 0); intercept(PathIOException.class, FSExceptionMessages.STREAM_IS_CLOSED, @@ -608,12 +610,14 @@ public void testCloseWithNoAbort() throws Throwable { "SELECT * FROM S3OBJECT s"); stream.setReadahead(0x1000L); SelectInputStream sis = (SelectInputStream) stream.getWrappedStream(); - S3AInstrumentation.InputStreamStatistics stats - = sis.getS3AStreamStatistics(); + S3AInputStreamStatistics stats + = (S3AInputStreamStatistics) + sis.getS3AStreamStatistics(); stream.close(); - assertEquals("Close count in " + sis, 1, stats.closed); - assertEquals("Abort count in " + sis, 0, stats.aborted); - assertTrue("No bytes read in close of " + sis, stats.bytesReadInClose > 0); + assertEquals("Close count in " + sis, 1, stats.getClosed()); + assertEquals("Abort count in " + sis, 0, stats.getAborted()); + assertTrue("No bytes read in close of " + sis, + stats.getBytesReadInClose() > 0); } @Test diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectLandsat.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectLandsat.java index 2099edd248b63..0322ff142db49 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectLandsat.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectLandsat.java @@ -35,8 +35,8 @@ import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy.Source; +import org.apache.hadoop.fs.s3a.impl.statistics.S3AInputStreamStatistics; import org.apache.hadoop.fs.s3a.S3AFileSystem; -import org.apache.hadoop.fs.s3a.S3AInstrumentation; import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.Statistic; import org.apache.hadoop.mapred.JobConf; @@ -381,7 +381,7 @@ public void testSelectSeekFullLandsat() throws Throwable { SELECT_EVERYTHING)) { SelectInputStream sis = (SelectInputStream) seekStream.getWrappedStream(); - S3AInstrumentation.InputStreamStatistics streamStats + S3AInputStreamStatistics streamStats = sis.getS3AStreamStatistics(); // lazy seek doesn't raise a problem here seekStream.seek(0); @@ -410,7 +410,7 @@ public void testSelectSeekFullLandsat() throws Throwable { assertEquals("byte at seek position", dataset[(int) seekStream.getPos()], seekStream.read()); assertEquals("Seek bytes skipped in " + streamStats, - seekRange, streamStats.bytesSkippedOnSeek); + seekRange, streamStats.getBytesSkippedOnSeek()); long offset; long increment = 64 * _1KB; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectMRJob.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectMRJob.java index 181d797767397..853b581a5a491 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectMRJob.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectMRJob.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.concurrent.atomic.AtomicLong; +import org.apache.hadoop.util.functional.RuntimeIOException; import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy.Source; import org.junit.Assume; @@ -30,7 +31,6 @@ import org.apache.hadoop.examples.WordCount; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.impl.FutureIOSupport; -import org.apache.hadoop.fs.impl.WrappedIOException; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.S3AUtils; @@ -209,7 +209,7 @@ private String readStringFromFile(Path path) throws IOException { IOUtils.readFully(in, buffer, 0, bytesLen); return new String(buffer); } catch (IOException ex) { - throw new WrappedIOException(ex); + throw new RuntimeIOException(ex); } })); } diff --git a/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties index 6e20fbcda7efd..fd354dd1e38dc 100644 --- a/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties +++ b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties @@ -75,3 +75,11 @@ log4j.logger.org.apache.hadoop.fs.s3a.s3guard.Operations=DEBUG log4j.logger.org.apache.hadoop.mapreduce.lib.output=DEBUG log4j.logger.org.apache.hadoop.fs.s3a.S3AStorageStatistics=INFO + +# Set to debug if you need to debug S3A endpoint problems. +#log4j.logger.org.apache.hadoop.fs.s3a.DefaultS3ClientFactory=DEBUG + +# This causes all remote iterator stats +# to be logged when the RemoteIterators.foreach() method is +# invoked +log4j.logger.org.apache.hadoop.util.functional.RemoteIterators=DEBUG \ No newline at end of file