diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BufferedFSInputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BufferedFSInputStream.java index 973b136bb3ab2..026eff427c9f7 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BufferedFSInputStream.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BufferedFSInputStream.java @@ -24,6 +24,10 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; + +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics; /** @@ -33,7 +37,8 @@ @InterfaceAudience.Private @InterfaceStability.Unstable public class BufferedFSInputStream extends BufferedInputStream -implements Seekable, PositionedReadable, HasFileDescriptor { +implements Seekable, PositionedReadable, HasFileDescriptor, + IOStatisticsSource { /** * Creates a BufferedFSInputStream * with the specified buffer size, @@ -126,4 +131,9 @@ public FileDescriptor getFileDescriptor() throws IOException { return null; } } + + @Override + public IOStatistics getIOStatistics() { + return retrieveIOStatistics(in); + } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java index cc9c284c9fa55..70bf5e177529e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java @@ -38,6 +38,9 @@ import org.apache.hadoop.fs.impl.OpenFileParameters; import org.apache.hadoop.fs.permission.AclEntry; import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.fs.statistics.IOStatisticsSupport; import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.LambdaUtils; import org.apache.hadoop.util.Progressable; @@ -134,7 +137,8 @@ private int getSumBufferSize(int bytesPerSum, int bufferSize) { * For open()'s FSInputStream * It verifies that data matches checksums. *******************************************************/ - private static class ChecksumFSInputChecker extends FSInputChecker { + private static class ChecksumFSInputChecker extends FSInputChecker implements + IOStatisticsSource { private ChecksumFileSystem fs; private FSDataInputStream datas; private FSDataInputStream sums; @@ -270,6 +274,17 @@ protected int readChunk(long pos, byte[] buf, int offset, int len, } return nread; } + + /** + * Get the IO Statistics of the nested stream, falling back to + * null if the stream does not implement the interface + * {@link IOStatisticsSource}. + * @return an IOStatistics instance or null + */ + @Override + public IOStatistics getIOStatistics() { + return IOStatisticsSupport.retrieveIOStatistics(datas); + } } private static class FSDataBoundedInputStream extends FSDataInputStream { @@ -395,7 +410,8 @@ public static long getChecksumLength(long size, int bytesPerSum) { /** This class provides an output stream for a checksummed file. * It generates checksums for data. */ - private static class ChecksumFSOutputSummer extends FSOutputSummer { + private static class ChecksumFSOutputSummer extends FSOutputSummer + implements IOStatisticsSource { private FSDataOutputStream datas; private FSDataOutputStream sums; private static final float CHKSUM_AS_FRACTION = 0.01f; @@ -449,6 +465,17 @@ protected void checkClosed() throws IOException { throw new ClosedChannelException(); } } + + /** + * Get the IO Statistics of the nested stream, falling back to + * null if the stream does not implement the interface + * {@link IOStatisticsSource}. + * @return an IOStatistics instance or null + */ + @Override + public IOStatistics getIOStatistics() { + return IOStatisticsSupport.retrieveIOStatistics(datas); + } } @Override diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataInputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataInputStream.java index 31f82975899e1..b63e047358c70 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataInputStream.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataInputStream.java @@ -29,6 +29,9 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.fs.statistics.IOStatisticsSupport; import org.apache.hadoop.io.ByteBufferPool; import org.apache.hadoop.util.IdentityHashStore; @@ -40,7 +43,7 @@ public class FSDataInputStream extends DataInputStream implements Seekable, PositionedReadable, ByteBufferReadable, HasFileDescriptor, CanSetDropBehind, CanSetReadahead, HasEnhancedByteBufferAccess, CanUnbuffer, StreamCapabilities, - ByteBufferPositionedReadable { + ByteBufferPositionedReadable, IOStatisticsSource { /** * Map ByteBuffers that we have handed out to readers to ByteBufferPool * objects @@ -267,4 +270,15 @@ public void readFully(long position, ByteBuffer buf) throws IOException { "unsupported by " + in.getClass().getCanonicalName()); } } + + /** + * Get the IO Statistics of the nested stream, falling back to + * null if the stream does not implement the interface + * {@link IOStatisticsSource}. + * @return an IOStatistics instance or null + */ + @Override + public IOStatistics getIOStatistics() { + return IOStatisticsSupport.retrieveIOStatistics(in); + } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStream.java index 5b604e58e2360..27d164b7d87ba 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStream.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStream.java @@ -24,13 +24,17 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.fs.statistics.IOStatisticsSupport; /** Utility that wraps a {@link OutputStream} in a {@link DataOutputStream}. */ @InterfaceAudience.Public @InterfaceStability.Stable public class FSDataOutputStream extends DataOutputStream - implements Syncable, CanSetDropBehind, StreamCapabilities { + implements Syncable, CanSetDropBehind, StreamCapabilities, + IOStatisticsSource { private final OutputStream wrappedStream; private static class PositionCache extends FilterOutputStream { @@ -155,4 +159,15 @@ public void setDropBehind(Boolean dropBehind) throws IOException { "not support setting the drop-behind caching setting."); } } + + /** + * Get the IO Statistics of the nested stream, falling back to + * empty statistics if the stream does not implement the interface + * {@link IOStatisticsSource}. + * @return an IOStatistics instance. + */ + @Override + public IOStatistics getIOStatistics() { + return IOStatisticsSupport.retrieveIOStatistics(wrappedStream); + } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSInputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSInputStream.java index 672ab15f16c3b..373120ce3078f 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSInputStream.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSInputStream.java @@ -24,6 +24,9 @@ import com.google.common.base.Preconditions; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.statistics.IOStatisticsLogging; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -134,4 +137,23 @@ public void readFully(long position, byte[] buffer) throws IOException { readFully(position, buffer, 0, buffer.length); } + + /** + * toString method returns the superclass toString, but if the subclass + * implements {@link IOStatisticsSource} then those statistics are + * extracted and included in the output. + * That is: statistics of subclasses are automatically reported. + * @return a string value. + */ + @Override + public String toString() { + final StringBuilder sb = new StringBuilder(super.toString()); + sb.append('{'); + if (this instanceof IOStatisticsSource) { + sb.append(IOStatisticsLogging.ioStatisticsSourceToString( + (IOStatisticsSource) this)); + } + sb.append('}'); + return sb.toString(); + } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java index cf2210575da15..7e26378128059 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java @@ -47,6 +47,10 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.fs.statistics.impl.BufferedIOStatisticsOutputStream; +import org.apache.hadoop.fs.statistics.impl.CounterIOStatistics; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.nativeio.NativeIO; import org.apache.hadoop.util.Progressable; @@ -54,6 +58,14 @@ import org.apache.hadoop.util.StringUtils; import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BYTES; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_EXCEPTIONS; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_SKIP_BYTES; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_SKIP_OPERATIONS; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_WRITE_BYTES; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_WRITE_EXCEPTIONS; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.counterIOStatistics; /**************************************************************** * Implement the FileSystem API for the raw local filesystem. @@ -105,10 +117,21 @@ public void initialize(URI uri, Configuration conf) throws IOException { /******************************************************* * For open()'s FSInputStream. *******************************************************/ - class LocalFSFileInputStream extends FSInputStream implements HasFileDescriptor { + class LocalFSFileInputStream extends FSInputStream implements HasFileDescriptor, + IOStatisticsSource { private FileInputStream fis; private long position; + /** + * Minimal set of counters. + */ + private final CounterIOStatistics ioStatistics = counterIOStatistics( + STREAM_READ_BYTES, + STREAM_READ_EXCEPTIONS, + STREAM_READ_SEEK_OPERATIONS, + STREAM_READ_SKIP_OPERATIONS, + STREAM_READ_SKIP_BYTES); + public LocalFSFileInputStream(Path f) throws IOException { fis = new FileInputStream(pathToFile(f)); } @@ -150,9 +173,11 @@ public int read() throws IOException { if (value >= 0) { this.position++; statistics.incrementBytesRead(1); + ioStatistics.increment(STREAM_READ_BYTES, 1); } return value; } catch (IOException e) { // unexpected exception + ioStatistics.increment(STREAM_READ_EXCEPTIONS, 1); throw new FSError(e); // assume native fs error } } @@ -166,9 +191,11 @@ public int read(byte[] b, int off, int len) throws IOException { if (value > 0) { this.position += value; statistics.incrementBytesRead(value); + ioStatistics.increment(STREAM_READ_BYTES, value); } return value; } catch (IOException e) { // unexpected exception + ioStatistics.increment(STREAM_READ_EXCEPTIONS, 1); throw new FSError(e); // assume native fs error } } @@ -187,18 +214,22 @@ public int read(long position, byte[] b, int off, int len) int value = fis.getChannel().read(bb, position); if (value > 0) { statistics.incrementBytesRead(value); + ioStatistics.increment(STREAM_READ_BYTES, value); } return value; } catch (IOException e) { + ioStatistics.increment(STREAM_READ_EXCEPTIONS, 1); throw new FSError(e); } } @Override public long skip(long n) throws IOException { + ioStatistics.increment(STREAM_READ_SKIP_OPERATIONS, 1); long value = fis.skip(n); if (value > 0) { this.position += value; + ioStatistics.increment(STREAM_READ_SKIP_BYTES, value); } return value; } @@ -207,6 +238,11 @@ public long skip(long n) throws IOException { public FileDescriptor getFileDescriptor() throws IOException { return fis.getFD(); } + + @Override + public IOStatistics getIOStatistics() { + return ioStatistics; + } } @Override @@ -231,9 +267,17 @@ public FSDataInputStream open(PathHandle fd, int bufferSize) /********************************************************* * For create()'s FSOutputStream. *********************************************************/ - class LocalFSFileOutputStream extends OutputStream { + class LocalFSFileOutputStream extends OutputStream implements + IOStatisticsSource { private FileOutputStream fos; - + + /** + * Minimal set of counters. + */ + private final CounterIOStatistics ioStatistics = counterIOStatistics( + STREAM_WRITE_BYTES, + STREAM_WRITE_EXCEPTIONS); + private LocalFSFileOutputStream(Path f, boolean append, FsPermission permission) throws IOException { File file = pathToFile(f); @@ -273,7 +317,9 @@ private LocalFSFileOutputStream(Path f, boolean append, public void write(byte[] b, int off, int len) throws IOException { try { fos.write(b, off, len); + ioStatistics.increment(STREAM_WRITE_BYTES, len); } catch (IOException e) { // unexpected exception + ioStatistics.increment(STREAM_WRITE_EXCEPTIONS, 1); throw new FSError(e); // assume native fs error } } @@ -282,10 +328,17 @@ public void write(byte[] b, int off, int len) throws IOException { public void write(int b) throws IOException { try { fos.write(b); + ioStatistics.increment(STREAM_WRITE_BYTES, 1); } catch (IOException e) { // unexpected exception + ioStatistics.increment(STREAM_WRITE_EXCEPTIONS, 1); throw new FSError(e); // assume native fs error } } + + @Override + public IOStatistics getIOStatistics() { + return ioStatistics; + } } @Override @@ -318,7 +371,7 @@ private FSDataOutputStream create(Path f, boolean overwrite, if (parent != null && !mkdirs(parent)) { throw new IOException("Mkdirs failed to create " + parent.toString()); } - return new FSDataOutputStream(new BufferedOutputStream( + return new FSDataOutputStream(new BufferedIOStatisticsOutputStream( createOutputStreamWithMode(f, false, permission), bufferSize), statistics); } @@ -340,7 +393,7 @@ public FSDataOutputStream createNonRecursive(Path f, FsPermission permission, if (exists(f) && !flags.contains(CreateFlag.OVERWRITE)) { throw new FileAlreadyExistsException("File already exists: " + f); } - return new FSDataOutputStream(new BufferedOutputStream( + return new FSDataOutputStream(new BufferedIOStatisticsOutputStream( createOutputStreamWithMode(f, false, permission), bufferSize), statistics); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageStatistics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageStatistics.java index 74631b5695537..2efe4566344ee 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageStatistics.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageStatistics.java @@ -19,6 +19,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.statistics.StoreStatisticNames; import java.util.Iterator; @@ -27,15 +28,16 @@ * instance. */ @InterfaceAudience.Public +@InterfaceStability.Stable public abstract class StorageStatistics { /** * These are common statistic names. - * + *

* The following names are considered general and preserved across different * StorageStatistics classes. When implementing a new StorageStatistics, it is * highly recommended to use the common statistic names. - * + *

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

+ * 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/statistics/IOStatisticEntry.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticEntry.java new file mode 100644 index 0000000000000..d614446a13d7b --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticEntry.java @@ -0,0 +1,316 @@ +/* + * 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.Arrays; +import java.util.Objects; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; +import static org.apache.commons.lang3.StringUtils.join; + +/** + * Entry of an IOStatistic. + *

+ * it's a pair of (type, values[]) such that it is + * trivially serializable and we can add new types + * with multiple entries. + *

+ * For example, the data for a mean value would be + * (mean, sample count); + * for a maximum it would just be the max value. + * What is key is that each entry MUST provide all + * the data needed to aggregate two entries together. + *

+ * This isn't perfect, as we'd really want a union of types, + * so doubles could be passed round too. + * The length of the array can be queried with the {@link #arity()} + * method; the {@code _1(), _2()} and {@code _3()} methods return + * the first, second and third entries in the array + * respectively. + *

+ * In this context, arity is the number of elements in the array. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public final class IOStatisticEntry implements Serializable { + + /** + * Counter type: {@value}. + *

+ * Arity 1: (count) + *

+ * aggregation {@code l._1() + r._1()} + */ + public static final int IOSTATISTIC_COUNTER = 0; + + + /** + * Min value type: {@value}. + *

+ * Arity 1: (min) + *

+ * Aggregation {@code min(l._1(), r._1()} + */ + public static final int IOSTATISTIC_MIN = 1; + + /** + * Max value type: {@value}. + *

+ * Arity 1: (max) + *

+ * Aggregation {@code max(l._1(), r._1()} + */ + public static final int IOSTATISTIC_MAX = 2; + + /** + * Counter type: {@value}. + *

+ * Arity 2: (mean, sample-count) + *

+ * Aggregation: + * {@code (l._1() * l._2() + r._1() * r._2()) / (l._2() + r._2())} + */ + public static final int IOSTATISTIC_MEAN = 3; + + /** + * Names of the known types. + */ + private static final String[] TYPE_NAMES = { + "counter", + "min", + "max", + "mean"}; + + /** + * Arity of the known types. + */ + private static final int[] TYPE_ARITY = {1, 1, 1, 2}; + + /** + * Names of the known types. + */ + private static final String[] TYPE_FORMATS = { + "counter", + "min", + "max", + "mean" + }; + + /** + * Serialization number. + */ + private static final long serialVersionUID = 5925626116768440629L; + + /** + * Type of statistic. + */ + private int type; + + /** + * data array. + */ + private long[] data; + + /** + * Instantiate from an array of long values. + * This includes a verification that the + * length of the array matches that expected of the given + * type (if it is known) + * @param type type of entry + * @param data values + */ + public IOStatisticEntry(final int type, final long[] data) { + checkArgument(type >= 0, + "type out of range %d", type); + this.type = type; + this.data = Objects.requireNonNull(data); + int len = data.length; + checkArgument(len > 0, + "entry data array is empty"); + if (type < TYPE_ARITY.length) { + checkArgument(len == TYPE_ARITY[type], + "arity value of %s does not match required value: %s", + len, TYPE_ARITY[type]); + } + } + + /** + * Empty constructor. + * This is for serialization rather than general use. + * Until the type and values are set, the entry is not + * valid. + */ + public IOStatisticEntry() { + type = -1; + data = new long[0]; + } + + /** + * Get the type of the entry. + * @return the type + */ + public int type() { + return type; + } + + + /** + * Get the length of the value array, i.e arity of the instance. + * @return the length of the data. + */ + public int arity() { + return data.length; + } + + /** + * Get the array of values. + * This is a mutable list, not a copy. + * @return the current list of values. + */ + public long[] getData() { + return data; + } + + + /** + * Require the type value of the instance to + * be of the specified value. + * @param r required tyoe. + * @throws IllegalStateException if the requirement is not met. + */ + public void requireType(int r) { + checkState(type == r, + "Required type=%d actual type=%d", + r, type); + } + + /** + * Required the arity to match that expected. + * @param r required arity. + * @throws IllegalStateException if the requirement is not met. + */ + public void requireArity(int r) { + checkState(arity() == r, + "Required length=%d actual length=%d", + r, arity()); + } + + /** + * Requre the type and arity of the two entries to match + * @param that the other entry. + * @throws IllegalStateException if the requirement is not met. + */ + public void requireCompatible(IOStatisticEntry that) { + that.requireTypeAndArity(type, arity()); + } + + /** + * Require the specific type and arity. + * @param t type + * @param a arity + */ + public void requireTypeAndArity(final int t, final int a) { + requireType(t); + requireArity(a); + } + + /** + * First entry in the tuple. + * @return a tuple element value + * @throws IndexOutOfBoundsException if the array is too short. + */ + public long _1() { + return data[0]; + } + + /** + * Second entry in the tuple. + * @return a tuple element value + * @throws IndexOutOfBoundsException if the array is too short. + */ + public long _2() { + return data[1]; + } + + /** + * Third entry in the tuple. + * @return a tuple element value + * @throws IndexOutOfBoundsException if the array is too short. + */ + public long _3() { + return data[2]; + } + + /** + * Get the single tuple value, verifying + * that the type is as expected. + * @param requiredType required type + * @return the value + */ + public long scalar(int requiredType) { + requireArity(1); + requireType(requiredType); + return _1(); + } + + /** + * Type to string; if it is a known type this + * will be a meaningful string. + * @return a value for logging. + */ + public String typeAsString() { + return type < TYPE_NAMES.length + ? TYPE_NAMES[type] + : Integer.toString(type); + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder(); + sb.append("(").append(typeAsString()); + sb.append(", (").append(join(data, ',')); + sb.append(')'); + return sb.toString(); + } + + @Override + public boolean equals(final Object o) { + if (this == o) { return true; } + if (o == null || getClass() != o.getClass()) { return false; } + IOStatisticEntry that = (IOStatisticEntry) o; + return type == that.type && + Arrays.equals(data, that.data); + } + + /** + * Entry with data varags. + * @param t type + * @param d data + * @return new entry + */ + public static IOStatisticEntry statsEntry(int t, long...d) { + return new IOStatisticEntry(t, d); + } + +} 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..5380447ecc49e --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatistics.java @@ -0,0 +1,109 @@ +/* + * 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 java.util.Set; + +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. + *

+ * The statistics MUST BE for the specific instance of the source; + * possibly including aggregate statistics from other objects + * created by that stores. + * For example, the statistics of a filesystem instance must be unique + * to that instant and not shared with any other. + * However, those statistics may also collect and aggregate statistics + * generated in the use of input and output streams created by that + * file system instance. + * + *

+ * The iterator is a possibly empty iterator over all monitored statistics. + *

    + *
  1. + * The set of statistic keys SHOULD remain unchanged, and MUST NOT + * remove keys. + *
  2. + *
  3. + * The statistics SHOULD be dynamic: every call to {@code iterator()} + * MAY return a current/recent set of statistics. + *
  4. + *
  5. + * The values MAY change across invocations of {@code iterator()}. + *
  6. + *
  7. + * The update MAY be in the iterable() call, or MAY be in the actual + * Iterable.next() operation. + *
  8. + *
  9. + * The returned Map.Entry instances MUST return the same value on + * repeated getValue() calls. + *
  10. + *
  11. + * Queries of statistics SHOULD Be fast and Nonblocking to the extent + * that if invoked during a long operation, they will prioritize + * returning fast over most timely values. + *
  12. + *
  13. + * The statistics MAY lag; especially for statistics collected in separate + * operations (e.g stream IO statistics as provided by a filesystem + * instance). + *
  14. + *
  15. + * Thread safety: an instance of IOStatistics can be shared across threads; + * a call to {@code iterator()} is thread safe. + * The actual Iterable returned MUST NOT be shared across threads. + *
  16. + * + *
+ */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public interface IOStatistics + extends Iterable> { + + /** + * Get the entry of a statistic. + * + * @return The entry of the statistic, or null if not tracked. + */ + IOStatisticEntry getStatistic(String key); + + /** + * Return true if a statistic is being tracked. + * + * @return True only if the statistic is being tracked. + */ + boolean isTracked(String key); + + /** + * Get the set of keys. + * No guarantees are made about the mutability/immutability + * of this set. + * @return the set of keys. + */ + Set keys(); + +} 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..29aedc82638d8 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsLogging.java @@ -0,0 +1,168 @@ +/* + * 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.IOStatisticsImplementationUtils; + +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsImplementationUtils.entrytoString; + +/** + * 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() { + } + + /** + * 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) { + int count = 0; + StringBuilder sb = new StringBuilder("("); + for (Map.Entry entry : statistics) { + if (count > 0) { + sb.append(' '); + } + count++; + sb.append(entrytoString(entry.getKey(), entry.getValue())); + } + sb.append(")"); + return sb.toString(); + } else { + return ""; + } + } + + /** + * Extract the statistics from a source object -or "" + * if it is not a source of statistics + *

+ * Exceptions are caught and downgraded to debug logging. + * @param source source of statistics. + * @return a string for logging. + */ + public static String ioStatisticsSourceToString(@Nullable Object source) { + try { + return iostatisticsToString(retrieveIOStatistics(source)); + } catch (RuntimeException e) { + LOG.debug("Ignoring", e); + return ""; + } + } + + /** + * On demand stringifier. + *

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

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

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

+ * This is for use in log statements where for the cost of creation + * of this entry is low; it is affordable to use in log statements. + * @param statistics statistics to scan. + * @return an object whose toString() operation returns the current values. + */ + public static Object demandStringify(@Nullable IOStatistics statistics) { + return new StatisticsToString(statistics); + } + + /** + * On demand stringifier. + *

+ * Whenever this object's toString() method is called, it + * retrieves the latest statistics instance and re-evaluates it. + */ + private static final class SourceToString { + + private final IOStatisticsSource source; + + private SourceToString(@Nullable IOStatisticsSource source) { + this.source = source; + } + + @Override + public String toString() { + return source != null + ? ioStatisticsSourceToString(source) + : IOStatisticsImplementationUtils.NULL_SOURCE; + } + } + + /** + * Stringifier of statistics: low cost to instantiate and every + * toString/logging will re-evaluate the statistics. + */ + private static final class StatisticsToString { + + private final IOStatistics statistics; + + /** + * Constructor. + * @param statistics statistics + */ + private StatisticsToString(@Nullable IOStatistics statistics) { + this.statistics = statistics; + } + + /** + * Evaluate and stringify the statistics. + * @return a string value. + */ + @Override + public String toString() { + return statistics != null + ? iostatisticsToString(statistics) + : IOStatisticsImplementationUtils.NULL_SOURCE; + } + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSource.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSource.java new file mode 100644 index 0000000000000..260853d1e2f39 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSource.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics; + +import org.apache.hadoop.classification.InterfaceStability; + +/** + * A source of IO statistics. + *

+ * These statistics MUST be instance specific, not thread local. + */ +@InterfaceStability.Unstable +public interface IOStatisticsSource { + + /** + * Return a statistics instance. + *

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

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

+ * This is not an atomic option. + *

+ * The instance can be serialized, and its + * {@code toString()} method lists all the values. + * @param statistics statistics + * @return a snapshot of the current values. + */ + public static X snapshot( + IOStatistics statistics) { + return IOStatisticsBinding.snapshotStatistics(statistics); + } + + + /** + * Get the IOStatistics of the source, falling back to + * null if the source does not implement + * {@link IOStatisticsSource}, or the return value + * of {@link IOStatisticsSource#getIOStatistics()} was null. + * @return an IOStatistics instance or null + */ + + public static IOStatistics retrieveIOStatistics( + final Object source) { + return (source instanceof IOStatisticsSource) + ? ((IOStatisticsSource) source).getIOStatistics() + : null; // null source or interface not implemented + } + +} 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..9510b010858a6 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.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; + +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.
  • + *
+ */ +@InterfaceStability.Unstable +public final class StoreStatisticNames { + + + private StoreStatisticNames() { + } + + /** {@value}. */ + public static final String OP_APPEND = "op_append"; + + /** {@value}. */ + public static final String OP_COPY_FROM_LOCAL_FILE = + "op_copy_from_local_file"; + + /** {@value}. */ + public static final String OP_CREATE = "op_create"; + + /** {@value}. */ + public static final String OP_CREATE_NON_RECURSIVE = + "op_create_non_recursive"; + + /** {@value}. */ + public static final String OP_DELETE = "op_delete"; + + /** {@value}. */ + public static final String OP_EXISTS = "op_exists"; + + /** {@value}. */ + public static final String OP_GET_CONTENT_SUMMARY = + "op_get_content_summary"; + + /** {@value}. */ + public static final String OP_GET_DELEGATION_TOKEN = + "op_get_delegation_token"; + + /** {@value}. */ + public static final String OP_GET_FILE_CHECKSUM = + "op_get_file_checksum"; + + /** {@value}. */ + public static final String OP_GET_FILE_STATUS = "op_get_file_status"; + + /** {@value}. */ + public static final String OP_GET_STATUS = "op_get_status"; + + /** {@value}. */ + public static final String OP_GLOB_STATUS = "op_glob_status"; + + /** {@value}. */ + public static final String OP_IS_FILE = "op_is_file"; + + /** {@value}. */ + public static final String OP_IS_DIRECTORY = "op_is_directory"; + + /** {@value}. */ + public static final String OP_LIST_FILES = "op_list_files"; + + /** {@value}. */ + public static final String OP_LIST_LOCATED_STATUS = + "op_list_located_status"; + + /** {@value}. */ + public static final String OP_LIST_STATUS = "op_list_status"; + + /** {@value}. */ + public static final String OP_MKDIRS = "op_mkdirs"; + + /** {@value}. */ + public static final String OP_MODIFY_ACL_ENTRIES = "op_modify_acl_entries"; + + /** {@value}. */ + public static final String OP_OPEN = "op_open"; + + /** {@value}. */ + public static final String OP_REMOVE_ACL = "op_remove_acl"; + + /** {@value}. */ + public static final String OP_REMOVE_ACL_ENTRIES = "op_remove_acl_entries"; + + /** {@value}. */ + public static final String OP_REMOVE_DEFAULT_ACL = "op_remove_default_acl"; + + /** {@value}. */ + public static final String OP_RENAME = "op_rename"; + + /** {@value}. */ + public static final String OP_SET_ACL = "op_set_acl"; + + /** {@value}. */ + public static final String OP_SET_OWNER = "op_set_owner"; + + /** {@value}. */ + public static final String OP_SET_PERMISSION = "op_set_permission"; + + /** {@value}. */ + public static final String OP_SET_TIMES = "op_set_times"; + + /** {@value}. */ + public static final String OP_TRUNCATE = "op_truncate"; + + /** {@value}. */ + public static final String DELEGATION_TOKENS_ISSUED + = "delegation_tokens_issued"; + + /** Requests throttled and retried: {@value}. */ + public static final String STORE_IO_THROTTLED + = "store_io_throttled"; + + /** Requests made of a store: {@value}. */ + public static final String STORE_IO_REQUEST + = "store_io_request"; + + /** + * IO retried: {@value}. + */ + public static final String STORE_IO_RETRY + = "store_io_retry"; + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StreamStatisticNames.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StreamStatisticNames.java new file mode 100644 index 0000000000000..9c37426e6f702 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StreamStatisticNames.java @@ -0,0 +1,165 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics; + +import org.apache.hadoop.classification.InterfaceStability; + +/** + * These are common statistic names. + *

+ * 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. + *
  • + *
+ */ +@InterfaceStability.Unstable +public final class StreamStatisticNames { + + private StreamStatisticNames() { + } + + /** {@value}. */ + public static final String STREAM_READ_ABORTED = "stream_aborted"; + + /** {@value}. */ + public static final String STREAM_READ_BYTES_DISCARDED_ABORT + = "stream_read_bytes_discarded_in_abort"; + + /** {@value}. */ + public static final String STREAM_READ_BYTES_DISCARDED_SEEK + = "stream_read_bytes_discarded_in_seek"; + + /** {@value}. */ + public static final String STREAM_READ_CLOSED = "stream_read_closed"; + + /** {@value}. */ + public static final String STREAM_READ_CLOSE_BYTES_READ + = "stream_read_bytes_read_in_close"; + + /** {@value}. */ + public static final String STREAM_READ_CLOSE_OPERATIONS + = "stream_read_close_operations"; + + /** {@value}. */ + public static final String STREAM_READ_OPENED = "stream_read_opened"; + + /** {@value}. */ + public static final String STREAM_READ_BYTES + = "stream_read_bytes"; + + /** {@value}. */ + public static final String STREAM_READ_EXCEPTIONS = + "stream_read_exceptions"; + + /** {@value}. */ + public static final String STREAM_READ_FULLY_OPERATIONS + = "stream_read_fully_operations"; + + /** {@value}. */ + public static final String STREAM_READ_OPERATIONS = + "stream_read_operations"; + + /** {@value}. */ + public static final String STREAM_READ_OPERATIONS_INCOMPLETE + = "stream_read_operations_incomplete"; + + /** {@value}. */ + public static final String STREAM_READ_VERSION_MISMATCHES + = "stream_read_version_mismatches"; + + /** {@value}. */ + public static final String STREAM_READ_SEEK_BYTES_BACKWARDS + = "stream_read_bytes_backwards_on_seek"; + + /** {@value}. */ + public static final String STREAM_READ_SEEK_FORWARD_OPERATIONS + = "stream_read_seek_forward_operations"; + + /** {@value}. */ + public static final String STREAM_READ_SEEK_OPERATIONS = + "stream_read_seek_operations"; + + /** {@value}. */ + public static final String STREAM_READ_SEEK_BACKWARD_OPERATIONS = + "stream_read_seek_backward_operations"; + + /** {@value}. */ + public static final String STREAM_READ_SEEK_BYTES_READ = + "stream_read_seek_bytes_read"; + + /** {@value}. */ + public static final String STREAM_READ_SEEK_BYTES_SKIPPED + = "stream_read_bytes_skipped_on_seek"; + + /** {@value}. */ + public static final String STREAM_READ_SKIP_OPERATIONS = + "stream_read_skip_operations"; + + /** {@value}. */ + public static final String STREAM_READ_SKIP_BYTES = + "stream_read_skip_bytes"; + + /** {@value}. */ + public static final String STREAM_WRITE_EXCEPTIONS = + "stream_write_exceptions"; + + /** {@value}. */ + public static final String STREAM_WRITE_BLOCK_UPLOADS + = "stream_write_block_uploads"; + + /** {@value}. */ + public static final String STREAM_WRITE_BLOCK_UPLOADS_ABORTED + = "stream_write_block_uploads_aborted"; + + /** {@value}. */ + public static final String STREAM_WRITE_BLOCK_UPLOADS_ACTIVE + = "stream_write_block_uploads_active"; + + /** {@value}. */ + public static final String STREAM_WRITE_BLOCK_UPLOADS_COMMITTED + = "stream_write_block_uploads_committed"; + + /** {@value}. */ + public static final String STREAM_WRITE_BLOCK_UPLOADS_PENDING + = "stream_write_block_uploads_pending"; + + /** {@value}. */ + public static final String STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING = + "stream_write_block_uploads_data_pending"; + + /** {@value}. */ + public static final String STREAM_WRITE_BYTES + = "stream_write_bytes"; + + /** {@value}. */ + public static final String STREAM_WRITE_TOTAL_TIME + = "stream_write_total_time"; + + /** {@value}. */ + public static final String STREAM_WRITE_QUEUE_DURATION + = "stream_write_queue_duration"; + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/BufferedIOStatisticsInputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/BufferedIOStatisticsInputStream.java new file mode 100644 index 0000000000000..3412fdd3be25e --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/BufferedIOStatisticsInputStream.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics.impl; + +import java.io.BufferedInputStream; +import java.io.InputStream; + +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; + +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics; + +/** + * An extension of {@code BufferedInputStream} which implements + * {@link IOStatisticsSource} and forwards requests for the + * {@link IOStatistics} to the wrapped stream. + * This should be used when any input stream needs buffering while + * allowing the inner stream to be a source of statistics. + */ +public class BufferedIOStatisticsInputStream + extends BufferedInputStream + implements IOStatisticsSource { + + public BufferedIOStatisticsInputStream(final InputStream in) { + super(in); + } + + public BufferedIOStatisticsInputStream(final InputStream in, final int size) { + super(in, size); + } + + @Override + public IOStatistics getIOStatistics() { + return retrieveIOStatistics(in); + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/BufferedIOStatisticsOutputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/BufferedIOStatisticsOutputStream.java new file mode 100644 index 0000000000000..4b81365fb94ff --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/BufferedIOStatisticsOutputStream.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics.impl; + +import java.io.BufferedOutputStream; +import java.io.OutputStream; + +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; + +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics; + +/** + * An extension of {@code BufferedOutputStream} which implements + * {@link IOStatisticsSource} and forwards requests for the + * {@link IOStatistics} to the wrapped stream. + * This should be used when any output stream needs buffering while + * allowing the inner stream to be a source of statistics. + */ +public class BufferedIOStatisticsOutputStream + extends BufferedOutputStream + implements IOStatisticsSource { + + public BufferedIOStatisticsOutputStream(final OutputStream out) { + super(out); + } + + public BufferedIOStatisticsOutputStream(final OutputStream out, + final int size) { + super(out, size); + } + + @Override + public IOStatistics getIOStatistics() { + return retrieveIOStatistics(out); + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/CounterIOStatistics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/CounterIOStatistics.java new file mode 100644 index 0000000000000..42b438e7459bb --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/CounterIOStatistics.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics.impl; + +import org.apache.hadoop.fs.statistics.IOStatistics; + +/** + * Interface an IOStatistics source where all the counters + * are implemented as a static set of counters. + *

+ * Thread safe. + */ +public interface CounterIOStatistics extends IOStatistics { + + long increment(String key, long value); + + /** + * Increment the counter by one. + * No-op if the counter is unknown. + * @param key statistics key + * @return old value or 0 + */ + default long increment(String key) { + return increment(key, 1); + } + + void set(String key, long value); + + /** + * Reset all counters. + * Unsynchronized. + */ + void resetCounters(); + + /** + * Update the counter values from a statistics source. + * The source must have all keys in this instance; + * extra keys are ignored. + * @param source source of statistics. + */ + void copy(IOStatistics source); + + /** + * Add the counter values from a statistics source. + *

+ * All entries must be counters. + *

+ * The source must have all keys in this instance; + * extra keys are ignored. + * @param source source of statistics. + */ + void add(IOStatistics source); + + /** + * Subtract the counter values from a statistics source. + *

+ * All entries must be counters. + *

+ * The source must have all keys in this instance; + * extra keys are ignored. + * @param source source of statistics. + */ + void subtract(IOStatistics source); +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/CounterIOStatisticsImpl.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/CounterIOStatisticsImpl.java new file mode 100644 index 0000000000000..8c552ed2e7a51 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/CounterIOStatisticsImpl.java @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics.impl; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; + + +import com.google.common.base.Preconditions; + +import org.apache.hadoop.fs.statistics.IOStatisticEntry; +import org.apache.hadoop.fs.statistics.IOStatistics; + +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.dynamicIOStatistics; + +/** + * Implement counter statistics as a map of AtomicLong counters + * created in the constructor. + */ +final class CounterIOStatisticsImpl extends WrappedIOStatistics + implements CounterIOStatistics { + + private final Map counters = new HashMap<>(); + + /** + * Constructor. + * @param keys keys to use for the counter statistics. + */ + CounterIOStatisticsImpl(String[] keys) { + super(null); + DynamicIOStatisticsBuilder builder = dynamicIOStatistics(); + for (String key : keys) { + AtomicLong counter = new AtomicLong(); + counters.put(key, counter); + builder.withAtomicLong(key, counter); + } + setSource(builder.build()); + } + + @Override + public long increment(final String key, final long value) { + AtomicLong counter = counters.get(key); + if (counter != null) { + return counter.getAndAdd(value); + } else { + return 0; + } + } + + @Override + public void set(final String key, final long value) { + AtomicLong counter = counters.get(key); + if (counter != null) { + counter.set(value); + } + } + + /** + * Reset all counters. + * Unsynchronized. + */ + @Override + public void resetCounters() { + counters.values().forEach(a -> a.set(0)); + } + + @Override + public void copy(final IOStatistics source) { + counters.entrySet().forEach(e -> { + String key = e.getKey(); + IOStatisticEntry statisticValue = source.getStatistic(key); + Preconditions.checkState(statisticValue != null, + "No statistic %s in IOStatistic source %s", + key, source); + e.getValue().set(statisticValue.scalar( + IOStatisticEntry.IOSTATISTIC_COUNTER + )); + }); + } + + @Override + public void add(final IOStatistics source) { + counters.entrySet().forEach(e -> { + String key = e.getKey(); + IOStatisticEntry statisticValue = source.getStatistic(key); + Preconditions.checkState(statisticValue != null, + "No statistic %s in IOStatistic source %s", + key, source); + long v = statisticValue.scalar( + IOStatisticEntry.IOSTATISTIC_COUNTER); + e.getValue().addAndGet(v); + }); + } + + @Override + public void subtract(final IOStatistics source) { + counters.entrySet().forEach(e -> { + String key = e.getKey(); + IOStatisticEntry statisticValue = source.getStatistic(key); + Preconditions.checkState(statisticValue != null, + "No statistic %s in IOStatistic source %s", + key, source); + long v = statisticValue.scalar( + IOStatisticEntry.IOSTATISTIC_COUNTER); + e.getValue().addAndGet(-v); + }); + } +} 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..3dd4dfe63340d --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/DynamicIOStatistics.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics.impl; + +import java.util.Iterator; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.function.Function; +import java.util.function.ToLongFunction; + +import org.apache.hadoop.fs.statistics.IOStatisticEntry; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsLogging; + +import static org.apache.hadoop.fs.statistics.IOStatisticEntry.statsEntry; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.snapshotStatistics; + +/** + * These statistics are dynamically evaluated by the supplied + * String -> Long 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 implements IOStatistics { + + /** + * Use a concurrent hash map for the ability to add across + * threads. + */ + private final Map> evaluators + = new TreeMap<>(); + + DynamicIOStatistics() { + } + + /** + * add a mapping of a key to a long function. + * @param key the key + * @param eval the evaluator + */ + void addLongFunction(String key, ToLongFunction eval) { + addFunction(key, k -> + statsEntry(IOStatisticEntry.IOSTATISTIC_COUNTER, + eval.applyAsLong(k))); + } + + /** + * add a mapping of a key to a long function. + * @param key the key + * @param eval the evaluator + */ + void addFunction(String key, Function eval) { + evaluators.put(key, eval); + } + + /** + * Get the value of a key. + * If the key is present, this will (re)evaluate it + * @param key key to look for. + * @return the latest value of that statistic, if found, else null. + */ + @Override + public IOStatisticEntry getStatistic(final String key) { + Function fn = evaluators.get(key); + return fn != null + ? fn.apply(key) + : null; + } + + @Override + public boolean isTracked(final String key) { + return evaluators.containsKey(key); + } + + /** + * Takes a snapshot and then provide an iterator around this. + * @return the iterator. + */ + @Override + public Iterator> iterator() { + return snapshotStatistics(this).iterator(); + } + + @Override + public Set keys() { + return evaluators.keySet(); + } + + @Override + public String toString() { + return IOStatisticsLogging.iostatisticsToString(this); + } +} 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..29181f4ae9677 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/DynamicIOStatisticsBuilder.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.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.IOStatisticEntry; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.metrics2.lib.MutableCounterLong; + +import static com.google.common.base.Preconditions.checkState; + +/** + * Builder of Dynamic IO Statistics which serve up up longs. + * 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(); + + /** + * Add a new evaluator to the statistics being built up. + * @param key key of this statistic + * @param eval evaluator for the statistic + * @return the builder. + */ + public DynamicIOStatisticsBuilder withFunction(String key, + Function eval) { + activeInstance().addFunction(key, eval); + return this; + } + + /** + * Add a new evaluator to the statistics being built up. + * @param key key of this statistic + * @param eval evaluator for the statistic + * @return the builder. + */ + public DynamicIOStatisticsBuilder withLongFunction(String key, + ToLongFunction eval) { + activeInstance().addLongFunction(key, eval); + return this; + } + + /** + * Add a 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 withAtomicLong(String key, + AtomicLong source) { + withLongFunction(key, s -> source.get()); + return this; + } + + /** + * Add a 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 withAtomicInteger(String key, + AtomicInteger source) { + withLongFunction(key, s -> source.get()); + return this; + } + + /** + * Build a dynamic 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) { + withLongFunction(key, s -> source.value()); + return this; + } + + /** + * 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; + } +} 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..b86db53ee0a44 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/EmptyIOStatistics.java @@ -0,0 +1,89 @@ +/* + * 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.Iterator; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Set; + +import org.apache.hadoop.fs.statistics.IOStatisticEntry; +import org.apache.hadoop.fs.statistics.IOStatistics; + +/** + * An empty IO Statistics implementation for classes which always + * want to return a non-null set of statistics. + */ +final class EmptyIOStatistics implements IOStatistics { + + /** + * The sole instance of this class. + */ + private static final EmptyIOStatistics INSTANCE = new EmptyIOStatistics(); + + private EmptyIOStatistics() { + } + + @Override + public IOStatisticEntry getStatistic(final String key) { + return null; + } + + @Override + public boolean isTracked(final String key) { + return false; + } + + @Override + public Iterator> iterator() { + return new EmptyIterator(); + } + + @Override + public Set keys() { + return Collections.emptySet(); + } + + /** + * The empty iterator has no entries. + */ + private static class EmptyIterator implements + Iterator> { + + @Override + public boolean hasNext() { + return false; + } + + @SuppressWarnings("NewExceptionWithoutArguments") + @Override + public Map.Entry next() { + throw new NoSuchElementException(); + } + } + + /** + * 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/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..4fc140aad424d --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java @@ -0,0 +1,95 @@ +/* + * 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 org.apache.hadoop.fs.StorageStatistics; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; + +/** + * Support for implementing IOStatistics interfaces. + */ +public final class IOStatisticsBinding { + + private IOStatisticsBinding() { + } + + /** + * Take a snapshot of the current statistics state. + * This is not an atomic option. + * The instance can be serialized. + * @param statistics statistics + * @return a snapshot of the current values. + */ + public static X snapshotStatistics( + IOStatistics statistics) { + return (X) new SnapshotIOStatistics(statistics); + } + + /** + * Create a builder for dynamic IO Statistics. + * @return a builder to be completed. + */ + public static DynamicIOStatisticsBuilder dynamicIOStatistics() { + return new DynamicIOStatisticsBuilder(); + } + + /** + * 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) { + return new IOStatisticsFromStorageStatistics(storageStatistics); + } + + /** + * 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 an IOStatistics instance from a varargs list of counter keys. + * This is the simplest way to build an IOStatistics instance as all + * the details are handled internally. + * + * @param keys key to use for the counter statistics. + * @return a new instance. + */ + public static CounterIOStatistics counterIOStatistics(String...keys) { + return new CounterIOStatisticsImpl(keys); + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsFromStorageStatistics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsFromStorageStatistics.java new file mode 100644 index 0000000000000..bcd943a6ca0e8 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsFromStorageStatistics.java @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics.impl; + +import java.util.Iterator; +import java.util.Map; +import java.util.Set; +import java.util.TreeSet; + +import com.google.common.base.Preconditions; + +import org.apache.hadoop.fs.StorageStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticEntry; +import org.apache.hadoop.fs.statistics.IOStatistics; + +import static org.apache.hadoop.fs.statistics.IOStatisticEntry.statsEntry; + +/** + * This provides an IOStatistics instance from a {@link StorageStatistics} + * instance. + */ +final class IOStatisticsFromStorageStatistics + implements IOStatistics { + + /** + * Source. + */ + private final StorageStatistics storageStatistics; + + /** + * Keys, calculated in the constructor. + */ + private final Set keys; + + /** + * Instantiate. This will iterate through the + * statistics to enumerate the keys. + * @param storageStatistics source + */ + IOStatisticsFromStorageStatistics( + final StorageStatistics storageStatistics) { + Preconditions.checkArgument(storageStatistics != null, + "Null storage statistics"); + this.storageStatistics = storageStatistics; + // build the keys. + keys = new TreeSet<>(); + final Iterator st + = storageStatistics.getLongStatistics(); + while (st.hasNext()) { + keys.add(st.next().getName()); + } + } + + @Override + public IOStatisticEntry getStatistic(final String key) { + return statsEntry(IOStatisticEntry.IOSTATISTIC_COUNTER, + storageStatistics.getLong(key)); + } + + @Override + public boolean isTracked(final String key) { + return storageStatistics.isTracked(key); + } + + @Override + public Iterator> iterator() { + return new MapEntryIterator(storageStatistics.getLongStatistics()); + } + + @Override + public Set keys() { + return keys; + } + + /** + * An iterator which takes a long statistic iterator from StorageStatistics + * and converts to an IOStatistics-compatible type. + */ + private static final class MapEntryIterator + implements Iterator> { + + /** + * The iterator over the storage statistics. + */ + private final Iterator iterator; + + private MapEntryIterator( + final Iterator iterator) { + this.iterator = iterator; + } + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public Map.Entry next() { + final StorageStatistics.LongStatistic entry = iterator.next(); + return StatsMapEntry.counter(entry.getName(), entry.getValue()); + } + + } + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsImplementationUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsImplementationUtils.java new file mode 100644 index 0000000000000..9e3d38108e6ab --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsImplementationUtils.java @@ -0,0 +1,136 @@ +/* + * 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.annotations.VisibleForTesting; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.statistics.IOStatisticEntry; + +import static org.apache.hadoop.fs.statistics.IOStatisticEntry.IOSTATISTIC_COUNTER; +import static org.apache.hadoop.fs.statistics.IOStatisticEntry.IOSTATISTIC_MAX; +import static org.apache.hadoop.fs.statistics.IOStatisticEntry.IOSTATISTIC_MEAN; +import static org.apache.hadoop.fs.statistics.IOStatisticEntry.IOSTATISTIC_MIN; +import static org.apache.hadoop.fs.statistics.IOStatisticEntry.statsEntry; + +/** + * Utility operations for implementing the classes within this package. + * + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public final class IOStatisticsImplementationUtils { + + /** 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 = "()"; + + /** + * Convert an entry to the string format used in logging. + * + * @param entry entry to evaluate + * @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 IOStatisticEntry value) { + return String.format( + ENTRY_PATTERN, + name, + value); + } + + public static IOStatisticEntry add(IOStatisticEntry left, + IOStatisticEntry right) { + left.requireCompatible(right); + left.requireTypeAndArity(IOSTATISTIC_COUNTER, 1); + return statsEntry(IOSTATISTIC_COUNTER, + left._1() + right._1()); + } + + public static IOStatisticEntry max(IOStatisticEntry left, + IOStatisticEntry right) { + left.requireCompatible(right); + left.requireTypeAndArity(IOSTATISTIC_MAX, 1); + return statsEntry(IOSTATISTIC_MAX, + Math.max(left._1(), right._1())); + } + + public static IOStatisticEntry min(IOStatisticEntry left, + IOStatisticEntry right) { + left.requireCompatible(right); + left.requireTypeAndArity(IOSTATISTIC_MIN, 1); + return statsEntry(IOSTATISTIC_MIN, + Math.min(left._1(), right._1())); + } + + public static IOStatisticEntry arithmeticMean( + IOStatisticEntry left, + IOStatisticEntry right) { + left.requireCompatible(right); + left.requireTypeAndArity(IOSTATISTIC_MEAN, 2); + long lSamples = left._2(); + long lSum = left._1() * lSamples; + long rSamples = right._2(); + double rSum = right._1() * rSamples; + long totalSamples = lSamples + rSamples; + return statsEntry(IOSTATISTIC_MEAN, + Math.round((lSum + rSum) / totalSamples), + totalSamples); + } + + public static IOStatisticEntry aggregate( + IOStatisticEntry left, + IOStatisticEntry right) { + left.requireCompatible(right); + switch (left.type()) { + case IOSTATISTIC_COUNTER: + return add(left, right); + case IOSTATISTIC_MIN: + return min(left, right); + case IOSTATISTIC_MAX: + return max(left, right); + case IOSTATISTIC_MEAN: + return arithmeticMean(left, right); + default: + // unknown value. + // rather than fail, just return the left value. + return left; + } + + } + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/SnapshotIOStatistics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/SnapshotIOStatistics.java new file mode 100644 index 0000000000000..b3dbcc9c05bfd --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/SnapshotIOStatistics.java @@ -0,0 +1,99 @@ +/* + * 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.Set; +import java.util.TreeMap; + +import org.apache.hadoop.fs.statistics.IOStatisticEntry; +import org.apache.hadoop.fs.statistics.IOStatistics; + +/** + * 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. + */ +class SnapshotIOStatistics implements IOStatistics, Serializable { + + + private static final long serialVersionUID = -1762522703841538084L; + + /** + * Treemaps sort their insertions so the iterator is ordered. + * They are also serializable. + */ + private final TreeMap entries + = new TreeMap<>(); + + /** + * Construct from a source statistics instance. + * @param source source stats. + */ + SnapshotIOStatistics(final IOStatistics source) { + snapshot(source); + } + + /** + * Empty constructor for deserialization. + */ + SnapshotIOStatistics() { + } + + @Override + public IOStatisticEntry getStatistic(final String key) { + return entries.get(key); + } + + @Override + public boolean isTracked(final String key) { + return false; + } + + @Override + public Iterator> iterator() { + return entries.entrySet().iterator(); + } + + @Override + public Set keys() { + return entries.keySet(); + } + + /** + * Take a snapshot. + * @param source statistics source. + */ + private void snapshot(IOStatistics source) { + entries.clear(); + // MUST NOT use iterator() because IOStatistics implementations + // may create a snapshot when iterator() is invoked; + // enumerating keys and querying values avoids stack + // overflows + for (String key : source.keys()) { + entries.put(key, source.getStatistic(key)); + } + } + + +} 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/StatsMapEntry.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StatsMapEntry.java new file mode 100644 index 0000000000000..b767f590e579b --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StatsMapEntry.java @@ -0,0 +1,82 @@ +/* + * 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.IOStatisticEntry; + +import static org.apache.hadoop.fs.statistics.IOStatisticEntry.statsEntry; + +/** + * A map entry for implementations to use if they need to. + */ +public final class StatsMapEntry + implements Map.Entry { + + /** + * Key. + */ + private final String key; + + /** + * Value. + */ + private IOStatisticEntry value; + + /** + * Constructor. + * @param key key + * @param value value + */ + StatsMapEntry(final String key, final IOStatisticEntry value) { + this.key = key; + this.value = value; + } + + public String getKey() { + return key; + } + + public IOStatisticEntry getValue() { + return value; + } + + @Override + public IOStatisticEntry setValue(final IOStatisticEntry val) { + this.value = val; + return val; + } + + @Override + public String toString() { + return String.format("(%s, %s)", key, value); + } + + + /** + * Counter value. + * @param key key + * @param value value + */ + public static StatsMapEntry counter(final String key, final long value) { + return new StatsMapEntry(key, + statsEntry(IOStatisticEntry.IOSTATISTIC_COUNTER, value)); + } +} 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..394783acb5daf --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/WrappedIOStatistics.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics.impl; + +import java.util.Iterator; +import java.util.Map; +import java.util.Set; + +import org.apache.hadoop.fs.statistics.IOStatisticEntry; +import org.apache.hadoop.fs.statistics.IOStatistics; + +/** + * Wrap IOStatistics source with another (dynamic) wrapper. + */ +public class WrappedIOStatistics implements IOStatistics { + + private IOStatistics source; + + public WrappedIOStatistics(final IOStatistics source) { + this.source = source; + } + + + protected IOStatistics getSource() { + return source; + } + + protected void setSource(final IOStatistics source) { + this.source = source; + } + + @Override + public IOStatisticEntry getStatistic(final String key) { + return source.getStatistic(key); + } + + @Override + public boolean isTracked(final String key) { + return source.isTracked(key); + } + + @Override + public Set keys() { + return source.keys(); + } + + @Override + public Iterator> iterator() { + return source.iterator(); + } +} 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..eb9e1e26a60f7 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/package-info.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. + */ + +/** + * 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..55ac9be89626b --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/package-info.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. + */ + +/** + * 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) for implementation of + * {@code IOStatisticsSource}, and, if it is, calls {@code getIOStatistics()} + * to get its statistics. + * If this is non-null, the client has statistics on the current + * state of the statistics. + *

+ * The expectation is that a statistics source is dynamic: when a value is + * looked up the most recent values are returned. + * When iterating through the set, the values of the iterator SHOULD + * be frozen at the time the iterator was requested. + *

+ * These statistics can be used to: log operations, profile applications, + * and make assertions about the state of the output. + *

+ * The names of statistics are a matter of choice of the specific source. + * However, {@link org.apache.hadoop.fs.statistics.StoreStatisticNames} + * contains a + * set of names recommended for object store operations. + * {@link org.apache.hadoop.fs.statistics.StreamStatisticNames} declares + * recommended names for statistics provided forInconsistentS3ClientFactoryInconsistentS3ClientFactory + * input and output streams. + *

+ * Utility classes are includes to assist use + *

    + *
  • + * {@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. + *
  • + *
+ * + *

+ * 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/site/markdown/filesystem/index.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md index df538ee6cf96b..25cc9d13d1fca 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md @@ -38,3 +38,4 @@ HDFS as these are commonly expected by Hadoop client applications. 2. [Testing with the Filesystem specification](testing.html) 2. [Extending the specification and its tests](extending.html) 1. [Uploading a file using Multiple Parts](multipartuploader.html) +1. [IOStatistics](iostatistics.html) diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/iostatistics.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/iostatistics.md new file mode 100644 index 0000000000000..7f917622300e9 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/iostatistics.md @@ -0,0 +1,272 @@ + + +# Introduction to the IOStatistics API + +```java +@InterfaceAudience.Public +@InterfaceStability.Unstable +``` + +The `IOStatistics` API is intended to provide statistics on individual IO +classes -such as input and output streams, *in a standard way which +applications can query* + +Many filesystem-related classes have implemented statistics gathering +and provided private/unstable ways to query this, but as they were +not common across implementations it was unsafe for applications +to reference these values. Example: `S3AInputStream` and its statistics +API. This is used in internal tests, but cannot be used downstream in +applications such as + +The new IOStatistics API is intended to + +1. Be instance specific:, rather than shared across multiple instances + of a class, or thread local. +1. Be public and stable enough to be used by applications. +1. Be easy to use in applications written in Java, Scala, and, via libhdfs, C/C++ +1. Have foundational interfaces and classes in the `hadoop-common` JAR. + +## Core model + +Any Hadoop I/O class *may* implement `IOStatisticsSource` in order to +provide statistics. + +Wrapper I/O Classes (e.g `FSDataInputStream`, `FSDataOutputStream` *should* +implement the interface and forward it to the wrapped class, if they also +implement it -or return `null` if they do not. + +`IOStatisticsSource` implementations `getIOStatistics()` return an +instance of `IOStatistics` enumerating the statistics of that specific +instance. + +The `IOStatistics` implementations provide + +* A way to enumerate all keys/statistics monitored. +* An iterator over all such keys and their latest values. +* A way to explitly request the value of specific statistic. + +A single statistic is reprsented by an `IOStatisticEntry` instance. +This supports multiple types of statistics, each as a type (integer) +and an array of values -which are viewed as a _tuple_ whose _arity_ +is the length of the array. + + + +## package `org.apache.hadoop.fs.statistics` + +This package contains the public statistics APIs intended +for use by applications. + + + + + + +### class `org.apache.hadoop.fs.statistics.IOStatisticEntry` + +```java +@InterfaceAudience.Public +@InterfaceStability.Unstable +public final class IOStatisticEntry implements Serializable { +... +} +``` + +This represents the evaluated value of a statistic. +It is more than just a simple scala value + +1. Serializable. +1. Extensible: new statistic types MAY be added. +1. non-scalar: a statistic may require more than one long value to represent it. +1. Aggregatable: two `IOStatisticEntry` instances of the same type + can be aggregated. For some types (e.g the mean), this requires more + than one element of data to +1. Forwards compatible to the extent that adding a new statistics type + MUST NOT break any existing applications, even if any new and unknown + types cannot be aggregated. + +| Type ID | Name | Data | Function | +|---------|------|------|----------| +| 0 | counter | `(count)` | a counter which can be aggregated through addition | +| 1 | min | `(value)` | a minimum recorded value | +| 2 | max | `(value)` | a maximum recorded value | +| 3 | mean | `(mean, sample_count)` | an arithmentic mean and the number of samples used | + +Please consult the javadocs for its full set of methods. + + + + + + +### class `org.apache.hadoop.fs.statistics.IOStatisticsSource` + +```java + +/** + * A source of IO statistics. + * These statistics MUST be instance specific, not thread local. + */ +@InterfaceStability.Unstable +public interface IOStatisticsSource { + + /** + * Return a statistics instance. + * It is not a requirement that the same instance is returned every time. + * {@link IOStatisticsSource}. + * If the object implementing this is Closeable, this method + * may return null if invoked on a closed object, even if + * it returns a valid instance when called earlier. + * @return an IOStatistics instance or null + */ + IOStatistics getIOStatistics(); +} +``` + +This is the interface which an object instance MUST implement if they are a source of +IOStatistics information. + + + + + +### class `org.apache.hadoop.fs.statistics.IOStatistics` + +These are per-instance statistics provided by an object which +implements `IOStatisticsSource`. + +```java +@InterfaceAudience.Public +@InterfaceStability.Unstable +public interface IOStatistics + extends Iterable> { + + /** + * Get the entry of a statistic. + * + * @return The entry of the statistic, or null if not tracked. + */ + IOStatisticEntry getStatistic(String key); + + /** + * Return true if a statistic is being tracked. + * + * @return True only if the statistic is being tracked. + */ + boolean isTracked(String key); + + /** + * Get the set of keys. + * No guarantees are made about the mutability/immutability + * of this set. + * @return the set of keys. + */ + Set keys(); + +} +``` + + +The statistics MUST BE for the specific instance of the source; +possibly including aggregate statistics from other objects +created by that stores. +For example, the statistics of a filesystem instance must be unique +to that instant and not shared with any other. +However, those statistics may also collect and aggregate statistics +generated in the use of input and output streams created by that +file system instance. + +The iterator is a possibly empty iterator over all monitored statistics. + +* The set of statistic keys SHOULD remain unchanged, and MUST NOT +remove keys. + +* The statistics SHOULD be dynamic: every call to `iterator()` + MAY return a current/recent set of statistics. + +* The values MAY change across invocations of `iterator()`. + +* The update MAY be in the `iterable()` call, or MAY be in the actual + `iterable.next()` operation. + +* The returned Map.Entry instances MUST return the same value on + repeated `getValue()` calls. + +* Queries of statistics SHOULD Be fast and Nonblocking 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). + + +### Thread Model + +1. An instance of IOStatistics can be shared across threads; + a call to `iterator()` is thread safe. + +1. The actual `Iterable` returned 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. + + +## 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 should be raised via the Hadoop development +channels. + +These MAY be used by those implementations of the Hadoop `FileSystem`, `AbstractFileSystem` +and related classes which are not in the hadoop source tree. Implementors MUST BR +that all 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/AbstractContractStreamIOStatisticsTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractStreamIOStatisticsTest.java new file mode 100644 index 0000000000000..acee128dbe286 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractStreamIOStatisticsTest.java @@ -0,0 +1,279 @@ +/* + * 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 static org.apache.hadoop.fs.contract.ContractTestUtils.dataset; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.extractStatistics; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.demandStringify; +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.keys()) + .describedAs("statistic keys of %s", statistics) + .containsAll(keys); + Assertions.assertThat(keys) + .describedAs("Statistics supported by the stream %s", out) + .contains(STREAM_WRITE_BYTES); + } finally { + fs.delete(path, false); + } + } + + /** + * If the stream writes in blocks, then counters during the write may be + * zero until a whole block is written -or the write has finished. + * @return true if writes are buffered into whole blocks. + */ + public boolean streamWritesInBlocks() { + return false; + } + + @Test + public void testWriteSingleByte() throws Throwable { + describe("Write a byte to a file and verify" + + " the stream statistics are updated"); + Path path = methodPath(); + FileSystem fs = getFileSystem(); + fs.mkdirs(path.getParent()); + boolean writesInBlocks = streamWritesInBlocks(); + try (FSDataOutputStream out = fs.create(path, true)) { + IOStatistics statistics = extractStatistics(out); + // before a write, no bytes + verifyStatisticCounterValue(statistics, STREAM_WRITE_BYTES, 0); + out.write('0'); + verifyStatisticCounterValue(statistics, STREAM_WRITE_BYTES, + writesInBlocks ? 0 : 1); + // close the stream + out.close(); + // statistics are still valid after the close + // always call the output stream to check that behavior + statistics = extractStatistics(out); + final String strVal = statistics.toString(); + LOG.info("Statistics = {}", strVal); + verifyStatisticCounterValue(statistics, STREAM_WRITE_BYTES, 1); + } finally { + fs.delete(path, false); + } + } + + @Test + public void testWriteByteArrays() throws Throwable { + describe("Write byte arrays to a file and verify" + + " the stream statistics are updated"); + Path path = methodPath(); + FileSystem fs = getFileSystem(); + fs.mkdirs(path.getParent()); + boolean writesInBlocks = streamWritesInBlocks(); + try (FSDataOutputStream out = fs.create(path, true)) { + Object demandStatsString = demandStringify(out); + // before a write, no bytes + final byte[] bytes = ContractTestUtils.toAsciiByteArray( + "statistically-speaking"); + final long len = bytes.length; + out.write(bytes); + out.flush(); + IOStatistics statistics = extractStatistics(out); + verifyStatisticCounterValue(statistics, STREAM_WRITE_BYTES, + writesInBlocks ? 0 : len); + out.write(bytes); + out.flush(); + verifyStatisticCounterValue(statistics, STREAM_WRITE_BYTES, + writesInBlocks ? 0 : len * 2); + // close the stream + out.close(); + // statistics are still valid after the close + // always call the output stream to check that behavior + statistics = extractStatistics(out); + verifyStatisticCounterValue(statistics, STREAM_WRITE_BYTES, len * 2); + // the to string value must contain the same counterHiCable you mean + Assertions.assertThat(demandStatsString.toString()) + .contains(Long.toString(len * 2)); + } finally { + fs.delete(path, false); + } + } + + @Test + public void testInputStreamStatisticKeys() throws Throwable { + describe("Look at the statistic keys of an input stream"); + Path path = methodPath(); + FileSystem fs = getFileSystem(); + ContractTestUtils.touch(fs, path); + try (FSDataInputStream in = fs.open(path)) { + IOStatistics statistics = extractStatistics(in); + final List keys = inputStreamStatisticKeys(); + Assertions.assertThat(statistics.keys()) + .describedAs("statistic keys of %s", statistics) + .containsAll(keys); + Assertions.assertThat(keys) + .describedAs("Statistics supported by the stream %s", in) + .contains(STREAM_READ_BYTES); + verifyStatisticCounterValue(statistics, STREAM_READ_BYTES, 0); + } finally { + fs.delete(path, false); + } + } + + @Test + public void testInputStreamStatisticRead() throws Throwable { + describe("Read Data from an input stream"); + Path path = methodPath(); + FileSystem fs = getFileSystem(); + final int fileLen = 1024; + final byte[] ds = dataset(fileLen, 'a', 26); + ContractTestUtils.writeDataset(fs, path, ds, fileLen, 8_000, true); + + try (FSDataInputStream in = fs.open(path)) { + long current = 0; + IOStatistics statistics = extractStatistics(in); + verifyStatisticCounterValue(statistics, STREAM_READ_BYTES, 0); + Assertions.assertThat(in.read()).isEqualTo('a'); + int bufferSize = readBufferSize(); + // either a single byte was read or a whole block + current = verifyBytesRead(statistics, current, 1, bufferSize); + final int bufferLen = 128; + byte[] buf128 = new byte[bufferLen]; + in.read(buf128); + current = verifyBytesRead(statistics, current, bufferLen, bufferSize); + in.readFully(buf128); + current = verifyBytesRead(statistics, current, bufferLen, bufferSize); + in.readFully(0, buf128); + current = verifyBytesRead(statistics, current, bufferLen, bufferSize); + // seek must not increment the read counter + in.seek(256); + verifyBytesRead(statistics, current, 0, bufferSize); + + // if a stream implements lazy-seek the seek operation + // may be postponed until the read + final int sublen = 32; + Assertions.assertThat(in.read(buf128, 0, sublen)) + .isEqualTo(sublen); + current = verifyBytesRead(statistics, current, sublen, bufferSize); + + // perform some read operations near the end of the file such that + // the buffer will not be completely read. + // skip these tests for buffered IO as it is too complex to work out + if (bufferSize == 0) { + final int pos = fileLen - sublen; + in.seek(pos); + Assertions.assertThat(in.read(buf128)) + .describedAs("Read overlapping EOF") + .isEqualTo(sublen); + current = verifyStatisticCounterValue(statistics, STREAM_READ_BYTES, + current + sublen); + Assertions.assertThat(in.read(pos, buf128, 0, bufferLen)) + .describedAs("Read(buffer) overlapping EOF") + .isEqualTo(sublen); + verifyStatisticCounterValue(statistics, STREAM_READ_BYTES, + current + sublen); + } + } finally { + fs.delete(path, false); + } + } + + /** + * Verify the bytes read value, taking into account block size. + * @param statistics stats + * @param current current count + * @param bytesRead bytes explicitly read + * @param bufferSize buffer size of stream + * @return the current count of bytes read ignoring block size + */ + public long verifyBytesRead(final IOStatistics statistics, + final long current, + final int bytesRead, final int bufferSize) { + // final position. for unbuffered read, this is the expected value + long finalPos = current + bytesRead; + long expected = finalPos; + if (bufferSize > 0) { + // buffered. count of read is number of buffers already read + // plus the current buffer, multiplied by that buffer size + expected = bufferSize * (1 + (current / bufferSize)); + } + verifyStatisticCounterValue(statistics, STREAM_READ_BYTES, expected); + return finalPos; + } + + /** + * Buffer size for reads. + * Filesystems performing block reads (checksum, etc) + * must return their buffer value is + * @return buffer capacity; 0 for unbuffered + */ + public int readBufferSize() { + return 0; + } + + /** + * Keys which the output stream must support. + * @return a list of keys + */ + public List outputStreamStatisticKeys() { + return Collections.singletonList(STREAM_WRITE_BYTES); + } + + /** + * Keys which the input stream must support. + * @return a list of keys + */ + public List inputStreamStatisticKeys() { + return Collections.singletonList(STREAM_READ_BYTES); + } + +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractStreamIOStatistics.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractStreamIOStatistics.java new file mode 100644 index 0000000000000..642baec502d2e --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractStreamIOStatistics.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.contract.localfs; + +import java.util.Arrays; +import java.util.List; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractStreamIOStatisticsTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; + +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BYTES; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_EXCEPTIONS; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_SKIP_BYTES; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_SKIP_OPERATIONS; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_WRITE_BYTES; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_WRITE_EXCEPTIONS; + +/** + * Test IOStatistics through the local FS. + */ +public class TestLocalFSContractStreamIOStatistics extends + AbstractContractStreamIOStatisticsTest { + + @Override + protected AbstractFSContract createContract(Configuration conf) { + return new LocalFSContract(conf); + } + + /** + * Keys which the input stream must support. + * @return a list of keys + */ + public List inputStreamStatisticKeys() { + return Arrays.asList(STREAM_READ_BYTES, + STREAM_READ_EXCEPTIONS, + STREAM_READ_SEEK_OPERATIONS, + STREAM_READ_SKIP_OPERATIONS, + STREAM_READ_SKIP_BYTES); + } + + /** + * Keys which the output stream must support. + * @return a list of keys + */ + @Override + public List outputStreamStatisticKeys() { + return Arrays.asList(STREAM_WRITE_BYTES, + STREAM_WRITE_EXCEPTIONS); + } + + @Override + public int readBufferSize() { + return 1024; + } + + @Override + public boolean streamWritesInBlocks() { + return true; + } + + +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/IOStatisticAssertions.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/IOStatisticAssertions.java new file mode 100644 index 0000000000000..cd7270b60c990 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/IOStatisticAssertions.java @@ -0,0 +1,176 @@ +/* + * 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 org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +import static org.apache.hadoop.fs.statistics.IOStatisticEntry.IOSTATISTIC_COUNTER; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Assertions and any other support for IOStatistics testing. + * If used downstream: know it is unstable. + * There's some oddness here related to AssertJ's handling of iterables; + * we need to explicitly cast it to call methods on the interface + * other than iterator(). + */ + +@InterfaceAudience.Private +@InterfaceStability.Unstable +public final class IOStatisticAssertions { + + private IOStatisticAssertions() { + } + + /** + * Get a required counter statistic + * @param stats statistics source + * @param key statistic key + * @return the value + */ + public static long extractCounterStatistic( + final IOStatistics stats, + final String key) { + final IOStatisticEntry statistic = stats.getStatistic(key); + assertThat(statistic) + .describedAs("Statistics %s and key %s", stats, + key) + .isNotNull(); + return statistic.scalar(IOSTATISTIC_COUNTER); + } + + /** + * Assert that a given statistic has an expected value. + * @param stats statistics source + * @param key statistic key + * @param value expected value. + * @return the value (which always equals the expected value) + */ + public static long verifyStatisticCounterValue( + final IOStatistics stats, + final String key, + final long value) { + final IOStatisticEntry statistic = stats.getStatistic(key); + assertThat(statistic) + .describedAs("Statistics %s and key %s with expected value %s", stats, + key, value) + .isNotNull() + .extracting(f -> f.scalar(IOSTATISTIC_COUNTER)) + .isEqualTo(value); + return statistic.scalar(IOSTATISTIC_COUNTER); + } + + /** + * Assert that a given statistic is unknown. + * @param stats statistics source + * @param key statistic key + */ + public static void assertStatisticIsUnknown( + final IOStatistics stats, + final String key) { + assertThat(stats.getStatistic(key)) + .describedAs("Statistics %s and key %s", stats, + key) + .isNull(); + } + + /** + * Assert that a given statistic is tracked. + * @param stats statistics source + * @param key statistic key + */ + public static void assertStatisticIsTracked( + final IOStatistics stats, + final String key) { + assertThat(stats.isTracked(key)) + .describedAs("Statistic %s is not tracked in %s", key, stats) + .isTrue(); + } + + /** + * Assert that a given statistic is untracked. + * @param stats statistics source + * @param key statistic key + */ + public static void assertStatisticIsUntracked( + final IOStatistics stats, + final String key) { + assertThat(stats.isTracked(key)) + .describedAs("Statistic %s is tracked in %s", key, stats) + .isFalse(); + } + + /** + * 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. + * @param source source object. + * @return the statistics it provides. + */ + public static IOStatistics extractStatistics(Object source) { + assertThat(source) + .describedAs("Object %s", source) + .isInstanceOf(IOStatisticsSource.class); + IOStatistics statistics = ((IOStatisticsSource) source).getIOStatistics(); + assertThat(statistics) + .describedAs("Statistics from %s", source) + .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..e130143122c91 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestDynamicIOStatistics.java @@ -0,0 +1,314 @@ +/* + * 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.assertStatisticIsTracked; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertStatisticIsUnknown; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertStatisticIsUntracked; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsImplementationUtils.NULL_SOURCE; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.demandStringify; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.iostatisticsToString; +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() + .withAtomicLong(ALONG, aLong) + .withAtomicInteger(AINT, aInt) + .withMutableCounter(COUNT, counter) + .withLongFunction(EVAL, x -> evalLong) + .build(); + statsSource = new SourceWrappedStatistics(statistics); + } + + /** + * The eval operation is foundational. + */ + @Test + public void testEval() throws Throwable { + verifyStatisticCounterValue(statistics, EVAL, 0); + evalLong = 10; + verifyStatisticCounterValue(statistics, EVAL, 10); + } + + /** + * Atomic Long statistic. + */ + @Test + public void testAlong() throws Throwable { + verifyStatisticCounterValue(statistics, ALONG, 0); + aLong.addAndGet(1); + verifyStatisticCounterValue(statistics, ALONG, 1); + } + + /** + * Atomic Int statistic. + */ + @Test + public void testAint() throws Throwable { + verifyStatisticCounterValue(statistics, AINT, 0); + aInt.addAndGet(1); + verifyStatisticCounterValue(statistics, AINT, 1); + } + + /** + * Metrics2 counter. + */ + @Test + public void testCounter() throws Throwable { + verifyStatisticCounterValue(statistics, COUNT, 0); + counter.incr(); + verifyStatisticCounterValue(statistics, COUNT, 1); + } + + /** + * keys() returns all the keys. + */ + @Test + public void testKeys() throws Throwable { + Assertions.assertThat(statistics.keys()) + .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) + .extracting(s -> s.getKey()) + .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.iterator(); + // reset 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().scalar(IOStatisticEntry.IOSTATISTIC_COUNTER)) + .describedAs("Value of entry %s", next) + .isEqualTo(1); + } + } + + @Test + public void testUnknownStatistic() throws Throwable { + assertStatisticIsUnknown(statistics, "anything"); + assertStatisticIsUntracked(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(() -> + assertStatisticIsTracked(statistics, "anything")) + .isInstanceOf(AssertionError.class); + } + + @Test + public void testStatisticsValueAssertion() throws Throwable { + // expect an exception to be raised when + // an assertion is made about the value of an unknown statistics + assertThatThrownBy(() -> + verifyStatisticCounterValue(statistics, "anything", 0)) + .isInstanceOf(AssertionError.class); + } + + /** + * Serialization round trip will preserve all the values. + */ + @Test + public void testSerDeser() throws Throwable { + incrementAllCounters(); + IOStatistics stat = IOStatisticsSupport.snapshot(statistics); + incrementAllCounters(); + IOStatistics deser = IOStatisticAssertions.roundTrip(stat); + assertThat(deser) + .extracting(s -> s.getKey()) + .containsExactlyInAnyOrder(KEYS); + for (Map.Entry e: deser) { + assertThat(e.getValue() + .scalar(IOStatisticEntry.IOSTATISTIC_COUNTER)) + .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 = "(along=(type=counter, (%d))"; + // this is not yet evaluated + Object demand = demandStringify(statistics); + // nor is this. + Object demandSource = demandStringify(statsSource); + + // show it evaluates + String formatted1 = String.format(counterPattern, 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.get()); + assertThat(demand + .toString()) + .doesNotContain(formatted1) + .contains(formatted2); + assertThat(demandSource + .toString()) + .doesNotContain(formatted1) + .contains(formatted2); + } + + @Test + public void testNullSourceStringification() throws Throwable { + assertThat(demandStringify((IOStatisticsSource)null) + .toString()) + .isEqualTo(NULL_SOURCE); + } + + @Test + public void testNullStatStringification() throws Throwable { + assertThat(demandStringify((IOStatistics)null) + .toString()) + .isEqualTo(NULL_SOURCE); + } + + @Test + public void testStringLogging() throws Throwable { + LOG.info("Output {}", demandStringify(statistics)); + } + + /** + * Increment all the counters from their current value. + */ + public void incrementAllCounters() { + aLong.incrementAndGet(); + aInt.incrementAndGet(); + evalLong += 1; + counter.incr(); + } + + /** + * Needed to provide a metrics info instance for the counter + * constructor. + */ + private 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..3ce6c0194680b --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestEmptyIOStatistics.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.statistics; + +import java.util.Iterator; +import java.util.Map; +import java.util.NoSuchElementException; + +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.assertStatisticIsTracked; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertStatisticIsUnknown; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertStatisticIsUntracked; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.iostatisticsToString; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.emptyStatistics; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; +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 testIterator() throws Throwable { + Iterator> iterator = empty.iterator(); + + assertThat(iterator.hasNext()) + .describedAs("iterator.hasNext()") + .isFalse(); + intercept(NoSuchElementException.class, iterator::next); + } + + @Test + public void testUnknownStatistic() throws Throwable { + assertStatisticIsUnknown(empty, "anything"); + assertStatisticIsUntracked(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(() -> + assertStatisticIsTracked(empty, "anything")) + .isInstanceOf(AssertionError.class); + } + + @Test + public void testStatisticsValueAssertion() throws Throwable { + // expect an exception to be raised when + // an assertion is made about the value of an unknown statistics + assertThatThrownBy(() -> + verifyStatisticCounterValue(empty, "anything", 0)) + .isInstanceOf(AssertionError.class); + } + + @Test + public void testEmptySnapshot() throws Throwable { + final IOStatistics stat = IOStatisticsSupport.snapshot(empty); + assertThat(stat.keys()) + .describedAs("keys of snapshot") + .isEmpty(); + IOStatistics deser = IOStatisticAssertions.roundTrip(stat); + assertThat(deser.keys()) + .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/impl/TestIOStatisticEntry.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/impl/TestIOStatisticEntry.java new file mode 100644 index 0000000000000..21101bc02f5d5 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/impl/TestIOStatisticEntry.java @@ -0,0 +1,143 @@ +/* + * 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.junit.Test; + +import org.apache.hadoop.fs.statistics.IOStatisticEntry; +import org.apache.hadoop.test.AbstractHadoopTestBase; + +import static org.apache.hadoop.fs.statistics.IOStatisticEntry.*; +import static org.apache.hadoop.fs.statistics.IOStatisticEntry.statsEntry; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsImplementationUtils.aggregate; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Tests for the IOStatisticEntry class and util methods in + * the statistics.impl package. + */ +public class TestIOStatisticEntry extends AbstractHadoopTestBase { + + /** + * This is a type we know is not in the list of known types, + * so lacks an aggregator or any rule about aggregation + */ + public static final int X = 10; + + private final IOStatisticEntry counter1 = statsEntry(IOSTATISTIC_COUNTER, 1); + private final IOStatisticEntry counter2 = statsEntry(IOSTATISTIC_COUNTER, 2); + private final IOStatisticEntry min1 = statsEntry(IOSTATISTIC_MIN, 1); + private final IOStatisticEntry min2 = statsEntry(IOSTATISTIC_MIN, 2); + private final IOStatisticEntry max1 = statsEntry(IOSTATISTIC_MAX, 1); + private final IOStatisticEntry max2 = statsEntry(IOSTATISTIC_MAX, 2); + private final IOStatisticEntry mean1 = statsEntry(IOSTATISTIC_MEAN, 1, 1); + private final IOStatisticEntry mean12 = statsEntry(IOSTATISTIC_MEAN, 1, 2); + private final IOStatisticEntry mean2 = statsEntry(IOSTATISTIC_MEAN, 2, 10); + + @Test + public void testCounterAdd() throws Throwable { + assertThat(aggregate(counter1, counter1)) + .isEqualTo(counter2); + } + + @Test + public void testMinAggregate() throws Throwable { + assertThat(aggregate(min1, min2)) + .isEqualTo(min1); + } + + @Test + public void testMaxAggregate() throws Throwable { + assertThat(aggregate(max1, max2)) + .isEqualTo(max2); + } + + @Test + public void testMeanAggregate1() throws Throwable { + assertThat(aggregate(mean1, mean1)) + .isEqualTo(mean12); + } + + @Test + public void testMeanAggregateRounding() throws Throwable { + assertThat(aggregate(mean1, mean2)) + .isEqualTo(statsEntry(IOSTATISTIC_MEAN, 2, 11)); + } + + /** + * Unknown types are aggregated just by taking + * the left value. + */ + @Test + public void testUnknownTypeAggregation() throws Throwable { + IOStatisticEntry e1 = statsEntry(X, 1); + IOStatisticEntry e2 = statsEntry(X, 2); + assertThat(aggregate(e1, e2)) + .isEqualTo(e1); + } + + @Test + public void testAccessors() throws Throwable { + IOStatisticEntry e1 = statsEntry(X, 1, 2, 3, 4); + assertThat(e1) + .extracting(IOStatisticEntry::_1) + .isEqualTo(1L); + assertThat(e1) + .extracting(IOStatisticEntry::_2) + .isEqualTo(2L); + assertThat(e1) + .extracting(IOStatisticEntry::_3) + .isEqualTo(3L); + assertThat(e1.arity()) + .isEqualTo(4); + assertThat(e1.typeAsString()) + .isEqualTo("10"); + long[] data = e1.getData(); + assertThat(data.length) + .isEqualTo(e1.arity()) + .isEqualTo(4); + } + + @Test + public void testStringAccessors() throws Throwable { + assertThat(counter1.typeAsString()) + .isEqualTo("counter"); + assertThat(min1.typeAsString()) + .isEqualTo("min"); + assertThat(max1.typeAsString()) + .isEqualTo("max"); + assertThat(mean1.typeAsString()) + .isEqualTo("mean"); + } + + @Test + public void testWrongArity() throws Throwable { + intercept(IllegalArgumentException.class, () -> + statsEntry(IOSTATISTIC_COUNTER, 1, 1)); + intercept(IllegalArgumentException.class, () -> + statsEntry(IOSTATISTIC_MEAN, 1)); + } + + @Test + public void testNoData() throws Throwable { + intercept(IllegalArgumentException.class, () -> + statsEntry(X)); + } +} 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/DefaultS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java index 276961bf8b7e1..bc412ce68f257 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,122 @@ public AmazonS3 createS3Client(URI name, if (!StringUtils.isEmpty(userAgentSuffix)) { awsConf.setUserAgentSuffix(userAgentSuffix); } - return configureAmazonS3Client( - newAmazonS3Client(credentials, awsConf), conf); + // optional metrics + RequestMetricCollector metrics = statisticsFromAwsSdk != null + ? new AwsStatisticsCollector(statisticsFromAwsSdk) + : null; + + return newAmazonS3Client( + credentials, + awsConf, + metrics, + conf.getTrimmed(ENDPOINT, ""), + conf.getBoolean(PATH_STYLE_ACCESS, false)); } /** - * Wrapper around constructor for {@link AmazonS3} client. + * Create an {@link AmazonS3} client. * Override this to provide an extended version of the client * @param credentials credentials to use * @param awsConf AWS configuration - * @return new AmazonS3 client + * @param metrics metrics collector or null + * @param endpoint endpoint string; may be "" + * @param pathStyleAccess enable path style access? + * @return new AmazonS3 client */ protected AmazonS3 newAmazonS3Client( - AWSCredentialsProvider credentials, ClientConfiguration awsConf) { - return new AmazonS3Client(credentials, awsConf); + final AWSCredentialsProvider credentials, + final ClientConfiguration awsConf, + final RequestMetricCollector metrics, + final String endpoint, + final boolean pathStyleAccess) { + if (metrics != null) { + LOG.debug("Building S3 client using the SDK builder API"); + return buildAmazonS3Client(credentials, awsConf, metrics, endpoint, + pathStyleAccess); + } else { + LOG.debug("Building S3 client using the SDK builder API"); + return classicAmazonS3Client(credentials, awsConf, endpoint, + pathStyleAccess); + } } /** - * Configure S3 client from the Hadoop configuration. - * + * Use the (newer) Builder SDK to create a an AWS S3 client. + *

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

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

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

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

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

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

+ * Note also that AWS signing is a mystery which nobody fully + * understands, especially given all problems surface in a + * "400 bad request" response, which, like all security systems, + * provides minimal diagnostics out of fear of leaking + * secrets. + * + * @param endpoint possibly null endpoint. + * @param awsConf config to build the URI from. + * @return a configuration for the S3 client builder. + */ + @VisibleForTesting + public static AwsClientBuilder.EndpointConfiguration + createEndpointConfiguration( + final String endpoint, final ClientConfiguration awsConf) { + LOG.debug("Creating endpoint configuration for {}", endpoint); + if (endpoint == null || endpoint.isEmpty()) { + // the default endpoint...we should be using null at this point. + LOG.debug("Using default endpoint -no need to generate a configuration"); + return null; + } + + final URI epr = RuntimeHttpUtils.toUri(endpoint, awsConf); + LOG.debug("Endpoint URI = {}", epr); + + String region; + if (!ServiceUtils.isS3USStandardEndpoint(endpoint)) { + LOG.debug("Endpoint {} is not the default; parsing", epr); + region = AwsHostNameUtils.parseRegion( + epr.getHost(), + S3_SERVICE_NAME); + } else { + // US-east, set region == null. + LOG.debug("Endpoint {} is the standard one; declare region as null", epr); + region = null; + } + LOG.debug("Region for endpoint {}, URI {} is determined as {}", + endpoint, epr, region); + return new AwsClientBuilder.EndpointConfiguration(endpoint, region); + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java index 932c472f5bea2..e99ec1ab28f75 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java @@ -20,6 +20,7 @@ import com.amazonaws.ClientConfiguration; import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.metrics.RequestMetricCollector; import com.amazonaws.services.s3.AmazonS3; import org.apache.hadoop.classification.InterfaceAudience; @@ -40,12 +41,21 @@ public class InconsistentS3ClientFactory extends DefaultS3ClientFactory { * Logs a warning that this is being done. * @param credentials credentials to use * @param awsConf AWS configuration + * @param metrics + * @param endpoint + * @param pathStyleAccess * @return an inconsistent client. */ @Override protected AmazonS3 newAmazonS3Client(AWSCredentialsProvider credentials, - ClientConfiguration awsConf) { + ClientConfiguration awsConf, + final RequestMetricCollector metrics, + final String endpoint, + final boolean pathStyleAccess) { LOG.warn("** FAILURE INJECTION ENABLED. Do not run in production! **"); - return new InconsistentAmazonS3Client(credentials, awsConf, getConf()); + InconsistentAmazonS3Client s3 + = new InconsistentAmazonS3Client(credentials, awsConf, getConf()); + configureAmazonS3Client(s3, endpoint, pathStyleAccess); + return s3; } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java index 15f69bde3f119..d9b774a11f1aa 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java @@ -49,10 +49,15 @@ import org.apache.hadoop.fs.StreamCapabilities; import org.apache.hadoop.fs.s3a.commit.CommitConstants; import org.apache.hadoop.fs.s3a.commit.PutTracker; +import org.apache.hadoop.fs.s3a.impl.statistics.BlockOutputStreamStatistics; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsLogging; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; import org.apache.hadoop.util.Progressable; import static org.apache.hadoop.fs.s3a.S3AUtils.*; import static org.apache.hadoop.fs.s3a.Statistic.*; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.emptyStatistics; import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; /** @@ -67,7 +72,7 @@ @InterfaceAudience.Private @InterfaceStability.Unstable class S3ABlockOutputStream extends OutputStream implements - StreamCapabilities { + StreamCapabilities, IOStatisticsSource { private static final Logger LOG = LoggerFactory.getLogger(S3ABlockOutputStream.class); @@ -81,6 +86,9 @@ class S3ABlockOutputStream extends OutputStream implements /** Size of all blocks. */ private final int blockSize; + /** IO Statistics. */ + private final IOStatistics iostatistics; + /** Total bytes for uploads submitted so far. */ private long bytesSubmitted; @@ -109,7 +117,7 @@ class S3ABlockOutputStream extends OutputStream implements private long blockCount = 0; /** Statistics to build up. */ - private final S3AInstrumentation.OutputStreamStatistics statistics; + private final BlockOutputStreamStatistics statistics; /** * Write operation helper; encapsulation of the filesystem operations. @@ -146,7 +154,7 @@ class S3ABlockOutputStream extends OutputStream implements Progressable progress, long blockSize, S3ADataBlocks.BlockFactory blockFactory, - S3AInstrumentation.OutputStreamStatistics statistics, + BlockOutputStreamStatistics statistics, WriteOperationHelper writeOperationHelper, PutTracker putTracker) throws IOException { @@ -155,6 +163,10 @@ class S3ABlockOutputStream extends OutputStream implements this.blockFactory = blockFactory; this.blockSize = (int) blockSize; this.statistics = statistics; + // test instantiations may not provide statistics; + iostatistics = statistics != null + ? statistics.getIOStatistics() + : emptyStatistics(); this.writeOperationHelper = writeOperationHelper; this.putTracker = putTracker; Preconditions.checkArgument(blockSize >= Constants.MULTIPART_MIN_SIZE, @@ -282,6 +294,7 @@ public synchronized void write(byte[] source, int offset, int len) if (len == 0) { return; } + statistics.writeBytes(len); S3ADataBlocks.DataBlock block = createBlockIfNeeded(); int written = block.write(source, offset, len); int remainingCapacity = block.remainingCapacity(); @@ -473,6 +486,7 @@ public String toString() { if (block != null) { sb.append(", activeBlock=").append(block); } + sb.append(IOStatisticsLogging.ioStatisticsSourceToString(this)); sb.append('}'); return sb.toString(); } @@ -493,7 +507,7 @@ private long now() { * Get the statistics for this stream. * @return stream statistics */ - S3AInstrumentation.OutputStreamStatistics getStatistics() { + BlockOutputStreamStatistics getStatistics() { return statistics; } @@ -525,6 +539,11 @@ public boolean hasCapability(String capability) { } } + @Override + public IOStatistics getIOStatistics() { + return iostatistics; + } + /** * Multiple partition upload. */ diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java index 156defb7ca031..9fbf0008cf397 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java @@ -37,6 +37,7 @@ import org.slf4j.LoggerFactory; import org.apache.hadoop.fs.FSExceptionMessages; +import org.apache.hadoop.fs.s3a.impl.statistics.BlockOutputStreamStatistics; import org.apache.hadoop.util.DirectBufferPool; import static org.apache.hadoop.fs.s3a.S3ADataBlocks.DataBlock.DestState.*; @@ -180,7 +181,7 @@ protected BlockFactory(S3AFileSystem owner) { * @return a new block. */ abstract DataBlock create(long index, int limit, - S3AInstrumentation.OutputStreamStatistics statistics) + BlockOutputStreamStatistics statistics) throws IOException; /** @@ -210,10 +211,10 @@ enum DestState {Writing, Upload, Closed} private volatile DestState state = Writing; protected final long index; - protected final S3AInstrumentation.OutputStreamStatistics statistics; + private final BlockOutputStreamStatistics statistics; protected DataBlock(long index, - S3AInstrumentation.OutputStreamStatistics statistics) { + BlockOutputStreamStatistics statistics) { this.index = index; this.statistics = statistics; } @@ -372,6 +373,10 @@ protected void blockReleased() { statistics.blockReleased(); } } + + protected BlockOutputStreamStatistics getStatistics() { + return statistics; + } } // ==================================================================== @@ -387,7 +392,7 @@ static class ArrayBlockFactory extends BlockFactory { @Override DataBlock create(long index, int limit, - S3AInstrumentation.OutputStreamStatistics statistics) + BlockOutputStreamStatistics statistics) throws IOException { return new ByteArrayBlock(0, limit, statistics); } @@ -432,7 +437,7 @@ static class ByteArrayBlock extends DataBlock { ByteArrayBlock(long index, int limit, - S3AInstrumentation.OutputStreamStatistics statistics) { + BlockOutputStreamStatistics statistics) { super(index, statistics); this.limit = limit; buffer = new S3AByteArrayOutputStream(limit); @@ -510,7 +515,7 @@ static class ByteBufferBlockFactory extends BlockFactory { @Override ByteBufferBlock create(long index, int limit, - S3AInstrumentation.OutputStreamStatistics statistics) + BlockOutputStreamStatistics statistics) throws IOException { return new ByteBufferBlock(index, limit, statistics); } @@ -560,7 +565,7 @@ class ByteBufferBlock extends DataBlock { */ ByteBufferBlock(long index, int bufferSize, - S3AInstrumentation.OutputStreamStatistics statistics) { + BlockOutputStreamStatistics statistics) { super(index, statistics); this.bufferSize = bufferSize; blockBuffer = requestBuffer(bufferSize); @@ -805,7 +810,7 @@ static class DiskBlockFactory extends BlockFactory { @Override DataBlock create(long index, int limit, - S3AInstrumentation.OutputStreamStatistics statistics) + BlockOutputStreamStatistics statistics) throws IOException { File destFile = getOwner() .createTmpFileForWrite(String.format("s3ablock-%04d-", index), @@ -829,7 +834,7 @@ static class DiskBlock extends DataBlock { DiskBlock(File bufferFile, int limit, long index, - S3AInstrumentation.OutputStreamStatistics statistics) + BlockOutputStreamStatistics statistics) throws FileNotFoundException { super(index, statistics); this.limit = limit; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index 6d2b3a84ca702..a1b2754a70e54 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -110,8 +110,15 @@ import org.apache.hadoop.fs.s3a.impl.RenameOperation; import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum; import org.apache.hadoop.fs.s3a.impl.StoreContext; +import org.apache.hadoop.fs.s3a.impl.statistics.CommitterStatistics; +import org.apache.hadoop.fs.s3a.impl.statistics.BondedS3AStatisticsContext; +import org.apache.hadoop.fs.s3a.impl.statistics.S3AStatisticsContext; +import org.apache.hadoop.fs.s3a.impl.statistics.StatisticsFromAwsSdk; import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; import org.apache.hadoop.fs.s3a.select.InternalSelectConstants; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.Text; import org.apache.hadoop.security.token.TokenIdentifier; @@ -174,6 +181,7 @@ import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit; import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletionIgnoringExceptions; import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isUnknownBucket; +import static org.apache.hadoop.fs.s3a.impl.InternalConstants.AWS_SDK_METRICS_ENABLED; import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_404; import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.fixBucketRegion; import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.logDnsLookup; @@ -195,7 +203,7 @@ @InterfaceAudience.Private @InterfaceStability.Evolving public class S3AFileSystem extends FileSystem implements StreamCapabilities, - AWSPolicyProvider, DelegationTokenProvider { + AWSPolicyProvider, DelegationTokenProvider, IOStatisticsSource { /** * Default blocksize as used in blocksize and FS status queries. */ @@ -250,6 +258,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, private S3AInstrumentation instrumentation; private final S3AStorageStatistics storageStatistics = createStorageStatistics(); + + private S3AStatisticsContext statisticsContext; + private long readAhead; private S3AInputPolicy inputPolicy; private ChangeDetectionPolicy changeDetectionPolicy; @@ -314,7 +325,8 @@ private static void addDeprecatedKeys() { * bucket-specific options are patched over the base ones before any use is * made of the config. */ - public void initialize(URI name, Configuration originalConf) + public void + initialize(URI name, Configuration originalConf) throws IOException { // get the host; this is guaranteed to be non-null, non-empty bucket = name.getHost(); @@ -343,6 +355,7 @@ public void initialize(URI name, Configuration originalConf) invoker = new Invoker(new S3ARetryPolicy(getConf()), onRetry); instrumentation = new S3AInstrumentation(uri); + initializeStatisticsBinding(); // Username is the current user at the time the FS was instantiated. owner = UserGroupInformation.getCurrentUser(); @@ -352,7 +365,8 @@ public void initialize(URI name, Configuration originalConf) s3guardInvoker = new Invoker(new S3GuardExistsRetryPolicy(getConf()), onRetry); - writeHelper = new WriteOperationHelper(this, getConf()); + writeHelper = new WriteOperationHelper(this, getConf(), + statisticsContext); failOnMetadataWriteError = conf.getBoolean(FAIL_ON_METADATA_WRITE_ERROR, FAIL_ON_METADATA_WRITE_ERROR_DEFAULT); @@ -502,6 +516,31 @@ private void doBucketProbing() throws IOException { } } + /** + * Initialize the statistics binding. + * This is done by creating an {@code IntegratedS3AStatisticsContext} + * with callbacks to get the FS's instrumentation and FileSystem.statistics + * field; the latter may change after {@link #initialize(URI, Configuration)}, + * so needs to be dynamically adapted. + * Protected so that (mock) subclasses can replace it with a + * different statistics binding, if desired. + */ + protected void initializeStatisticsBinding() { + statisticsContext = new BondedS3AStatisticsContext( + new BondedS3AStatisticsContext.S3AFSStatisticsSource() { + + @Override + public S3AInstrumentation getInstrumentation() { + return S3AFileSystem.this.getInstrumentation(); + } + + @Override + public Statistics getInstanceStatistics() { + return S3AFileSystem.this.statistics; + } + }); + } + /** * Initialize the thread pool. * This must be re-invoked after replacing the S3Client during test @@ -581,6 +620,7 @@ protected void verifyBucketExistsV2() * Get S3A Instrumentation. For test purposes. * @return this instance's instrumentation. */ + @VisibleForTesting public S3AInstrumentation getInstrumentation() { return instrumentation; } @@ -640,8 +680,16 @@ private void bindAWSClient(URI name, boolean dtEnabled) throws IOException { S3_CLIENT_FACTORY_IMPL, DEFAULT_S3_CLIENT_FACTORY_IMPL, S3ClientFactory.class); + StatisticsFromAwsSdk awsStats = null; +// TODO: when the S3 client building code works with different regions, +// then non-null stats can be passed in here. + if (AWS_SDK_METRICS_ENABLED) { + awsStats = statisticsContext.newStatisticsFromAwsSdk(); + } + s3 = ReflectionUtils.newInstance(s3ClientFactoryClass, conf) - .createS3Client(getUri(), bucket, credentials, uaSuffix); + .createS3Client(getUri(), bucket, credentials, uaSuffix, + awsStats); } /** @@ -1139,7 +1187,7 @@ private S3AReadOpContext createReadContext( invoker, s3guardInvoker, statistics, - instrumentation, + statisticsContext, fileStatus, seekPolicy, changePolicy, @@ -1244,7 +1292,7 @@ public FSDataOutputStream create(Path f, FsPermission permission, progress, partSize, blockFactory, - instrumentation.newOutputStreamStatistics(statistics), + statisticsContext.newOutputStreamStatistics(), getWriteOperationHelper(), putTracker), null); @@ -1706,7 +1754,7 @@ protected void incrementStatistic(Statistic statistic) { * @param count the count to increment */ protected void incrementStatistic(Statistic statistic, long count) { - instrumentation.incrementCounter(statistic, count); + statisticsContext.incrementCounter(statistic, count); storageStatistics.incrementCounter(statistic, count); } @@ -1716,7 +1764,7 @@ protected void incrementStatistic(Statistic statistic, long count) { * @param count the count to decrement */ protected void decrementGauge(Statistic statistic, long count) { - instrumentation.decrementGauge(statistic, count); + statisticsContext.decrementGauge(statistic, count); } /** @@ -1725,7 +1773,7 @@ protected void decrementGauge(Statistic statistic, long count) { * @param count the count to increment */ protected void incrementGauge(Statistic statistic, long count) { - instrumentation.incrementGauge(statistic, count); + statisticsContext.incrementGauge(statistic, count); } /** @@ -1738,6 +1786,7 @@ public void operationRetried(Exception ex) { if (isThrottleException(ex)) { operationThrottled(false); } else { + incrementStatistic(STORE_IO_RETRY); incrementStatistic(IGNORED_ERRORS); } } @@ -1789,11 +1838,11 @@ private void operationThrottled(boolean metastore) { LOG.debug("Request throttled on {}", metastore ? "S3": "DynamoDB"); if (metastore) { incrementStatistic(S3GUARD_METADATASTORE_THROTTLED); - instrumentation.addValueToQuantiles(S3GUARD_METADATASTORE_THROTTLE_RATE, + statisticsContext.addValueToQuantiles(S3GUARD_METADATASTORE_THROTTLE_RATE, 1); } else { incrementStatistic(STORE_IO_THROTTLED); - instrumentation.addValueToQuantiles(STORE_IO_THROTTLE_RATE, 1); + statisticsContext.addValueToQuantiles(STORE_IO_THROTTLE_RATE, 1); } } @@ -1806,6 +1855,16 @@ public S3AStorageStatistics getStorageStatistics() { return storageStatistics; } + /** + * Get this filesystem's storage statistics as IO Statistics. + * @return statistics + */ + @Override + public IOStatistics getIOStatistics() { + return IOStatisticsBinding.fromStorageStatistics( + storageStatistics); + } + /** * Request object metadata; increments counters in the process. * Retry policy: retry untranslated. @@ -3441,8 +3500,9 @@ private CopyResult copyFile(String srcKey, String dstKey, long size, ChangeTracker changeTracker = new ChangeTracker( keyToQualifiedPath(srcKey).toString(), changeDetectionPolicy, - readContext.instrumentation.newInputStreamStatistics() - .getVersionMismatchCounter(), + readContext.getS3AStatisticsContext() + .newInputStreamStatistics() + .getChangeTrackerStatistics(), srcAttributes); String action = "copyFile(" + srcKey + ", " + dstKey + ")"; @@ -4448,8 +4508,8 @@ void abortMultipartUpload(MultipartUpload upload) { * Create a new instance of the committer statistics. * @return a new committer statistics instance */ - public S3AInstrumentation.CommitterStatistics newCommitterStatistics() { - return instrumentation.newCommitterStatistics(); + public CommitterStatistics newCommitterStatistics() { + return statisticsContext.newCommitterStatistics(); } @SuppressWarnings("deprecation") @@ -4563,8 +4623,9 @@ private FSDataInputStream select(final Path source, ChangeTracker changeTracker = new ChangeTracker(uri.toString(), changeDetectionPolicy, - readContext.instrumentation.newInputStreamStatistics() - .getVersionMismatchCounter(), + readContext.getS3AStatisticsContext() + .newInputStreamStatistics() + .getChangeTrackerStatistics(), objectAttributes); // will retry internally if wrong version detected @@ -4720,7 +4781,7 @@ public StoreContext createStoreContext() { boundedThreadPool, executorCapacity, invoker, - getInstrumentation(), + statisticsContext, getStorageStatistics(), getInputPolicy(), changeDetectionPolicy, diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java index 9c8b9ae7a156e..956eeb4608f74 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java @@ -33,10 +33,13 @@ import org.apache.hadoop.fs.CanSetReadahead; import org.apache.hadoop.fs.CanUnbuffer; import org.apache.hadoop.fs.FSExceptionMessages; -import org.apache.hadoop.fs.FSInputStream; +import org.apache.hadoop.fs.s3a.impl.statistics.S3AInputStreamStatistics; +import org.apache.hadoop.fs.s3a.impl.ChangeTracker; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.StreamCapabilities; -import org.apache.hadoop.fs.s3a.impl.ChangeTracker; +import org.apache.hadoop.fs.FSInputStream; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -68,7 +71,7 @@ @InterfaceAudience.Private @InterfaceStability.Evolving public class S3AInputStream extends FSInputStream implements CanSetReadahead, - CanUnbuffer, StreamCapabilities { + CanUnbuffer, StreamCapabilities, IOStatisticsSource { public static final String E_NEGATIVE_READAHEAD_VALUE = "Negative readahead value"; @@ -97,7 +100,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead, private final String uri; private static final Logger LOG = LoggerFactory.getLogger(S3AInputStream.class); - private final S3AInstrumentation.InputStreamStatistics streamStatistics; + private final S3AInputStreamStatistics streamStatistics; private S3AEncryptionMethods serverSideEncryptionAlgorithm; private String serverSideEncryptionKey; private S3AInputPolicy inputPolicy; @@ -123,6 +126,11 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead, /** change tracker. */ private final ChangeTracker changeTracker; + /** + * IOStatistics report. + */ + private final IOStatistics ioStatistics; + /** * Create the stream. * This does not attempt to open it; that is only done on the first @@ -146,13 +154,15 @@ public S3AInputStream(S3AReadOpContext ctx, this.contentLength = l; this.client = client; this.uri = "s3a://" + this.bucket + "/" + this.key; - this.streamStatistics = ctx.instrumentation.newInputStreamStatistics(); + this.streamStatistics = ctx.getS3AStatisticsContext() + .newInputStreamStatistics(); + this.ioStatistics = streamStatistics.getIOStatistics(); this.serverSideEncryptionAlgorithm = s3Attributes.getServerSideEncryptionAlgorithm(); this.serverSideEncryptionKey = s3Attributes.getServerSideEncryptionKey(); this.changeTracker = new ChangeTracker(uri, ctx.getChangeDetectionPolicy(), - streamStatistics.getVersionMismatchCounter(), + streamStatistics.getChangeTrackerStatistics(), s3Attributes); setInputPolicy(ctx.getInputPolicy()); setReadahead(ctx.getReadahead()); @@ -290,8 +300,6 @@ private void seekInStream(long targetPos, long length) throws IOException { long skipped = wrappedStream.skip(diff); if (skipped > 0) { pos += skipped; - // as these bytes have been read, they are included in the counter - incrementBytesRead(diff); } if (pos == targetPos) { @@ -348,7 +356,7 @@ private void lazySeek(long targetPos, long len) throws IOException { // open. After that, an exception generally means the file has changed // and there is no point retrying anymore. Invoker invoker = context.getReadInvoker(); - invoker.maybeRetry(streamStatistics.openOperations == 0, + invoker.maybeRetry(streamStatistics.getOpenOperations() == 0, "lazySeek", pathStr, true, () -> { //For lazy seek @@ -747,7 +755,7 @@ public void readFully(long position, byte[] buffer, int offset, int length) */ @InterfaceAudience.Private @InterfaceStability.Unstable - public S3AInstrumentation.InputStreamStatistics getS3AStreamStatistics() { + public S3AInputStreamStatistics getS3AStreamStatistics() { return streamStatistics; } @@ -850,4 +858,9 @@ public boolean hasCapability(String capability) { boolean isObjectStreamOpen() { return wrappedStream != null; } + + @Override + public IOStatistics getIOStatistics() { + return ioStatistics; + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java index b9918b5098946..040895a40c9e5 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java @@ -18,14 +18,31 @@ package org.apache.hadoop.fs.s3a; +import javax.annotation.Nullable; + import com.google.common.annotations.VisibleForTesting; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.fs.FileSystem.Statistics; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.s3a.impl.statistics.ChangeTrackerStatistics; +import org.apache.hadoop.fs.s3a.impl.statistics.CommitterStatistics; +import org.apache.hadoop.fs.s3a.impl.statistics.CountersAndGauges; +import org.apache.hadoop.fs.s3a.impl.statistics.CountingChangeTracker; +import org.apache.hadoop.fs.s3a.impl.statistics.DelegationTokenStatistics; +import org.apache.hadoop.fs.s3a.impl.statistics.S3AInputStreamStatistics; +import org.apache.hadoop.fs.s3a.impl.statistics.BlockOutputStreamStatistics; +import org.apache.hadoop.fs.s3a.impl.statistics.StatisticsFromAwsSdk; import org.apache.hadoop.fs.s3a.s3guard.MetastoreInstrumentation; +import org.apache.hadoop.fs.statistics.IOStatisticEntry; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsLogging; +import org.apache.hadoop.fs.statistics.IOStatisticsSupport; +import org.apache.hadoop.fs.statistics.StreamStatisticNames; +import org.apache.hadoop.fs.statistics.impl.CounterIOStatistics; +import org.apache.hadoop.fs.statistics.impl.DynamicIOStatisticsBuilder; import org.apache.hadoop.metrics2.AbstractMetric; import org.apache.hadoop.metrics2.MetricStringBuilder; import org.apache.hadoop.metrics2.MetricsCollector; @@ -43,18 +60,37 @@ import java.io.Closeable; import java.net.URI; +import java.time.Duration; import java.util.HashMap; import java.util.Map; import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_OPENED; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.counterIOStatistics; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.dynamicIOStatistics; import static org.apache.hadoop.fs.s3a.Statistic.*; /** * Instrumentation of S3a. - * Derived from the {@code AzureFileSystemInstrumentation}. - * + *

+ * 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,32 +99,33 @@ */ @InterfaceAudience.Private @InterfaceStability.Evolving -public class S3AInstrumentation implements Closeable, MetricsSource { +public class S3AInstrumentation implements Closeable, MetricsSource, + CountersAndGauges { private static final Logger LOG = LoggerFactory.getLogger( S3AInstrumentation.class); private static final String METRICS_SOURCE_BASENAME = "S3AMetrics"; /** - * {@value #METRICS_SYSTEM_NAME} The name of the s3a-specific metrics + * {@value} The name of the s3a-specific metrics * system instance used for s3a metrics. */ public static final String METRICS_SYSTEM_NAME = "s3a-file-system"; /** - * {@value #CONTEXT} Currently all s3a metrics are placed in a single + * {@value} Currently all s3a metrics are placed in a single * "context". Distinct contexts may be used in the future. */ public static final String CONTEXT = "s3aFileSystem"; /** - * {@value #METRIC_TAG_FILESYSTEM_ID} The name of a field added to metrics + * {@value} The name of a field added to metrics * records that uniquely identifies a specific FileSystem instance. */ public static final String METRIC_TAG_FILESYSTEM_ID = "s3aFileSystemId"; /** - * {@value #METRIC_TAG_BUCKET} The name of a field added to metrics records + * {@value} The name of a field added to metrics records * that indicates the hostname portion of the FS URL. */ public static final String METRIC_TAG_BUCKET = "bucket"; @@ -192,6 +229,8 @@ public class S3AInstrumentation implements Closeable, MetricsSource { S3GUARD_METADATASTORE_THROTTLED, S3GUARD_METADATASTORE_AUTHORITATIVE_DIRECTORIES_UPDATED, STORE_IO_THROTTLED, + STORE_IO_REQUEST, + STORE_IO_RETRY, DELEGATION_TOKENS_ISSUED, FILES_DELETE_REJECTED }; @@ -561,9 +600,11 @@ public void decrementGauge(Statistic op, long count) { /** * Create a stream input statistics instance. * @return the new instance + * @param filesystemStatistics FS Stats. */ - public InputStreamStatistics newInputStreamStatistics() { - return new InputStreamStatistics(); + public S3AInputStreamStatistics newInputStreamStatistics( + final FileSystem.Statistics filesystemStatistics) { + return new InputStreamStatisticsImpl(filesystemStatistics); } /** @@ -579,8 +620,8 @@ public MetastoreInstrumentation getS3GuardInstrumentation() { * Create a new instance of the committer statistics. * @return a new committer statistics instance */ - CommitterStatistics newCommitterStatistics() { - return new CommitterStatistics(); + public CommitterStatistics newCommitterStatistics() { + return new CommitterStatisticsImpl(); } /** @@ -588,25 +629,27 @@ CommitterStatistics newCommitterStatistics() { * the filesystem-wide statistics. * @param statistics stream statistics */ - private void mergeInputStreamStatistics(InputStreamStatistics statistics) { - streamOpenOperations.incr(statistics.openOperations); - streamCloseOperations.incr(statistics.closeOperations); - streamClosed.incr(statistics.closed); - streamAborted.incr(statistics.aborted); - streamSeekOperations.incr(statistics.seekOperations); - streamReadExceptions.incr(statistics.readExceptions); - streamForwardSeekOperations.incr(statistics.forwardSeekOperations); - streamBytesSkippedOnSeek.incr(statistics.bytesSkippedOnSeek); - streamBackwardSeekOperations.incr(statistics.backwardSeekOperations); - streamBytesBackwardsOnSeek.incr(statistics.bytesBackwardsOnSeek); - streamBytesRead.incr(statistics.bytesRead); - streamReadOperations.incr(statistics.readOperations); - streamReadFullyOperations.incr(statistics.readFullyOperations); - streamReadsIncomplete.incr(statistics.readsIncomplete); - streamBytesReadInClose.incr(statistics.bytesReadInClose); - streamBytesDiscardedInAbort.incr(statistics.bytesDiscardedInAbort); + private void mergeInputStreamStatistics( + InputStreamStatisticsImpl statistics) { + + streamOpenOperations.incr(statistics.getStatistic(STREAM_READ_OPENED)); + streamCloseOperations.incr(statistics.getCloseOperations()); + streamClosed.incr(statistics.getClosed()); + streamAborted.incr(statistics.getAborted()); + streamSeekOperations.incr(statistics.getSeekOperations()); + streamReadExceptions.incr(statistics.getReadExceptions()); + streamForwardSeekOperations.incr(statistics.getForwardSeekOperations()); + streamBytesSkippedOnSeek.incr(statistics.getBytesSkippedOnSeek()); + streamBackwardSeekOperations.incr(statistics.getBackwardSeekOperations()); + streamBytesBackwardsOnSeek.incr(statistics.getBytesBackwardsOnSeek()); + streamBytesRead.incr(statistics.getBytesRead()); + streamReadOperations.incr(statistics.getReadOperations()); + streamReadFullyOperations.incr(statistics.getReadFullyOperations()); + streamReadsIncomplete.incr(statistics.getReadsIncomplete()); + streamBytesReadInClose.incr(statistics.getBytesReadInClose()); + streamBytesDiscardedInAbort.incr(statistics.getBytesDiscardedInAbort()); incrementCounter(STREAM_READ_VERSION_MISMATCHES, - statistics.versionMismatches.get()); + statistics.getVersionMismatches()); } @Override @@ -622,7 +665,8 @@ public void close() { throttleRateQuantile.stop(); s3GuardThrottleRateQuantile.stop(); metricsSystem.unregisterSource(metricsSourceName); - int activeSources = --metricsSourceActiveCounter; + metricsSourceActiveCounter--; + int activeSources = metricsSourceActiveCounter; if (activeSources == 0) { LOG.debug("Shutting down metrics publisher"); metricsSystem.publishMetricsNow(); @@ -634,35 +678,62 @@ public void close() { /** * Statistics updated by an input stream during its actual operation. - * These counters not thread-safe and are for use in a single instance - * of a stream. */ - @InterfaceAudience.Private - @InterfaceStability.Unstable - public final class InputStreamStatistics implements AutoCloseable { - public long openOperations; - public long closeOperations; - public long closed; - public long aborted; - public long seekOperations; - public long readExceptions; - public long forwardSeekOperations; - public long backwardSeekOperations; - public long bytesRead; - public long bytesSkippedOnSeek; - public long bytesBackwardsOnSeek; - public long readOperations; - public long readFullyOperations; - public long readsIncomplete; - public long bytesReadInClose; - public long bytesDiscardedInAbort; - public long policySetCount; - public long inputPolicy; - /** This is atomic so that it can be passed as a reference. */ + private final class InputStreamStatisticsImpl implements + S3AInputStreamStatistics { + + /** + * Distance used when incrementing FS stats. + */ + private static final int DISTANCE = 5; + + private final FileSystem.Statistics filesystemStatistics; + + /** + * Counters implemented directly in the IOStatistics. + */ + private final CounterIOStatistics statsCounters = counterIOStatistics( + StreamStatisticNames.STREAM_READ_ABORTED, + StreamStatisticNames.STREAM_READ_BYTES_DISCARDED_ABORT, + StreamStatisticNames.STREAM_READ_CLOSED, + StreamStatisticNames.STREAM_READ_CLOSE_BYTES_READ, + StreamStatisticNames.STREAM_READ_CLOSE_OPERATIONS, + StreamStatisticNames.STREAM_READ_OPENED, + StreamStatisticNames.STREAM_READ_BYTES, + StreamStatisticNames.STREAM_READ_EXCEPTIONS, + StreamStatisticNames.STREAM_READ_FULLY_OPERATIONS, + StreamStatisticNames.STREAM_READ_OPERATIONS, + StreamStatisticNames.STREAM_READ_OPERATIONS_INCOMPLETE, + StreamStatisticNames.STREAM_READ_VERSION_MISMATCHES, + StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS, + StreamStatisticNames.STREAM_READ_SEEK_BACKWARD_OPERATIONS, + StreamStatisticNames.STREAM_READ_SEEK_FORWARD_OPERATIONS, + StreamStatisticNames.STREAM_READ_SEEK_BYTES_BACKWARDS, + StreamStatisticNames.STREAM_READ_SEEK_BYTES_READ, + StreamStatisticNames.STREAM_READ_SEEK_BYTES_SKIPPED); + + private final AtomicLong policySetCount= new AtomicLong(0); + private volatile long inputPolicy; private final AtomicLong versionMismatches = new AtomicLong(0); - private InputStreamStatistics mergedStats; + private InputStreamStatisticsImpl mergedStats; + + private InputStreamStatisticsImpl( + FileSystem.Statistics filesystemStatistics) { + this.filesystemStatistics = filesystemStatistics; + } - private InputStreamStatistics() { + private long increment(String name) { + return statsCounters.increment(name); + } + + private long increment(String name, long value) { + return statsCounters.increment(name, value); + } + + @Override + public Long getStatistic(final String name) { + return statsCounters.getStatistic(name) + .scalar(IOStatisticEntry.IOSTATISTIC_COUNTER); } /** @@ -670,10 +741,12 @@ private InputStreamStatistics() { * @param negativeOffset how far was the seek? * This is expected to be negative. */ + @Override public void seekBackwards(long negativeOffset) { - seekOperations++; - backwardSeekOperations++; - bytesBackwardsOnSeek -= negativeOffset; + increment(StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS); + increment(StreamStatisticNames.STREAM_READ_SEEK_BACKWARD_OPERATIONS); + increment(StreamStatisticNames.STREAM_READ_SEEK_BYTES_BACKWARDS, + -negativeOffset); } /** @@ -682,11 +755,13 @@ public void seekBackwards(long negativeOffset) { * @param skipped number of bytes skipped by reading from the stream. * If the seek was implemented by a close + reopen, set this to zero. */ + @Override public void seekForwards(long skipped) { - seekOperations++; - forwardSeekOperations++; + increment(StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS); + increment(StreamStatisticNames.STREAM_READ_SEEK_FORWARD_OPERATIONS); if (skipped > 0) { - bytesSkippedOnSeek += skipped; + increment(StreamStatisticNames.STREAM_READ_SEEK_BYTES_SKIPPED, + skipped); } } @@ -694,10 +769,9 @@ public void seekForwards(long skipped) { * The inner stream was opened. * @return the previous count */ + @Override public long streamOpened() { - long count = openOperations; - openOperations++; - return count; + return increment(STREAM_READ_OPENED); } /** @@ -707,23 +781,27 @@ public long streamOpened() { * @param remainingInCurrentRequest the number of bytes remaining in * the current request. */ + @Override public void streamClose(boolean abortedConnection, long remainingInCurrentRequest) { - closeOperations++; + increment(StreamStatisticNames.STREAM_READ_CLOSED); if (abortedConnection) { - this.aborted++; - bytesDiscardedInAbort += remainingInCurrentRequest; + increment(StreamStatisticNames.STREAM_READ_ABORTED); + increment(StreamStatisticNames.STREAM_READ_BYTES_DISCARDED_ABORT, + remainingInCurrentRequest); } else { - closed++; - bytesReadInClose += remainingInCurrentRequest; + increment(StreamStatisticNames.STREAM_READ_CLOSED); + increment(StreamStatisticNames.STREAM_READ_CLOSE_BYTES_READ, + remainingInCurrentRequest); } } /** * An ignored stream read exception was received. */ + @Override public void readException() { - readExceptions++; + increment(StreamStatisticNames.STREAM_READ_EXCEPTIONS); } /** @@ -731,9 +809,10 @@ public void readException() { * no-op if the argument is negative. * @param bytes number of bytes read */ + @Override public void bytesRead(long bytes) { if (bytes > 0) { - bytesRead += bytes; + increment(StreamStatisticNames.STREAM_READ_BYTES, bytes); } } @@ -742,8 +821,9 @@ public void bytesRead(long bytes) { * @param pos starting position of the read * @param len length of bytes to read */ + @Override public void readOperationStarted(long pos, long len) { - readOperations++; + increment(StreamStatisticNames.STREAM_READ_OPERATIONS); } /** @@ -752,8 +832,9 @@ public void readOperationStarted(long pos, long len) { * @param pos starting position of the read * @param len length of bytes to read */ + @Override public void readFullyOperationStarted(long pos, long len) { - readFullyOperations++; + increment(StreamStatisticNames.STREAM_READ_FULLY_OPERATIONS); } /** @@ -761,9 +842,10 @@ public void readFullyOperationStarted(long pos, long len) { * @param requested number of requested bytes * @param actual the actual number of bytes */ + @Override public void readOperationCompleted(int requested, int actual) { if (requested > actual) { - readsIncomplete++; + increment(StreamStatisticNames.STREAM_READ_OPERATIONS_INCOMPLETE); } } @@ -780,17 +862,20 @@ public void close() { * The input policy has been switched. * @param updatedPolicy enum value of new policy. */ + @Override public void inputPolicySet(int updatedPolicy) { - policySetCount++; + policySetCount.incrementAndGet(); inputPolicy = updatedPolicy; } /** - * Get a reference to the version mismatch counter. - * @return a counter which can be incremented. + * The change tracker increments {@code versionMismatches} on any + * mismatch. + * @return change tracking. */ - public AtomicLong getVersionMismatchCounter() { - return versionMismatches; + @Override + public ChangeTrackerStatistics getChangeTrackerStatistics() { + return new CountingChangeTracker(versionMismatches); } /** @@ -804,26 +889,7 @@ public AtomicLong getVersionMismatchCounter() { public String toString() { final StringBuilder sb = new StringBuilder( "StreamStatistics{"); - sb.append("OpenOperations=").append(openOperations); - sb.append(", CloseOperations=").append(closeOperations); - sb.append(", Closed=").append(closed); - sb.append(", Aborted=").append(aborted); - sb.append(", SeekOperations=").append(seekOperations); - sb.append(", ReadExceptions=").append(readExceptions); - sb.append(", ForwardSeekOperations=") - .append(forwardSeekOperations); - sb.append(", BackwardSeekOperations=") - .append(backwardSeekOperations); - sb.append(", BytesSkippedOnSeek=").append(bytesSkippedOnSeek); - sb.append(", BytesBackwardsOnSeek=").append(bytesBackwardsOnSeek); - sb.append(", BytesRead=").append(bytesRead); - sb.append(", BytesRead excluding skipped=") - .append(bytesRead - bytesSkippedOnSeek); - sb.append(", ReadOperations=").append(readOperations); - sb.append(", ReadFullyOperations=").append(readFullyOperations); - sb.append(", ReadsIncomplete=").append(readsIncomplete); - sb.append(", BytesReadInClose=").append(bytesReadInClose); - sb.append(", BytesDiscardedInAbort=").append(bytesDiscardedInAbort); + sb.append(IOStatisticsLogging.iostatisticsToString(statsCounters)); sb.append(", InputPolicy=").append(inputPolicy); sb.append(", InputPolicySetCount=").append(policySetCount); sb.append(", versionMismatches=").append(versionMismatches.get()); @@ -836,50 +902,56 @@ public String toString() { * Takes a diff between the current version of the stats and the * version of the stats when merge was last called, and merges the diff * into the instrumentation instance. Used to periodically merge the - * stats into the fs-wide stats. Behavior is undefined if called on a - * closed instance. + * stats into the fs-wide stats. + *

+ * Behavior is undefined if called on a closed instance. */ - void merge(boolean isClosed) { + @Override + public void merge(boolean isClosed) { if (mergedStats != null) { - mergeInputStreamStatistics(diff(mergedStats)); + mergeInputStreamStatistics(setd(mergedStats)); } else { mergeInputStreamStatistics(this); } // If stats are closed, no need to create another copy if (!isClosed) { mergedStats = copy(); + } else { + // stream is being closed. + // increment the filesystem statistics for this thread. + if (filesystemStatistics != null) { + filesystemStatistics.incrementBytesReadByDistance(DISTANCE, + getBytesRead() - getBytesReadInClose()); + } } } /** - * Returns a diff between this {@link InputStreamStatistics} instance and - * the given {@link InputStreamStatistics} instance. + * Set the dest variable to the difference of the two + * other values. + * @param dest destination + * @param l left side + * @param r right side + */ + private void setd(AtomicLong dest, AtomicLong l, AtomicLong r) { + dest.set(l.get() - r.get()); + } + + /** + * Returns a diff between this {@link InputStreamStatisticsImpl} instance + * and the given {@link InputStreamStatisticsImpl} instance. */ - private InputStreamStatistics diff(InputStreamStatistics inputStats) { - InputStreamStatistics diff = new InputStreamStatistics(); - diff.openOperations = openOperations - inputStats.openOperations; - diff.closeOperations = closeOperations - inputStats.closeOperations; - diff.closed = closed - inputStats.closed; - diff.aborted = aborted - inputStats.aborted; - diff.seekOperations = seekOperations - inputStats.seekOperations; - diff.readExceptions = readExceptions - inputStats.readExceptions; - diff.forwardSeekOperations = - forwardSeekOperations - inputStats.forwardSeekOperations; - diff.backwardSeekOperations = - backwardSeekOperations - inputStats.backwardSeekOperations; - diff.bytesRead = bytesRead - inputStats.bytesRead; - diff.bytesSkippedOnSeek = - bytesSkippedOnSeek - inputStats.bytesSkippedOnSeek; - diff.bytesBackwardsOnSeek = - bytesBackwardsOnSeek - inputStats.bytesBackwardsOnSeek; - diff.readOperations = readOperations - inputStats.readOperations; - diff.readFullyOperations = - readFullyOperations - inputStats.readFullyOperations; - diff.readsIncomplete = readsIncomplete - inputStats.readsIncomplete; - diff.bytesReadInClose = bytesReadInClose - inputStats.bytesReadInClose; - diff.bytesDiscardedInAbort = - bytesDiscardedInAbort - inputStats.bytesDiscardedInAbort; - diff.policySetCount = policySetCount - inputStats.policySetCount; + private InputStreamStatisticsImpl setd( + InputStreamStatisticsImpl inputStats) { + InputStreamStatisticsImpl diff = + new InputStreamStatisticsImpl(filesystemStatistics); + + // build the io stats diff + diff.statsCounters.copy(statsCounters); + diff.statsCounters.subtract(inputStats.statsCounters); + + setd(diff.policySetCount, policySetCount, + inputStats.policySetCount); diff.inputPolicy = inputPolicy - inputStats.inputPolicy; diff.versionMismatches.set(versionMismatches.longValue() - inputStats.versionMismatches.longValue()); @@ -887,62 +959,161 @@ private InputStreamStatistics diff(InputStreamStatistics inputStats) { } /** - * Returns a new {@link InputStreamStatistics} instance with all the same - * values as this {@link InputStreamStatistics}. + * Returns a new {@link InputStreamStatisticsImpl} instance with + * all the same values as this {@link InputStreamStatisticsImpl}. */ - private InputStreamStatistics copy() { - InputStreamStatistics copy = new InputStreamStatistics(); - copy.openOperations = openOperations; - copy.closeOperations = closeOperations; - copy.closed = closed; - copy.aborted = aborted; - copy.seekOperations = seekOperations; - copy.readExceptions = readExceptions; - copy.forwardSeekOperations = forwardSeekOperations; - copy.backwardSeekOperations = backwardSeekOperations; - copy.bytesRead = bytesRead; - copy.bytesSkippedOnSeek = bytesSkippedOnSeek; - copy.bytesBackwardsOnSeek = bytesBackwardsOnSeek; - copy.readOperations = readOperations; - copy.readFullyOperations = readFullyOperations; - copy.readsIncomplete = readsIncomplete; - copy.bytesReadInClose = bytesReadInClose; - copy.bytesDiscardedInAbort = bytesDiscardedInAbort; - copy.policySetCount = policySetCount; + private InputStreamStatisticsImpl copy() { + InputStreamStatisticsImpl copy = + new InputStreamStatisticsImpl(filesystemStatistics); + + copy.policySetCount.set(policySetCount.get()); + copy.versionMismatches.set(versionMismatches.get()); copy.inputPolicy = inputPolicy; return copy; } + + /** + * Create an IOStatistics instance which is dynamically updated. + * @return statistics + */ + @Override + public IOStatistics getIOStatistics() { + return IOStatisticsSupport.snapshot(statsCounters); + } + + @Override + public long getCloseOperations() { + return getStatistic(StreamStatisticNames.STREAM_READ_CLOSE_OPERATIONS); + } + + @Override + public long getClosed() { + return getStatistic(StreamStatisticNames.STREAM_READ_CLOSED); + } + + @Override + public long getAborted() { + return getStatistic(StreamStatisticNames.STREAM_READ_ABORTED); + } + + @Override + public long getForwardSeekOperations() { + return getStatistic( + StreamStatisticNames.STREAM_READ_SEEK_FORWARD_OPERATIONS); + } + + @Override + public long getBackwardSeekOperations() { + return getStatistic( + StreamStatisticNames.STREAM_READ_SEEK_BACKWARD_OPERATIONS); + } + + @Override + public long getBytesRead() { + return getStatistic(StreamStatisticNames.STREAM_READ_BYTES); + } + + @Override + public long getBytesSkippedOnSeek() { + return getStatistic(StreamStatisticNames.STREAM_READ_SEEK_BYTES_SKIPPED); + } + + @Override + public long getBytesBackwardsOnSeek() { + return getStatistic( + StreamStatisticNames.STREAM_READ_SEEK_BYTES_BACKWARDS); + } + + @Override + public long getBytesReadInClose() { + return getStatistic(StreamStatisticNames.STREAM_READ_CLOSE_BYTES_READ); + } + + @Override + public long getBytesDiscardedInAbort() { + return getStatistic( + StreamStatisticNames.STREAM_READ_BYTES_DISCARDED_ABORT); + } + + @Override + public long getOpenOperations() { + return getStatistic(STREAM_READ_OPENED); + } + + @Override + public long getSeekOperations() { + return getStatistic(StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS); + } + + @Override + public long getReadExceptions() { + return getStatistic(StreamStatisticNames.STREAM_READ_EXCEPTIONS); + } + + @Override + public long getReadOperations() { + return getStatistic(StreamStatisticNames.STREAM_READ_CLOSED); + } + + @Override + public long getReadFullyOperations() { + return getStatistic(StreamStatisticNames.STREAM_READ_CLOSED); + } + + @Override + public long getReadsIncomplete() { + return getStatistic(StreamStatisticNames.STREAM_READ_CLOSED); + } + + @Override + public long getPolicySetCount() { + return getStatistic(StreamStatisticNames.STREAM_READ_CLOSED); + } + + @Override + public long getVersionMismatches() { + return versionMismatches.get(); + } + + @Override + public long getInputPolicy() { + return inputPolicy; + } } /** * Create a stream output statistics instance. * @return the new instance */ - OutputStreamStatistics newOutputStreamStatistics(Statistics statistics) { - return new OutputStreamStatistics(statistics); + public BlockOutputStreamStatistics newOutputStreamStatistics( + FileSystem.Statistics filesystemStatistics) { + return new BlockOutputStreamStatisticsImpl(filesystemStatistics); } /** * Merge in the statistics of a single output stream into * the filesystem-wide statistics. - * @param statistics stream statistics + * @param source stream statistics */ - private void mergeOutputStreamStatistics(OutputStreamStatistics statistics) { - incrementCounter(STREAM_WRITE_TOTAL_TIME, statistics.totalUploadDuration()); - incrementCounter(STREAM_WRITE_QUEUE_DURATION, statistics.queueDuration); - incrementCounter(STREAM_WRITE_TOTAL_DATA, statistics.bytesUploaded); + private void mergeOutputStreamStatistics( + BlockOutputStreamStatisticsImpl source) { + incrementCounter(STREAM_WRITE_TOTAL_TIME, source.totalUploadDuration()); + incrementCounter(STREAM_WRITE_QUEUE_DURATION, source.queueDuration); + incrementCounter(STREAM_WRITE_TOTAL_DATA, source.bytesUploaded); incrementCounter(STREAM_WRITE_BLOCK_UPLOADS, - statistics.blockUploadsCompleted); + source.blockUploadsCompleted); + incrementCounter(STREAM_WRITE_FAILURES, source.blockUploadsFailed); } /** * Statistics updated by an output stream during its actual operation. - * Some of these stats may be relayed. However, as block upload is - * spans multiple + *

+ * Some of these stats are propagated to any passed in + * {@link FileSystem.Statistics} instance; this is only done + * in close() for better cross-thread accounting. */ - @InterfaceAudience.Private - @InterfaceStability.Unstable - public final class OutputStreamStatistics implements Closeable { + private final class BlockOutputStreamStatisticsImpl implements + BlockOutputStreamStatistics { private final AtomicLong blocksSubmitted = new AtomicLong(0); private final AtomicLong blocksInQueue = new AtomicLong(0); private final AtomicLong blocksActive = new AtomicLong(0); @@ -950,6 +1121,7 @@ public final class OutputStreamStatistics implements Closeable { private final AtomicLong blockUploadsFailed = new AtomicLong(0); private final AtomicLong bytesPendingUpload = new AtomicLong(0); + private final AtomicLong bytesWritten = new AtomicLong(0); private final AtomicLong bytesUploaded = new AtomicLong(0); private final AtomicLong transferDuration = new AtomicLong(0); private final AtomicLong queueDuration = new AtomicLong(0); @@ -957,30 +1129,34 @@ public final class OutputStreamStatistics implements Closeable { private final AtomicInteger blocksAllocated = new AtomicInteger(0); private final AtomicInteger blocksReleased = new AtomicInteger(0); - private Statistics statistics; + private final FileSystem.Statistics filesystemStatistics; - public OutputStreamStatistics(Statistics statistics){ - this.statistics = statistics; + private BlockOutputStreamStatisticsImpl( + @Nullable FileSystem.Statistics filesystemStatistics) { + this.filesystemStatistics = filesystemStatistics; } /** * A block has been allocated. */ - void blockAllocated() { + @Override + public void blockAllocated() { blocksAllocated.incrementAndGet(); } /** * A block has been released. */ - void blockReleased() { + @Override + public void blockReleased() { blocksReleased.incrementAndGet(); } /** * Block is queued for upload. */ - void blockUploadQueued(int blockSize) { + @Override + public void blockUploadQueued(int blockSize) { blocksSubmitted.incrementAndGet(); blocksInQueue.incrementAndGet(); bytesPendingUpload.addAndGet(blockSize); @@ -989,7 +1165,8 @@ void blockUploadQueued(int blockSize) { } /** Queued block has been scheduled for upload. */ - void blockUploadStarted(long duration, int blockSize) { + @Override + public void blockUploadStarted(long duration, int blockSize) { queueDuration.addAndGet(duration); blocksInQueue.decrementAndGet(); blocksActive.incrementAndGet(); @@ -998,8 +1175,9 @@ void blockUploadStarted(long duration, int blockSize) { } /** A block upload has completed. */ - void blockUploadCompleted(long duration, int blockSize) { - this.transferDuration.addAndGet(duration); + @Override + public void blockUploadCompleted(long duration, int blockSize) { + transferDuration.addAndGet(duration); incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_ACTIVE, -1); blocksActive.decrementAndGet(); blockUploadsCompleted.incrementAndGet(); @@ -1010,14 +1188,15 @@ void blockUploadCompleted(long duration, int blockSize) { * A final transfer completed event is still expected, so this * does not decrement the active block counter. */ - void blockUploadFailed(long duration, int blockSize) { + @Override + public void blockUploadFailed(long duration, int blockSize) { blockUploadsFailed.incrementAndGet(); } /** Intermediate report of bytes uploaded. */ - void bytesTransferred(long byteCount) { + @Override + public void bytesTransferred(long byteCount) { bytesUploaded.addAndGet(byteCount); - statistics.incrementBytesWritten(byteCount); bytesPendingUpload.addAndGet(-byteCount); incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING, -byteCount); } @@ -1026,7 +1205,8 @@ void bytesTransferred(long byteCount) { * Note exception in a multipart complete. * @param count count of exceptions */ - void exceptionInMultipartComplete(int count) { + @Override + public void exceptionInMultipartComplete(int count) { if (count > 0) { exceptionsInMultipartFinalize.addAndGet(count); } @@ -1035,7 +1215,8 @@ void exceptionInMultipartComplete(int count) { /** * Note an exception in a multipart abort. */ - void exceptionInMultipartAbort() { + @Override + public void exceptionInMultipartAbort() { exceptionsInMultipartFinalize.incrementAndGet(); } @@ -1043,6 +1224,7 @@ void exceptionInMultipartAbort() { * Get the number of bytes pending upload. * @return the number of bytes in the pending upload state. */ + @Override public long getBytesPendingUpload() { return bytesPendingUpload.get(); } @@ -1052,6 +1234,7 @@ public long getBytesPendingUpload() { * to be called at the end of the write. * @param size size in bytes */ + @Override public void commitUploaded(long size) { incrementCounter(COMMITTER_BYTES_UPLOADED, size); } @@ -1067,6 +1250,12 @@ public void close() { " as pending upload in {}", this); } mergeOutputStreamStatistics(this); + // and patch the FS statistics. + // provided the stream is closed in the worker thread, this will + // ensure that the thread-specific worker stats are updated. + if (filesystemStatistics != null) { + filesystemStatistics.incrementBytesWritten(bytesUploaded.get()); + } } long averageQueueTime() { @@ -1084,23 +1273,43 @@ long totalUploadDuration() { return queueDuration.get() + transferDuration.get(); } - public int blocksAllocated() { + @Override + public int getBlocksAllocated() { return blocksAllocated.get(); } - public int blocksReleased() { + @Override + public int getBlocksReleased() { return blocksReleased.get(); } /** - * Get counters of blocks actively allocated; my be inaccurate + * Get counters of blocks actively allocated; may be inaccurate * if the numbers change during the (non-synchronized) calculation. * @return the number of actively allocated blocks. */ - public int blocksActivelyAllocated() { + @Override + public int getBlocksActivelyAllocated() { return blocksAllocated.get() - blocksReleased.get(); } + /** + * Record bytes written. + * @param count number of bytes + */ + @Override + public void writeBytes(long count) { + bytesWritten.addAndGet(count); + } + + /** + * Get the current count of bytes written. + * @return the counter value. + */ + @Override + public long getBytesWritten() { + return bytesWritten.get(); + } @Override public String toString() { @@ -1113,11 +1322,13 @@ public String toString() { sb.append(", blockUploadsFailed=").append(blockUploadsFailed); sb.append(", bytesPendingUpload=").append(bytesPendingUpload); sb.append(", bytesUploaded=").append(bytesUploaded); + sb.append(", bytesWritten=").append(bytesWritten); sb.append(", blocksAllocated=").append(blocksAllocated); sb.append(", blocksReleased=").append(blocksReleased); - sb.append(", blocksActivelyAllocated=").append(blocksActivelyAllocated()); - sb.append(", exceptionsInMultipartFinalize=").append( - exceptionsInMultipartFinalize); + sb.append(", blocksActivelyAllocated=") + .append(getBlocksActivelyAllocated()); + sb.append(", exceptionsInMultipartFinalize=") + .append(exceptionsInMultipartFinalize); sb.append(", transferDuration=").append(transferDuration).append(" ms"); sb.append(", queueDuration=").append(queueDuration).append(" ms"); sb.append(", averageQueueTime=").append(averageQueueTime()).append(" ms"); @@ -1128,6 +1339,24 @@ public String toString() { sb.append('}'); return sb.toString(); } + + /** + * Create an IOStatistics instance which is dynamically updated. + * @return statistics + */ + @Override + public IOStatistics getIOStatistics() { + DynamicIOStatisticsBuilder builder = dynamicIOStatistics(); + + builder + .withAtomicLong(StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS, + blocksSubmitted) + .withAtomicLong(StreamStatisticNames.STREAM_WRITE_BYTES, + bytesWritten) + .withAtomicLong(StreamStatisticNames.STREAM_WRITE_EXCEPTIONS, + blockUploadsFailed); + return builder.build(); + } } /** @@ -1192,13 +1421,12 @@ public void entryAdded(final long durationNanos) { } /** - * Instrumentation exported to S3Guard Committers. + * Instrumentation exported to S3A Committers. */ - @InterfaceAudience.Private - @InterfaceStability.Unstable - public final class CommitterStatistics { + private final class CommitterStatisticsImpl implements CommitterStatistics { /** A commit has been created. */ + @Override public void commitCreated() { incrementCounter(COMMITTER_COMMITS_CREATED, 1); } @@ -1207,6 +1435,7 @@ public void commitCreated() { * Data has been uploaded to be committed in a subsequent operation. * @param size size in bytes */ + @Override public void commitUploaded(long size) { incrementCounter(COMMITTER_BYTES_UPLOADED, size); } @@ -1215,24 +1444,29 @@ public void commitUploaded(long size) { * A commit has been completed. * @param size size in bytes */ + @Override public void commitCompleted(long size) { incrementCounter(COMMITTER_COMMITS_COMPLETED, 1); incrementCounter(COMMITTER_BYTES_COMMITTED, size); } /** A commit has been aborted. */ + @Override public void commitAborted() { incrementCounter(COMMITTER_COMMITS_ABORTED, 1); } + @Override public void commitReverted() { incrementCounter(COMMITTER_COMMITS_REVERTED, 1); } + @Override public void commitFailed() { incrementCounter(COMMITTER_COMMITS_FAILED, 1); } + @Override public void taskCompleted(boolean success) { incrementCounter( success ? COMMITTER_TASKS_SUCCEEDED @@ -1240,6 +1474,7 @@ public void taskCompleted(boolean success) { 1); } + @Override public void jobCompleted(boolean success) { incrementCounter( success ? COMMITTER_JOBS_SUCCEEDED @@ -1253,20 +1488,20 @@ public void jobCompleted(boolean success) { * @return an instance of delegation token statistics */ public DelegationTokenStatistics newDelegationTokenStatistics() { - return new DelegationTokenStatistics(); + return new DelegationTokenStatisticsImpl(); } /** * Instrumentation exported to S3A Delegation Token support. */ - @InterfaceAudience.Private - @InterfaceStability.Unstable - public final class DelegationTokenStatistics { + private final class DelegationTokenStatisticsImpl implements + DelegationTokenStatistics { - private DelegationTokenStatistics() { + private DelegationTokenStatisticsImpl() { } /** A token has been issued. */ + @Override public void tokenIssued() { incrementCounter(DELEGATION_TOKENS_ISSUED, 1); } @@ -1366,4 +1601,67 @@ public Map getMap() { return map; } } + + public StatisticsFromAwsSdk newStatisticsFromAwsSdk() { + return new StatisticsFromAwsSdkImpl(this); + } + + /** + * 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. + */ + private static final class StatisticsFromAwsSdkImpl implements + StatisticsFromAwsSdk { + + private final CountersAndGauges countersAndGauges; + + private 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/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/S3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java index e0a1d780ccf5f..8aa328149129a 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 @@ -22,10 +22,12 @@ import java.net.URI; import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.metrics.RequestMetricCollector; import com.amazonaws.services.s3.AmazonS3; 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 +43,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 1d3d4758028c6..4c36c04f9f27a 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,39 +56,56 @@ 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"), @@ -106,51 +130,75 @@ public enum Statistic { "number of bytes queued for upload/being actively uploaded"), OBJECT_SELECT_REQUESTS("object_select_requests", "Count of S3 Select requests issued"), - STREAM_ABORTED("stream_aborted", + STREAM_ABORTED( + StreamStatisticNames.STREAM_READ_ABORTED, "Count of times the TCP stream was aborted"), - STREAM_BACKWARD_SEEK_OPERATIONS("stream_backward_seek_operations", + STREAM_BACKWARD_SEEK_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", + STREAM_CLOSED( + StreamStatisticNames.STREAM_READ_CLOSED, + "Count of times the TCP stream was closed"), + STREAM_CLOSE_OPERATIONS( + StreamStatisticNames.STREAM_READ_CLOSE_OPERATIONS, "Total count of times an attempt to close a data stream was made"), - STREAM_FORWARD_SEEK_OPERATIONS("stream_forward_seek_operations", + STREAM_FORWARD_SEEK_OPERATIONS( + StreamStatisticNames.STREAM_READ_SEEK_FORWARD_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", + STREAM_OPENED( + StreamStatisticNames.STREAM_READ_OPENED, + "Total count of times an input stream to object store data was opened"), + 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 streams"), - STREAM_READ_OPERATIONS("stream_read_operations", + STREAM_READ_OPERATIONS( + StreamStatisticNames.STREAM_READ_OPERATIONS, "Count of read() operations in streams"), - STREAM_READ_OPERATIONS_INCOMPLETE("stream_read_operations_incomplete", + STREAM_READ_OPERATIONS_INCOMPLETE( + StreamStatisticNames.STREAM_READ_OPERATIONS_INCOMPLETE, "Count of incomplete read() operations in streams"), - STREAM_READ_VERSION_MISMATCHES("stream_read_version_mismatches", + STREAM_READ_VERSION_MISMATCHES( + StreamStatisticNames.STREAM_READ_VERSION_MISMATCHES, "Count of version mismatches encountered while reading streams"), - STREAM_SEEK_BYTES_BACKWARDS("stream_bytes_backwards_on_seek", + STREAM_SEEK_BYTES_BACKWARDS( + StreamStatisticNames.STREAM_READ_SEEK_BYTES_BACKWARDS, "Count of bytes moved backwards during seek operations"), - STREAM_SEEK_BYTES_READ("stream_bytes_read", + STREAM_SEEK_BYTES_READ( + StreamStatisticNames.STREAM_READ_SEEK_BYTES_READ, "Count of bytes read during seek() in stream operations"), - STREAM_SEEK_BYTES_SKIPPED("stream_bytes_skipped_on_seek", + STREAM_SEEK_BYTES_SKIPPED( + StreamStatisticNames.STREAM_READ_SEEK_BYTES_SKIPPED, "Count of bytes skipped during forward seek operation"), - STREAM_SEEK_OPERATIONS("stream_seek_operations", + STREAM_SEEK_OPERATIONS( + StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS, "Number of seek operations during stream IO."), - STREAM_CLOSE_BYTES_READ("stream_bytes_read_in_close", + STREAM_CLOSE_BYTES_READ( + StreamStatisticNames.STREAM_READ_CLOSE_BYTES_READ, "Count of bytes read when closing streams during seek operations."), - STREAM_ABORT_BYTES_DISCARDED("stream_bytes_discarded_in_abort", + STREAM_ABORT_BYTES_DISCARDED( + StreamStatisticNames.STREAM_READ_BYTES_DISCARDED_ABORT, "Count of bytes discarded by aborting the stream"), - STREAM_WRITE_FAILURES("stream_write_failures", + STREAM_WRITE_FAILURES( + StreamStatisticNames.STREAM_WRITE_EXCEPTIONS, "Count of stream write failures reported"), - STREAM_WRITE_BLOCK_UPLOADS("stream_write_block_uploads", + 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", @@ -159,6 +207,10 @@ public enum Statistic { "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_BYTES( + StreamStatisticNames.STREAM_WRITE_BYTES, + "Count of bytes written to output stream" + + " (including all not yet uploaded"), STREAM_WRITE_QUEUE_DURATION("stream_write_queue_duration", "Total queue duration of all block uploads"), @@ -233,8 +285,15 @@ public enum Statistic { STORE_IO_THROTTLE_RATE("store_io_throttle_rate", "Rate of S3 request throttling"), - DELEGATION_TOKENS_ISSUED("delegation_tokens_issued", - "Number of delegation tokens issued"); + DELEGATION_TOKENS_ISSUED( + StoreStatisticNames.DELEGATION_TOKENS_ISSUED, + "Number of delegation tokens issued"), + + 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 ab53486d648b2..357cfebda92bb 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 { /** 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 5005436c8242a..6811d3d9b3825 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; @@ -134,9 +134,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 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..f8ab910970685 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,13 +45,13 @@ 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.io.IOUtils; import org.apache.hadoop.util.DurationInfo; @@ -81,7 +81,7 @@ public class CommitOperations { private final S3AFileSystem fs; /** Statistics. */ - private final S3AInstrumentation.CommitterStatistics statistics; + private final CommitterStatistics statistics; /** * Write operations for the destination fs. @@ -128,7 +128,7 @@ public String toString() { } /** @return statistics. */ - protected S3AInstrumentation.CommitterStatistics getStatistics() { + protected CommitterStatistics getStatistics() { return statistics; } 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/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..7c9189d2353d3 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; 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/StoreContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java index 88480db753515..7e0b4fe39cfc4 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 @@ -31,9 +31,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; @@ -84,7 +84,7 @@ public class StoreContext { private final Invoker invoker; /** Instrumentation and statistics. */ - private final S3AInstrumentation instrumentation; + private final S3AStatisticsContext instrumentation; private final S3AStorageStatistics storageStatistics; /** Seek policy. */ @@ -126,7 +126,7 @@ public StoreContext( final ListeningExecutorService executor, final int executorCapacity, final Invoker invoker, - final S3AInstrumentation instrumentation, + final S3AStatisticsContext instrumentation, final S3AStorageStatistics storageStatistics, final S3AInputPolicy inputPolicy, final ChangeDetectionPolicy changeDetectionPolicy, @@ -183,7 +183,7 @@ public Invoker getInvoker() { return invoker; } - public S3AInstrumentation getInstrumentation() { + public S3AStatisticsContext getInstrumentation() { return instrumentation; } 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..cf1ceea56ef93 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/AwsStatisticsCollector.java @@ -0,0 +1,126 @@ +/* + * 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 update our 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..428ec61805069 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/BlockOutputStreamStatistics.java @@ -0,0 +1,109 @@ +/* + * 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; + +import org.apache.hadoop.fs.statistics.IOStatisticsSource; + +/** + * Block output stream statistics. + */ +public interface BlockOutputStreamStatistics extends Closeable, + IOStatisticsSource { + + /** + * 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(); +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/BondedS3AStatisticsContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/BondedS3AStatisticsContext.java new file mode 100644 index 0000000000000..2c9fd0cfecc3f --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/BondedS3AStatisticsContext.java @@ -0,0 +1,209 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl.statistics; + +import javax.annotation.Nullable; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.s3a.S3AInstrumentation; +import org.apache.hadoop.fs.s3a.Statistic; +import org.apache.hadoop.fs.s3a.s3guard.MetastoreInstrumentation; + +/** + * An S3A statistics context which is bonded to a + * S3AInstrumentation instance -inevitably that of an S3AFileSystem + * instance. + *

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

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

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

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

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

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

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

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

+ * This class is here primarily to aid in testing, but it also allows for + * classes to require a non-empty statistics context in their constructor -yet + * still be instantiated without one bound to any filesystem. + */ +public final class EmptyS3AStatisticsContext implements S3AStatisticsContext { + + @Override + public MetastoreInstrumentation getS3GuardInstrumentation() { + return new MetastoreInstrumentationImpl(); + } + + @Override + public S3AInputStreamStatistics newInputStreamStatistics() { + return new EmptyInputStreamStatistics(); + } + + @Override + public CommitterStatistics newCommitterStatistics() { + return new EmptyCommitterStatistics(); + } + + @Override + public BlockOutputStreamStatistics newOutputStreamStatistics() { + return new EmptyBlockOutputStreamStatistics(); + } + + @Override + public DelegationTokenStatistics newDelegationTokenStatistics() { + return new EmptyDelegationTokenStatistics(); + } + + @Override + public StatisticsFromAwsSdk newStatisticsFromAwsSdk() { + return new EmptyStatisticsFromAwsSdk(); + } + + @Override + public void incrementCounter(final Statistic op, final long count) { + + } + + @Override + public void incrementGauge(final Statistic op, final long count) { + + } + + @Override + public void decrementGauge(final Statistic op, final long count) { + + } + + @Override + public void addValueToQuantiles(final Statistic op, final long value) { + + } + + /** + * Input Stream statistics callbacks. + */ + private static final class EmptyInputStreamStatistics + implements S3AInputStreamStatistics { + + @Override + public void seekBackwards(final long negativeOffset) { + + } + + @Override + public void seekForwards(final long skipped) { + + } + + @Override + public long streamOpened() { + return 0; + } + + @Override + public void streamClose(final boolean abortedConnection, + final long remainingInCurrentRequest) { + + } + + @Override + public void readException() { + + } + + @Override + public void bytesRead(final long bytes) { + + } + + @Override + public void readOperationStarted(final long pos, final long len) { + + } + + @Override + public void readFullyOperationStarted(final long pos, final long len) { + + } + + @Override + public void readOperationCompleted(final int requested, final int actual) { + + } + + @Override + public void close() { + + } + + @Override + public void inputPolicySet(final int updatedPolicy) { + + } + + @Override + public void merge(final boolean isClosed) { + + } + + /** + * Return an IO statistics instance. + * @return an empty IO statistics instance. + */ + @Override + public IOStatistics getIOStatistics() { + return emptyStatistics(); + } + + @Override + public long getCloseOperations() { + return 0; + } + + @Override + public long getClosed() { + return 0; + } + + @Override + public long getAborted() { + return 0; + } + + @Override + public long getForwardSeekOperations() { + return 0; + } + + @Override + public long getBackwardSeekOperations() { + return 0; + } + + @Override + public long getBytesRead() { + return 0; + } + + @Override + public long getBytesSkippedOnSeek() { + return 0; + } + + @Override + public long getBytesBackwardsOnSeek() { + return 0; + } + + @Override + public long getBytesReadInClose() { + return 0; + } + + @Override + public long getBytesDiscardedInAbort() { + return 0; + } + + @Override + public long getOpenOperations() { + return 0; + } + + @Override + public long getSeekOperations() { + return 0; + } + + @Override + public long getReadExceptions() { + return 0; + } + + @Override + public long getReadOperations() { + return 0; + } + + @Override + public long getReadFullyOperations() { + return 0; + } + + @Override + public long getReadsIncomplete() { + return 0; + } + + @Override + public long getPolicySetCount() { + return 0; + } + + @Override + public long getVersionMismatches() { + return 0; + } + + @Override + public long getInputPolicy() { + return 0; + } + + @Override + public Long getStatistic(final String name) { + return 0L; + } + + @Override + public ChangeTrackerStatistics getChangeTrackerStatistics() { + return new CountingChangeTracker(); + } + } + + private static final class EmptyCommitterStatistics + implements CommitterStatistics { + + @Override + public void commitCreated() { + + } + + @Override + public void commitUploaded(final long size) { + + } + + @Override + public void commitCompleted(final long size) { + + } + + @Override + public void commitAborted() { + + } + + @Override + public void commitReverted() { + + } + + @Override + public void commitFailed() { + + } + + @Override + public void taskCompleted(final boolean success) { + + } + + @Override + public void jobCompleted(final boolean success) { + + } + } + + private static final class EmptyBlockOutputStreamStatistics + implements BlockOutputStreamStatistics { + + @Override + public void blockUploadQueued(final int blockSize) { + + } + + @Override + public void blockUploadStarted(final long duration, final int blockSize) { + + } + + @Override + public void blockUploadCompleted(final long duration, final int blockSize) { + + } + + @Override + public void blockUploadFailed(final long duration, final int blockSize) { + + } + + @Override + public void bytesTransferred(final long byteCount) { + + } + + @Override + public void exceptionInMultipartComplete(final int count) { + + } + + @Override + public void exceptionInMultipartAbort() { + + } + + @Override + public long getBytesPendingUpload() { + return 0; + } + + @Override + public void commitUploaded(final long size) { + + } + + @Override + public int getBlocksAllocated() { + return 0; + } + + @Override + public int getBlocksReleased() { + return 0; + } + + @Override + public int getBlocksActivelyAllocated() { + return 0; + } + + @Override + public IOStatistics getIOStatistics() { + return emptyStatistics(); + } + + @Override + public void blockAllocated() { + + } + + @Override + public void blockReleased() { + + } + + @Override + public void writeBytes(final long count) { + } + + @Override + public long getBytesWritten() { + return 0; + } + + @Override + public void close() throws IOException { + + } + } + + private static final class EmptyDelegationTokenStatistics + implements DelegationTokenStatistics { + + @Override + public void tokenIssued() { + + } + } + + private static final class EmptyStatisticsFromAwsSdk implements + StatisticsFromAwsSdk { + + @Override + public void updateAwsRequestCount(final long longValue) { + + } + + @Override + public void updateAwsRetryCount(final long longValue) { + + } + + @Override + public void updateAwsThrottleExceptionsCount(final long longValue) { + + } + + @Override + public void noteAwsRequestTime(final Duration ofMillis) { + + } + + @Override + public void noteAwsClientExecuteTime(final Duration ofMillis) { + + } + + @Override + public void noteRequestMarshallTime(final Duration duration) { + + } + + @Override + public void noteRequestSigningTime(final Duration duration) { + + } + + @Override + public void noteResponseProcessingTime(final Duration duration) { + + } + } + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AInputStreamStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AInputStreamStatistics.java new file mode 100644 index 0000000000000..c47c8fb82d1e3 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AInputStreamStatistics.java @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl.statistics; + +import org.apache.hadoop.fs.statistics.IOStatisticsSource; + +/** + * Statistics updated by an input stream during its actual operation. + * It also contains getters for tests. + */ +public interface S3AInputStreamStatistics extends AutoCloseable, + IOStatisticsSource { + + /** + * Seek backwards, incrementing the seek and backward seek counters. + * @param negativeOffset how far was the seek? + * This is expected to be negative. + */ + void seekBackwards(long negativeOffset); + + /** + * Record a forward seek, adding a seek operation, a forward + * seek operation, and any bytes skipped. + * @param skipped number of bytes skipped by reading from the stream. + * If the seek was implemented by a close + reopen, set this to zero. + */ + void seekForwards(long skipped); + + /** + * The inner stream was opened. + * @return the previous count + */ + long streamOpened(); + + /** + * The inner stream was closed. + * @param abortedConnection flag to indicate the stream was aborted, + * rather than closed cleanly + * @param remainingInCurrentRequest the number of bytes remaining in + * the current request. + */ + void streamClose(boolean abortedConnection, + long remainingInCurrentRequest); + + /** + * An ignored stream read exception was received. + */ + void readException(); + + /** + * Increment the bytes read counter by the number of bytes; + * no-op if the argument is negative. + * @param bytes number of bytes read + */ + void bytesRead(long bytes); + + /** + * A {@code read(byte[] buf, int off, int len)} operation has started. + * @param pos starting position of the read + * @param len length of bytes to read + */ + void readOperationStarted(long pos, long len); + + /** + * A {@code PositionedRead.read(position, buffer, offset, length)} + * operation has just started. + * @param pos starting position of the read + * @param len length of bytes to read + */ + void readFullyOperationStarted(long pos, long len); + + /** + * A read operation has completed. + * @param requested number of requested bytes + * @param actual the actual number of bytes + */ + void readOperationCompleted(int requested, int actual); + + @Override + void close(); + + /** + * The input policy has been switched. + * @param updatedPolicy enum value of new policy. + */ + void inputPolicySet(int updatedPolicy); + + /** + * Get a reference to the change tracker statistics for this + * stream. + * @return a reference to the change tracker statistics + */ + ChangeTrackerStatistics getChangeTrackerStatistics(); + + /** + * Merge the statistics into the filesystem's instrumentation instance. + *

+ * Takes a diff between the current version of the stats and the + * version of the stats when merge was last called, and merges the diff + * into the instrumentation instance. Used to periodically merge the + * stats into the fs-wide stats + *

+ * Behavior is undefined if called on a closed instance. + */ + void merge(boolean isClosed); + + long getCloseOperations(); + + long getClosed(); + + long getAborted(); + + long getForwardSeekOperations(); + + long getBackwardSeekOperations(); + + long getBytesRead(); + + long getBytesSkippedOnSeek(); + + long getBytesBackwardsOnSeek(); + + long getBytesReadInClose(); + + long getBytesDiscardedInAbort(); + + long getOpenOperations(); + + long getSeekOperations(); + + long getReadExceptions(); + + long getReadOperations(); + + long getReadFullyOperations(); + + long getReadsIncomplete(); + + long getPolicySetCount(); + + long getVersionMismatches(); + + long getInputPolicy(); + + Long getStatistic(String name); +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AStatisticsContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AStatisticsContext.java new file mode 100644 index 0000000000000..96c081a5c585d --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AStatisticsContext.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl.statistics; + +import org.apache.hadoop.fs.s3a.s3guard.MetastoreInstrumentation; + +/** + * This is the statistics context for ongoing operations in S3A. + */ +public interface S3AStatisticsContext extends CountersAndGauges { + + /** + * Get the metastore instrumentation. + * @return an instance of the metastore statistics tracking. + */ + MetastoreInstrumentation getS3GuardInstrumentation(); + + /** + * Create a stream input statistics instance. + * @return the new instance + */ + S3AInputStreamStatistics newInputStreamStatistics(); + + /** + * Create a new instance of the committer statistics. + * @return a new committer statistics instance + */ + CommitterStatistics newCommitterStatistics(); + + /** + * Create a stream output statistics instance. + * @return the new instance + */ + BlockOutputStreamStatistics newOutputStreamStatistics(); + + /** + * Create a delegation token statistics instance. + * @return an instance of delegation token statistics + */ + DelegationTokenStatistics newDelegationTokenStatistics(); + + /** + * Create a StatisticsFromAwsSdk instance. + * @return an instance of StatisticsFromAwsSdk + */ + StatisticsFromAwsSdk newStatisticsFromAwsSdk(); + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/StatisticsFromAwsSdk.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/StatisticsFromAwsSdk.java new file mode 100644 index 0000000000000..d04de7a0d5603 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/StatisticsFromAwsSdk.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl.statistics; + +import java.time.Duration; + +/** + * interface to receive statistics events from the AWS SDK + * by way of {@link AwsStatisticsCollector}. + */ +public interface StatisticsFromAwsSdk { + + /** + * Record a number of AWS requests. + * @param count number of events. + */ + void updateAwsRequestCount(long count); + + /** + * Record a number of AWS request retries. + * @param count number of events. + */ + void updateAwsRetryCount(long count); + + /** + * Record a number of throttle exceptions received. + * @param count number of events. + */ + void updateAwsThrottleExceptionsCount(long count); + + /** + * Record how long a request took overall. + * @param duration duration of operation. + */ + void noteAwsRequestTime(Duration duration); + + /** + * Record how long a request took to execute on the + * client. + * @param duration duration of operation. + */ + void noteAwsClientExecuteTime(Duration duration); + + /** + * Record how long a request took to marshall into + * XML. + * @param duration duration of operation. + */ + void noteRequestMarshallTime(Duration duration); + + /** + * Record how long a request took to sign, including + * any calls to EC2 credential endpoints. + * @param duration duration of operation. + */ + void noteRequestSigningTime(Duration duration); + + /** + * Record how long it took to process the response. + * @param duration duration of operation. + */ + void noteResponseProcessingTime(Duration duration); +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/package-info.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/package-info.java new file mode 100644 index 0000000000000..5d9d3b4d8d81a --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/package-info.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Statistics collection for the S3A connector. + */ + +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.fs.s3a.impl.statistics; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectInputStream.java index f4bd8d11708ef..b99ddb830c785 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectInputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectInputStream.java @@ -39,9 +39,9 @@ import org.apache.hadoop.fs.FSInputStream; import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.s3a.Retries; -import org.apache.hadoop.fs.s3a.S3AInstrumentation; import org.apache.hadoop.fs.s3a.S3AReadOpContext; import org.apache.hadoop.fs.s3a.S3ObjectAttributes; +import org.apache.hadoop.fs.s3a.impl.statistics.S3AInputStreamStatistics; import org.apache.hadoop.io.IOUtils; import static com.google.common.base.Preconditions.checkNotNull; @@ -103,7 +103,7 @@ public class SelectInputStream extends FSInputStream implements private final S3AReadOpContext readContext; - private final S3AInstrumentation.InputStreamStatistics streamStatistics; + private final S3AInputStreamStatistics streamStatistics; private long readahead; @@ -130,7 +130,7 @@ public SelectInputStream( this.uri = "s3a://" + this.bucket + "/" + this.key; this.readContext = readContext; this.readahead = readContext.getReadahead(); - this.streamStatistics = readContext.getInstrumentation() + this.streamStatistics = readContext.getS3AStatisticsContext() .newInputStreamStatistics(); SelectRecordsInputStream stream = once( "S3 Select", @@ -428,7 +428,7 @@ private void incrementBytesRead(long bytesRead) { */ @InterfaceAudience.Private @InterfaceStability.Unstable - public S3AInstrumentation.InputStreamStatistics getS3AStreamStatistics() { + public S3AInputStreamStatistics getS3AStreamStatistics() { return streamStatistics; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java index afd3ec2bd3473..3e2786dbe475f 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java @@ -22,6 +22,7 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.s3a.impl.statistics.BlockOutputStreamStatistics; import org.apache.hadoop.io.IOUtils; import org.junit.BeforeClass; @@ -94,16 +95,16 @@ public void testBlocksClosed() throws Throwable { Path dest = path("testBlocksClosed"); describe(" testBlocksClosed"); FSDataOutputStream stream = getFileSystem().create(dest, true); - S3AInstrumentation.OutputStreamStatistics statistics + BlockOutputStreamStatistics statistics = S3ATestUtils.getOutputStreamStatistics(stream); byte[] data = ContractTestUtils.dataset(16, 'a', 26); stream.write(data); LOG.info("closing output stream"); stream.close(); assertEquals("total allocated blocks in " + statistics, - 1, statistics.blocksAllocated()); + 1, statistics.getBlocksAllocated()); assertEquals("actively allocated blocks in " + statistics, - 0, statistics.blocksActivelyAllocated()); + 0, statistics.getBlocksActivelyAllocated()); LOG.info("end of test case"); } @@ -129,7 +130,7 @@ private void markAndResetDatablock(S3ADataBlocks.BlockFactory factory) throws Exception { S3AInstrumentation instrumentation = new S3AInstrumentation(new URI("s3a://example")); - S3AInstrumentation.OutputStreamStatistics outstats + BlockOutputStreamStatistics outstats = instrumentation.newOutputStreamStatistics(null); S3ADataBlocks.DataBlock block = factory.create(1, BLOCK_SIZE, outstats); block.write(dataset, 0, dataset.length); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AUnbuffer.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AUnbuffer.java index 2ba3fd7a65cde..e95a6d1f4bd62 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AUnbuffer.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AUnbuffer.java @@ -71,7 +71,7 @@ public void testUnbuffer() throws IOException { /** * Test that calling {@link S3AInputStream#unbuffer()} merges a stream's - * {@link org.apache.hadoop.fs.s3a.S3AInstrumentation.InputStreamStatistics} + * {@code InputStreamStatistics} * into the {@link S3AFileSystem}'s {@link S3AInstrumentation} instance. */ @Test @@ -108,7 +108,7 @@ public void testUnbufferStreamStatistics() throws IOException { // Validate that the input stream stats are correct when the file is closed assertEquals("S3AInputStream statistics were not updated properly", 12, ((S3AInputStream) inputStream.getWrappedStream()) - .getS3AStreamStatistics().bytesRead); + .getS3AStreamStatistics().getBytesRead()); } private boolean isObjectStreamOpen(FSDataInputStream inputStream) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java index e30269298111d..0bddd264a6c15 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java @@ -39,11 +39,12 @@ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets; import org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase; +import org.apache.hadoop.fs.s3a.impl.statistics.CommitterStatistics; +import org.apache.hadoop.fs.s3a.impl.statistics.EmptyS3AStatisticsContext; import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; import org.apache.hadoop.util.Progressable; import static com.google.common.base.Preconditions.checkNotNull; -import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; /** * Relays FS calls to the mocked FS, allows for some extra logging with @@ -83,8 +84,6 @@ public class MockS3AFileSystem extends S3AFileSystem { * mock FS. */ private int logEvents = LOG_NAME; - private final S3AInstrumentation instrumentation = - new S3AInstrumentation(FS_URI); private Configuration conf; private WriteOperationHelper writeHelper; @@ -146,12 +145,12 @@ public Path qualify(final Path path) { public void initialize(URI name, Configuration originalConf) throws IOException { conf = originalConf; - writeHelper = new WriteOperationHelper(this, conf); + writeHelper = new WriteOperationHelper(this, conf, + new EmptyS3AStatisticsContext()); } @Override public void close() { - cleanupWithLogger(LOG, instrumentation); } @Override @@ -359,12 +358,56 @@ public String toString() { } @Override - public S3AInstrumentation.CommitterStatistics newCommitterStatistics() { - return instrumentation.newCommitterStatistics(); + public CommitterStatistics newCommitterStatistics() { + return new StubCommitterStatistics(); } @Override public void operationRetried(Exception ex) { /** no-op */ } + + private final class StubCommitterStatistics implements CommitterStatistics { + + @Override + public void commitCreated() { + + } + + @Override + public void commitUploaded(final long size) { + + } + + @Override + public void commitCompleted(final long size) { + + } + + @Override + public void commitAborted() { + + } + + @Override + public void commitReverted() { + + } + + @Override + public void commitFailed() { + + } + + @Override + public void taskCompleted(final boolean success) { + + } + + @Override + public void jobCompleted(final boolean success) { + + } + } + } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java index 4644cf24764ae..86a1675fae2f5 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java @@ -28,6 +28,8 @@ import com.amazonaws.services.s3.model.MultipartUploadListing; import com.amazonaws.services.s3.model.Region; +import org.apache.hadoop.fs.s3a.impl.statistics.StatisticsFromAwsSdk; + /** * An {@link S3ClientFactory} that returns Mockito mocks of the {@link AmazonS3} * interface suitable for unit testing. @@ -38,7 +40,8 @@ public class MockS3ClientFactory implements S3ClientFactory { public AmazonS3 createS3Client(URI name, final String bucket, final AWSCredentialsProvider credentialSet, - final String userAgentSuffix) { + final String userAgentSuffix, + final StatisticsFromAwsSdk statisticsFromAwsSdks) { AmazonS3 s3 = mock(AmazonS3.class); when(s3.doesBucketExist(bucket)).thenReturn(true); when(s3.doesBucketExistV2(bucket)).thenReturn(true); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java index aa5979dbf751e..79e82de4537ed 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java @@ -34,8 +34,8 @@ import org.apache.hadoop.fs.s3a.auth.MarshalledCredentialBinding; import org.apache.hadoop.fs.s3a.auth.MarshalledCredentials; import org.apache.hadoop.fs.s3a.commit.CommitConstants; - import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum; +import org.apache.hadoop.fs.s3a.impl.statistics.BlockOutputStreamStatistics; import org.apache.hadoop.fs.s3a.s3guard.MetadataStore; import org.apache.hadoop.fs.s3a.s3guard.MetadataStoreCapabilities; import org.apache.hadoop.fs.s3native.S3xLoginHelper; @@ -1162,7 +1162,7 @@ public static void assume(String message, boolean condition) { * @param out output stream * @return the (active) stats of the write */ - public static S3AInstrumentation.OutputStreamStatistics + public static BlockOutputStreamStatistics getOutputStreamStatistics(FSDataOutputStream out) { S3ABlockOutputStream blockOutputStream = (S3ABlockOutputStream) out.getWrappedStream(); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java index 60904d7ae83f1..c96a137ef1834 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java @@ -21,6 +21,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.s3a.commit.PutTracker; +import org.apache.hadoop.fs.s3a.impl.statistics.EmptyS3AStatisticsContext; import org.apache.hadoop.util.Progressable; import org.junit.Before; import org.junit.Test; @@ -50,11 +51,10 @@ public void setUp() throws Exception { S3ADataBlocks.BlockFactory blockFactory = mock(S3ADataBlocks.BlockFactory.class); long blockSize = Constants.DEFAULT_MULTIPART_SIZE; - S3AInstrumentation.OutputStreamStatistics statistics = null; WriteOperationHelper oHelper = mock(WriteOperationHelper.class); PutTracker putTracker = mock(PutTracker.class); stream = spy(new S3ABlockOutputStream(fs, "", executorService, - progressable, blockSize, blockFactory, statistics, oHelper, + progressable, blockSize, blockFactory, null, oHelper, putTracker)); } @@ -70,7 +70,8 @@ public void testWriteOperationHelperPartLimits() throws Throwable { S3AFileSystem s3a = mock(S3AFileSystem.class); when(s3a.getBucket()).thenReturn("bucket"); WriteOperationHelper woh = new WriteOperationHelper(s3a, - new Configuration()); + new Configuration(), + new EmptyS3AStatisticsContext()); ByteArrayInputStream inputStream = new ByteArrayInputStream( "a".getBytes()); // first one works diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java index 3d7cdfc08dec4..41ff4c2fbf3fd 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java @@ -18,8 +18,6 @@ package org.apache.hadoop.fs.s3a; -import java.util.concurrent.atomic.AtomicLong; - import com.amazonaws.AmazonServiceException; import com.amazonaws.SdkBaseException; import com.amazonaws.services.s3.Headers; @@ -36,6 +34,7 @@ import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; import org.apache.hadoop.fs.s3a.impl.ChangeTracker; +import org.apache.hadoop.fs.s3a.impl.statistics.CountingChangeTracker; import org.apache.hadoop.test.HadoopTestBase; import static org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy.CHANGE_DETECTED; @@ -359,7 +358,7 @@ protected void assertTrackerMismatchCount( final ChangeTracker tracker, final int expectedCount) { assertEquals("counter in tracker " + tracker, - expectedCount, tracker.getVersionMismatches().get()); + expectedCount, tracker.getVersionMismatches()); } /** @@ -386,7 +385,7 @@ protected ChangeTracker newTracker(final ChangeDetectionPolicy.Mode mode, source, requireVersion); ChangeTracker tracker = new ChangeTracker(URI, policy, - new AtomicLong(0), objectAttributes); + new CountingChangeTracker(), objectAttributes); if (objectAttributes.getVersionId() == null && objectAttributes.getETag() == null) { assertFalse("Tracker should not have applied constraints " + tracker, diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationInFileystem.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationInFileystem.java index f6c908ea2f7e6..1910ed91fe98d 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationInFileystem.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationInFileystem.java @@ -44,6 +44,8 @@ import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.Statistic; +import org.apache.hadoop.fs.s3a.impl.statistics.EmptyS3AStatisticsContext; +import org.apache.hadoop.fs.s3a.impl.statistics.StatisticsFromAwsSdk; import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.security.TokenCache; @@ -70,6 +72,7 @@ import static org.apache.hadoop.fs.s3a.auth.delegation.MiniKerberizedHadoopCluster.ALICE; import static org.apache.hadoop.fs.s3a.auth.delegation.MiniKerberizedHadoopCluster.assertSecurityEnabled; import static org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens.lookupS3ADelegationToken; +import static org.apache.hadoop.fs.s3a.impl.InternalConstants.AWS_SDK_METRICS_ENABLED; import static org.apache.hadoop.test.LambdaTestUtils.doAs; import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.hamcrest.Matchers.containsString; @@ -305,6 +308,9 @@ public void testDelegatedFileSystem() throws Throwable { describe("Delegation tokens can be passed to a new filesystem;" + " if role restricted, permissions are tightened."); S3AFileSystem fs = getFileSystem(); + // force a probe of the remote FS to make sure its endpoint is valid + // (this always hits S3, even when S3Guard is enabled) + fs.getObjectMetadata(new Path("/")); readLandsatMetadata(fs); URI uri = fs.getUri(); @@ -561,8 +567,13 @@ protected ObjectMetadata readLandsatMetadata(final S3AFileSystem delegatedFS) conf.set(ENDPOINT, ""); factory.setConf(conf); String host = landsat.getHost(); + StatisticsFromAwsSdk awsStats = null; + if (AWS_SDK_METRICS_ENABLED) { + awsStats = new EmptyS3AStatisticsContext() + .newStatisticsFromAwsSdk(); + } AmazonS3 s3 = factory.createS3Client(landsat, host, testing, - "ITestSessionDelegationInFileystem"); + "ITestSessionDelegationInFileystem", awsStats); return Invoker.once("HEAD", host, () -> s3.getObjectMetadata(host, landsat.getPath().substring(1))); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNetworkBinding.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNetworkBinding.java new file mode 100644 index 0000000000000..10fe339174fc5 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNetworkBinding.java @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl; + +import com.amazonaws.ClientConfiguration; +import com.amazonaws.client.builder.AwsClientBuilder; +import org.junit.Ignore; +import org.junit.Test; + +import org.apache.hadoop.test.AbstractHadoopTestBase; + +import static org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.createEndpointConfiguration; +import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.fixBucketRegion; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Unit tests related to the {@link NetworkBinding} class. + */ +public class TestNetworkBinding extends AbstractHadoopTestBase { + + private static final String US_EAST_1 = "us-east-1"; + + private static final String US_WEST_2 = "us-west-2"; + + @Test + public void testUSEast() { + assertRegionFixup(US_EAST_1, US_EAST_1); + } + + @Test + public void testUSWest() { + assertRegionFixup(US_WEST_2, US_WEST_2); + } + + @Test + public void testRegionUStoUSEast() { + assertRegionFixup("US", US_EAST_1); + } + + @Test + public void testRegionNullToUSEast() { + assertRegionFixup(null, US_EAST_1); + } + + private static void assertRegionFixup(String region, String expected) { + assertThat(fixBucketRegion(region)) + .describedAs("Fixup of %s", region) + .isEqualTo(expected); + } + + @Test + public void testNull() throws Throwable { + expectEndpoint("", true, "unused"); + } + + @Test + @Ignore("disabled until endpoint logic works for S3 client builder API") + public void testUSEastEndpoint() throws Throwable { + expectEndpoint(US_EAST_1, false, US_EAST_1); + } + + @Test + @Ignore("disabled until endpoint logic works for S3 client builder API") + public void testUSWestEndpoint() throws Throwable { + expectEndpoint(US_WEST_2, false, US_WEST_2); + } + + public void expectEndpoint(final String src, + final boolean expectNull, + final String expectRegion) { + AwsClientBuilder.EndpointConfiguration epr = + createEndpointConfiguration(src, new ClientConfiguration()); + String eprStr = epr == null + ? "(empty)" + : ("(" + epr.getServiceEndpoint() + " " + epr.getSigningRegion()); + if (expectNull) { + assertThat(epr) + .describedAs("Endpoint configuration of %s =", + src, eprStr) + .isNull(); + } else { + assertThat(epr) + .describedAs("Endpoint configuration of %s =", + src, eprStr) + .hasFieldOrPropertyWithValue("serviceEndpoint", src) + .hasFieldOrPropertyWithValue("signingRegion", expectRegion); + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNeworkBinding.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNeworkBinding.java deleted file mode 100644 index eebc3bfdf2fe4..0000000000000 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNeworkBinding.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.s3a.impl; - -import org.assertj.core.api.Assertions; -import org.junit.Test; - -import org.apache.hadoop.test.HadoopTestBase; - -import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.fixBucketRegion; - -/** - * Unit tests related to the {@link NetworkBinding} class. - */ -public class TestNeworkBinding extends HadoopTestBase { - - private static final String US_EAST_1 = "us-east-1"; - private static final String US_WEST_2 = "us-west-2"; - - @Test - public void testUSEast() { - assertRegionFixup(US_EAST_1, US_EAST_1); - } - - @Test - public void testUSWest() { - assertRegionFixup(US_WEST_2, US_WEST_2); - } - - @Test - public void testRegionUStoUSEast() { - assertRegionFixup("US", US_EAST_1); - } - - @Test - public void testRegionNullToUSEast() { - assertRegionFixup(null, US_EAST_1); - } - - private static void assertRegionFixup(String region, String expected) { - Assertions.assertThat(fixBucketRegion(region)) - .describedAs("Fixup of %s", region) - .isEqualTo(expected); - } -} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java index 244d2eed324c7..bfac41a48a48e 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java @@ -48,8 +48,8 @@ import org.apache.hadoop.fs.s3a.Invoker; import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.S3AInputPolicy; -import org.apache.hadoop.fs.s3a.S3AInstrumentation; import org.apache.hadoop.fs.s3a.S3AStorageStatistics; +import org.apache.hadoop.fs.s3a.impl.statistics.EmptyS3AStatisticsContext; import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; import org.apache.hadoop.fs.s3a.s3guard.DirListingMetadata; import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider; @@ -216,7 +216,7 @@ private StoreContext createMockStoreContext(boolean multiDelete, "s3a-transfer-shared"), Constants.DEFAULT_EXECUTOR_CAPACITY, new Invoker(RetryPolicies.TRY_ONCE_THEN_FAIL, Invoker.LOG_EVENT), - new S3AInstrumentation(name), + new EmptyS3AStatisticsContext(), new S3AStorageStatistics(), S3AInputPolicy.Normal, ChangeDetectionPolicy.createPolicy(ChangeDetectionPolicy.Mode.None, diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/statistics/ITestS3AContractStreamIOStatistics.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/statistics/ITestS3AContractStreamIOStatistics.java new file mode 100644 index 0000000000000..697dd195e8b22 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/statistics/ITestS3AContractStreamIOStatistics.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl.statistics; + +import java.util.Arrays; +import java.util.List; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractStreamIOStatisticsTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.fs.contract.s3a.S3AContract; +import org.apache.hadoop.fs.statistics.StreamStatisticNames; + +import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeEnableS3Guard; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.*; + +/** + * Test the S3A Streams IOStatistics support. + */ +public class ITestS3AContractStreamIOStatistics extends + AbstractContractStreamIOStatisticsTest { + + @Override + protected Configuration createConfiguration() { + Configuration conf = super.createConfiguration(); + // patch in S3Guard options + maybeEnableS3Guard(conf); + return conf; + } + + @Override + protected AbstractFSContract createContract(Configuration conf) { + return new S3AContract(conf); + } + + /** + * Keys which the input stream must support. + * @return a list of keys + */ + public List 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/scale/AbstractSTestS3AHugeFiles.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java index 315d1fe7285be..884b15d382a20 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.demandStringify; +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics; +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.snapshot; /** * 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,19 @@ public void test_010_CreateHugeFile() throws IOException { logFSState(); bandwidth(timer, filesize); LOG.info("Statistics after stream closed: {}", streamStatistics); + IOStatistics iostats = snapshot(retrieveIOStatistics(getFileSystem())); + LOG.info("IOStatistics after upload: {}", + demandStringify(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 +250,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/ITestS3AInputStreamPerformance.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java index efd96c4e7387e..e444b463af876 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,7 +27,7 @@ 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.io.IOUtils; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.compress.CompressionCodec; @@ -58,7 +58,7 @@ 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; @@ -187,7 +187,7 @@ protected void assertStreamOpenedExactlyOnce() { */ private void assertOpenOperationCount(long expected) { assertEquals("open operations in\n" + in, - expected, streamStatistics.openOperations); + expected, streamStatistics.getOpenOperations()); } /** @@ -295,7 +295,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 @@ -391,8 +391,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 +419,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 +427,12 @@ 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()); } /** @@ -466,8 +467,8 @@ 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(); return timer; } @@ -525,7 +526,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 eb80bc579f6c1..1980f2ba3d51d 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; @@ -162,7 +163,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..378ce4a69e29f 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, @@ -588,13 +588,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 +609,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/resources/log4j.properties b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties index 6e20fbcda7efd..8228e3151907e 100644 --- a/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties +++ b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties @@ -75,3 +75,6 @@ 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