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..2c7d2805fc8fa 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/StorageStatistics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageStatistics.java index 74631b5695537..4be56a53678ab 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,6 +28,7 @@ * instance. */ @InterfaceAudience.Public +@InterfaceStability.Stable public abstract class StorageStatistics { /** @@ -43,39 +45,40 @@ public abstract class StorageStatistics { * underscores. *
  • the value of the constants are lowercase of the constant names.
  • * + * 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/IOStatistics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatistics.java new file mode 100644 index 0000000000000..65dd0d89294fa --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatistics.java @@ -0,0 +1,151 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.InterfaceStability; + +/** + * IO Statistics. + *

    + * These are low-cost per-instance statistics provided by any IO components. + *

    + * 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 attributes of an instance can be probed for with + * {@link #hasAttribute(Attributes)}. + *
    2. + *
    3. + * The set of statistic keys SHOULD remain unchanged, and MUST NOT + * ever remove keys. + *
    4. + * The statistics MAY BE dynamic: every call to {@code iterator()} + * MAY return a current/recent set of statistics. + * This + * + *
    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. + *
    17. + * If the instance declares that it has the attribute {@link Attributes#Snapshotted}, + * then it will take a snapshot of the attribute values in the call {@link #snapshot()}. + * These values MUST NOT change until a subsequent snapshot() operation. + *
    18. + *
    19. + * A snapshot MAY NOT be consistent, i.e. during the snapshot operation + * the underlying values may change. + *
    20. + * + *
    + */ +@InterfaceStability.Unstable +public interface IOStatistics extends Iterable> { + + /** + * Get the value of a statistic. + * + * @return The value of the statistic, or null if not tracked. + */ + Long getStatistic(String key); + + /** + * Return true if a statistic is being tracked. + * + * @return True only if the statistic is being tracked. + */ + boolean isTracked(String key); + + /** + * Probe for an attribute of this statistics set. + * @return true if the source has the attribute. + */ + default boolean hasAttribute(Attributes attr) { + return false; + } + + /** + * Create a snapshot; no-op if not supported. + * @return true if this call had any effect + */ + default boolean snapshot() { + return false; + } + + /** + * Get the set of keys. + * No guarantees are made about the mutability/immutability + * of this set. + * @return the set of keys. + */ + Set keys(); + + /** + * Possible attributes of the statistics. + * This is very limited right now + */ + enum Attributes { + /** The attributes never update. */ + Static, + + /** The statistics are dynamic: when you re-read a value it may change. */ + Dynamic, + + /** + * The statistics are actually snapshots, updated when you call snapshot(), + * or iterator(); + */ + Snapshotted + } +} 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..e8f69871ddcac --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsLogging.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.statistics; + +import javax.annotation.Nullable; +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics; + +/** + * Utility operations to work with IO Statistics, especially log them. + */ +public class IOStatisticsLogging { + + private static final Logger LOG = + LoggerFactory.getLogger(IOStatisticsLogging.class); + + /** + * Convert IOStatistics to a string form. + * @param statistics A statistics instance. + * @return string value or the emtpy string if null + */ + private static String iostatisticsToString( + @Nullable final IOStatistics statistics) { + if (statistics != null) { + StringBuilder sb = new StringBuilder(" {"); + for (Map.Entry entry : statistics) { + sb.append("{") + .append(entry.getKey()) + .append("=") + .append(entry.getValue()) + .append("} "); + } + sb.append('}'); + return sb.toString(); + } else { + return null; + } + } + + /** + * Extract the statistics from a source. + * Exceptions are caught and downgraded to debug logging. + * @param source source of statistics. + * @return a string for logging. + */ + public static String iostatisticsSourceToString(final IOStatisticsSource 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 + * retrieves the latest statistics instance and re-evaluates it. + */ + public static final class SourceToString { + + private final String origin; + + private final IOStatisticsSource source; + + public SourceToString(String origin, IOStatisticsSource source) { + this.origin = origin; + this.source = source; + } + + @Override + public String toString() { + return source != null + ? ("Statistics of " + origin + " " + iostatisticsSourceToString(source)) + : ""; + } + } + + /** + * Stringifier of statistics: low cost to instantiate and every + * toString/logging will re-evaluate the statistics. + */ + public static final class StatisticsToString { + + private final String origin; + + private final IOStatistics statistics; + + /** + * Constructor. + * @param origin source (for message) + * @param statistics statistics + */ + public StatisticsToString(String origin, IOStatistics statistics) { + this.origin = origin; + this.statistics = statistics; + } + + @Override + public String toString() { + return statistics != null + ? ("Statistics of " + origin + " " + iostatisticsToString(statistics)) + : ""; + } + } +} 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..9553b84a97415 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSource.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.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..1c6682f312021 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSupport.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics; + +import org.apache.hadoop.fs.statistics.impl.IOStatisticsImplementationHelper; + +/** + * Support for working with statistics. + */ +public final class IOStatisticsSupport { + + private IOStatisticsSupport() { + } + + /** + * Wrap a (dynamic) source with a snapshot IOStatistics instance. + * @param source source + * @return a wrapped instance. + */ + public static IOStatistics takeSnapshot(IOStatistics source) { + return IOStatisticsImplementationHelper.wrapWithSnapshot(source); + } + + /** + * 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; + } +} 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..89d72a9bdeb7e --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.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; + +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: + * + */ +@InterfaceStability.Unstable +public class StoreStatisticNames { + + /** {@value}. */ + public static final String OP_APPEND = "op_append"; + + /** {@value}. */ + public static final String OP_COPY_FROM_LOCAL_FILE = + "op_copy_from_local_file"; + + /** {@value}. */ + public static final String OP_CREATE = "op_create"; + + /** {@value}. */ + public static final String OP_CREATE_NON_RECURSIVE = "op_create_non_recursive"; + + /** {@value}. */ + public static final String OP_DELETE = "op_delete"; + + /** {@value}. */ + public static final String OP_EXISTS = "op_exists"; + + /** {@value}. */ + public static final String OP_GET_CONTENT_SUMMARY = "op_get_content_summary"; + + /** {@value}. */ + public static final String OP_GET_DELEGATION_TOKEN = "op_get_delegation_token"; + + /** {@value}. */ + public static final String OP_GET_FILE_CHECKSUM = "op_get_file_checksum"; + + /** {@value}. */ + public static final String OP_GET_FILE_STATUS = "op_get_file_status"; + + /** {@value}. */ + public static final String OP_GET_STATUS = "op_get_status"; + + /** {@value}. */ + public static final String OP_GLOB_STATUS = "op_glob_status"; + + /** {@value}. */ + public static final String OP_IS_FILE = "op_is_file"; + + /** {@value}. */ + public static final String OP_IS_DIRECTORY = "op_is_directory"; + + /** {@value}. */ + public static final String OP_LIST_FILES = "op_list_files"; + + /** {@value}. */ + public static final String OP_LIST_LOCATED_STATUS = "op_list_located_status"; + + /** {@value}. */ + public static final String OP_LIST_STATUS = "op_list_status"; + + /** {@value}. */ + public static final String OP_MKDIRS = "op_mkdirs"; + + /** {@value}. */ + public static final String OP_MODIFY_ACL_ENTRIES = "op_modify_acl_entries"; + + /** {@value}. */ + public static final String OP_OPEN = "op_open"; + + /** {@value}. */ + public static final String OP_REMOVE_ACL = "op_remove_acl"; + + /** {@value}. */ + public static final String OP_REMOVE_ACL_ENTRIES = "op_remove_acl_entries"; + + /** {@value}. */ + public static final String OP_REMOVE_DEFAULT_ACL = "op_remove_default_acl"; + + /** {@value}. */ + public static final String OP_RENAME = "op_rename"; + + /** {@value}. */ + public static final String OP_SET_ACL = "op_set_acl"; + + /** {@value}. */ + public static final String OP_SET_OWNER = "op_set_owner"; + + /** {@value}. */ + public static final String OP_SET_PERMISSION = "op_set_permission"; + + /** {@value}. */ + public static final String OP_SET_TIMES = "op_set_times"; + + /** {@value}. */ + public static final String OP_TRUNCATE = "op_truncate"; + + /** {@value}. */ + public static final String DELEGATION_TOKENS_ISSUED + = "delegation_tokens_issued"; + + /** Requests throttled and retried: {@value}. */ + public static final String STORE_IO_THROTTLED + = "store_io_throttled"; + + /** Requests made of a store: {@value}. */ + public static final String STORE_IO_REQUEST + = "store_io_request"; + + /** + * IO retried: {@value}. + */ + public static final String STORE_IO_RETRY + = "store_io_retry"; + +} 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..5c05e81d796a7 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StreamStatisticNames.java @@ -0,0 +1,155 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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: + * + */ +@InterfaceStability.Unstable +public class StreamStatisticNames { + + /** {@value}. */ + public static final String STREAM_ABORTED = "stream_aborted"; + + /** {@value}. */ + public static final String STREAM_BYTES_DISCARDED_ABORT + = "stream_bytes_discarded_in_abort"; + + /** {@value}. */ + public static final String STREAM_BYTES_DISCARDED_SEEK + = "stream_bytes_discarded_in_seek"; + + /** {@value}. */ + public static final String STREAM_CLOSED = "stream_closed"; + + /** {@value}. */ + public static final String STREAM_CLOSE_BYTES_READ + = "stream_bytes_read_in_close"; + + /** {@value}. */ + public static final String STREAM_CLOSE_OPERATIONS + = "stream_close_operations"; + + /** {@value}. */ + public static final String STREAM_OPENED = "stream_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_bytes_backwards_on_seek"; + + /** {@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_bytes_skipped_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_WRITE_FAILURES = + "stream_write_failures"; + + /** {@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/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..96bc4c41bfd7e --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/DynamicIOStatistics.java @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.ToLongFunction; + +import org.apache.hadoop.fs.statistics.IOStatistics; + +/** + * 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; the returned a value is fixed. + */ +final class DynamicIOStatistics implements IOStatistics { + + /** + * Treemaps sort their insertions so the iterator is ordered. + */ + private final Map> evaluators + = new TreeMap<>(); + + DynamicIOStatistics() { + } + + /** + * add a mapping of a key to an evaluator. + * @param key the key + * @param eval the evaluator + */ + void add(String key, ToLongFunction 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 Long getStatistic(final String key) { + ToLongFunction fn = evaluators.get(key); + return fn != null + ? fn.applyAsLong(key) + : null; + } + + @Override + public boolean isTracked(final String key) { + return evaluators.containsKey(key); + } + + @Override + public Iterator> iterator() { + return new DynamicIterator(); + } + + @Override + public boolean hasAttribute(final Attributes attr) { + return Attributes.Dynamic == attr; + } + + @Override + public Set keys() { + return evaluators.keySet(); + } + + /** + * Iterator over the entries, evaluating each one in the next() call. + */ + private final class DynamicIterator + implements Iterator> { + + private final Iterator>> + iterator = evaluators.entrySet().iterator(); + + private DynamicIterator() { + } + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public Map.Entry next() { + final Map.Entry> entry = iterator.next(); + return new IOStatisticsImplementationHelper.StatsMapEntry( + entry.getKey(), + entry.getValue().applyAsLong(entry.getKey())); + } + + } + +} 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..aa77f42b96d6a --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/DynamicIOStatisticsBuilder.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.ToLongFunction; + +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. + * Instantiate through + * {@link IOStatisticsImplementationHelper#createDynamicIOStatistics()}. + */ +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 add(String key, + ToLongFunction eval) { + activeInstance().add(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 add(String key, + AtomicLong source) { + add(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 add(String key, + AtomicInteger source) { + add(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 add(String key, + MutableCounterLong source) { + add(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..171342127ce71 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/EmptyIOStatistics.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics.impl; + +import java.util.Collections; +import java.util.Iterator; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Set; + +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. + */ +public final class EmptyIOStatistics implements IOStatistics { + + /** + * The sole instance of this class. + */ + private static final EmptyIOStatistics INSTANCE = new EmptyIOStatistics(); + + private EmptyIOStatistics() { + } + + @Override + public Long getStatistic(final String key) { + return null; + } + + @Override + public boolean isTracked(final String key) { + return false; + } + + @Override + public boolean hasAttribute(final Attributes attr) { + return Attributes.Static == attr; + } + + @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/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..25cb9c0d46757 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsFromStorageStatistics.java @@ -0,0 +1,154 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics.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.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; + +/** + * This provides an IOStatistics implementation from a storage statistics + * instance. + * If a null statistics instance is passed in, the statistics are empty. + * This makes it possible to instantiate this from any filesystem.t + */ +public class IOStatisticsFromStorageStatistics + implements IOStatisticsSource { + + private final IOStatistics binding; + + /** + * Instantiate from a storage statistics instance, which may be null, + * in which case the statistics are empty. + * @param storageStatistics from storage statistics. + */ + public IOStatisticsFromStorageStatistics( + final StorageStatistics storageStatistics) { + if (storageStatistics != null) { + binding = new IOStatisticsBinding(storageStatistics); + } else { + binding = null; + } + } + + /** + * Get any IO statistics. + * @return the IO statistics bound to. + */ + @Override + public IOStatistics getIOStatistics() { + return binding; + } + + /** + * The internal binding. + */ + private static final class IOStatisticsBinding implements IOStatistics { + + /** + * Source. + */ + private final StorageStatistics storageStatistics; + + /** + * Keys, calculated in the constructor. + */ + private final Set keys; + + private IOStatisticsBinding(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 boolean hasAttribute(final Attributes attr) { + return Attributes.Dynamic == attr; + } + + @Override + public Long getStatistic(final String key) { + return 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 statistic s. + */ + 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 new IOStatisticsImplementationHelper.StatsMapEntry( + entry.getName(), entry.getValue()); + } + + @Override + public void remove() { + iterator.remove(); + } + } + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsImplementationHelper.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsImplementationHelper.java new file mode 100644 index 0000000000000..be053eccafe60 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsImplementationHelper.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.StorageStatistics; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; + +/** + * Support for implementing IOStatistics interfaces. + */ +public final class IOStatisticsImplementationHelper { + + private IOStatisticsImplementationHelper() { + } + + /** + * Wrap a (dynamic) source with a snapshot IOStatistics instance. + * @param source source + * @return a wrapped instance. + */ + public static IOStatistics wrapWithSnapshot(IOStatistics source) { + return new SnapshotIOStatistics(source); + } + + /** + * Create a builder for dynamic IO Statistics. + * @return a builder to be completed. + */ + public static DynamicIOStatisticsBuilder + createDynamicIOStatistics() { + + return new DynamicIOStatisticsBuilder(); + } + + /** + * Create an IO statistics source 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 IOStatisticsSource createFromStorageStatistics( + StorageStatistics storageStatistics) { + return new IOStatisticsFromStorageStatistics(storageStatistics); + } + + /** + * A map entry for implementations to return. + */ + static final class StatsMapEntry implements Map.Entry { + + /** + * Key. + */ + private final String key; + + /** + * Value. + */ + private Long value; + + /** + * Constructor. + * @param key key + * @param value value + */ + StatsMapEntry(final String key, final Long value) { + this.key = key; + this.value = value; + } + + public String getKey() { + return key; + } + + public Long getValue() { + return value; + } + + @SuppressWarnings("NestedAssignment") + @Override + public Long setValue(final Long value) { + return this.value = value; + } + } +} 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..31578bab2f93a --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/SnapshotIOStatistics.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.Iterator; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; + +import org.apache.hadoop.fs.statistics.IOStatistics; + +/** + * Snapshotted IO statistics; will update on a call to snapshot(). + */ +class SnapshotIOStatistics implements IOStatistics { + + /** + * Treemaps sort their insertions so the iterator is ordered. + */ + private final Map entries + = new TreeMap<>(); + + /** + * Snapshot source. + */ + private final IOStatistics source; + + SnapshotIOStatistics(final IOStatistics source) { + this.source = source; + snapshot(); + } + + @Override + public Long 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 boolean hasAttribute(final Attributes attr) { + return Attributes.Snapshotted == attr; + } + + @Override + public Set keys() { + return entries.keySet(); + } + + @Override + public synchronized boolean snapshot() { + entries.clear(); + for (Map.Entry sourceEntry : source) { + entries.put(sourceEntry.getKey(), sourceEntry.getValue()); + } + return true; + } +} 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..b09893ce67eb3 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/package-info.java @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * This package contains support for statistic collection and reporting. + * This is the public API; implementation classes are to be kept elsewhere. + * + * This package is 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. + * If dynamic, statistics can be enumerated and whenever + * they are retrieved: the latest value will be returned. + * + * These statistics can be used to: log operations, profile applications, make + * assertions about the state of the output. + * + * The names of statistics are a matter of choice of the specific source. + * However, {@link org.apache.hadoop.fs.statistics.StoreStatisticNames} + * contains a + * set of names recommended for object store operations. + * {@link org.apache.hadoop.fs.statistics.StreamStatisticNames} declares + * recommended names for statistics provided for + * input and output streams. + * + * They can also be serialized to build statistics on the overall cost of + * operations, or printed to help diagnose performance/cost issues. + * + * Implementors notes + *
      + *
    1. + * IOStatistics keys SHOULD be standard names where possible. + *
    2. + *
    3. + * MUST be unique to that specific instance of {@link IOStatisticsSource}. + *
    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/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..ac5088769f0b3 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractStreamIOStatisticsTest.java @@ -0,0 +1,236 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.EnumSet; +import java.util.List; +import java.util.Set; + +import org.assertj.core.api.Assertions; +import org.junit.Test; + +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatistics.Attributes; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.*; +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 streams. + * Requires both the input and output streams to offer statistics. + */ +public abstract class AbstractContractStreamIOStatisticsTest + extends AbstractFSContractTestBase { + + @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); + outputStreamAttributes().forEach(a -> + assertIOStatisticsHasAttribute(statistics, a)); + 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); + } + } + + @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()); + try (FSDataOutputStream out = fs.create(path, true)) { + IOStatistics statistics = extractStatistics(out); + // before a write, no bytes + verifyStatisticValue(statistics, STREAM_WRITE_BYTES, 0); + out.write('0'); + statistics = maybeUpdate(statistics, out); + verifyStatisticValue(statistics, STREAM_WRITE_BYTES, 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); + verifyStatisticValue(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()); + try (FSDataOutputStream out = fs.create(path, true)) { + // before a write, no bytes + final byte[] bytes = ContractTestUtils.toAsciiByteArray( + "statistically-speaking"); + final int len = bytes.length; + out.write(bytes); + IOStatistics statistics = extractStatistics(out); + verifyStatisticValue(statistics, STREAM_WRITE_BYTES, len); + out.write(bytes); + statistics = maybeUpdate(statistics, out); + verifyStatisticValue(statistics, STREAM_WRITE_BYTES, 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); + verifyStatisticValue(statistics, STREAM_WRITE_BYTES, 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); + inputStreamAttributes().forEach(a -> + assertIOStatisticsHasAttribute(statistics, a)); + 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); + verifyStatisticValue(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); + verifyStatisticValue(statistics, STREAM_READ_BYTES, 0); + Assertions.assertThat(in.read()).isEqualTo('a'); + current = verifyStatisticValue(statistics, STREAM_READ_BYTES, 1); + final int bufferLen = 128; + byte[] buf128 = new byte[bufferLen]; + in.read(buf128); + current = verifyStatisticValue(statistics, STREAM_READ_BYTES, current + + + bufferLen); + in.readFully(buf128); + current = verifyStatisticValue(statistics, STREAM_READ_BYTES, current + + bufferLen); + in.readFully(0, buf128); + current = verifyStatisticValue(statistics, STREAM_READ_BYTES, current + + bufferLen); + // seek must not increment the read counter + in.seek(256); + verifyStatisticValue(statistics, STREAM_READ_BYTES, current); + + // 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 = verifyStatisticValue(statistics, STREAM_READ_BYTES, + current + sublen); + + // perform some read operations near the end of the file such that + // the buffer will not be completely read. + final int pos = fileLen - sublen; + in.seek(pos); + Assertions.assertThat(in.read(buf128)) + .describedAs("Read overlapping EOF") + .isEqualTo(sublen); + current = verifyStatisticValue(statistics, STREAM_READ_BYTES, + current + sublen); + Assertions.assertThat(in.read(pos, buf128, 0, bufferLen)) + .describedAs("Read(buffer) overlapping EOF") + .isEqualTo(sublen); + current = verifyStatisticValue(statistics, STREAM_READ_BYTES, + current + sublen); + } finally { + fs.delete(path, false); + } + } + + /** + * Attributes of the output stream's statistics. + * @return all attributes which are expected. + */ + public Set outputStreamAttributes() { + return EnumSet.of(Attributes.Dynamic); + } + + /** + * Keys which the output stream must support. + * @return a list of keys + */ + public List outputStreamStatisticKeys() { + return Collections.singletonList(STREAM_WRITE_BYTES); + } + + /** + * Attributes of the input stream's statistics. + * @return all attributes which are expected. + */ + public Set inputStreamAttributes() { + return EnumSet.of(Attributes.Dynamic); + } + + /** + * Keys which the output 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/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..a0789686eaddd --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/IOStatisticAssertions.java @@ -0,0 +1,180 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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; + +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(). + */ +@InterfaceStability.Unstable +public final class IOStatisticAssertions { + + private IOStatisticAssertions() { + } + + /** + * Assert that a statistics instance has an attribute. + * Note: some type inference in Assertions causes confusion + * with the .matches predicate; it needs to be cast down to its type + * again. + * @param stats statistics source + * @param attr attribute to probe for + */ + public static void assertIOStatisticsHasAttribute( + final IOStatistics stats, + final IOStatistics.Attributes attr) { + assertThat(stats) + .describedAs("Statistics %s and attribute %s", stats, attr) + .isNotNull() + .matches(s -> ((IOStatistics) s).hasAttribute(attr), + "Does not have attribute " + attr); + } + + /** + * Assert that a statistics instance has an attribute. + * Note: some type inference in Assertions causes confusion + * with the .matches predicate; it needs to be cast down to its type + * again. + * @param stats statistics source + * @param attr attribute to probe for + */ + public static void assertIOStatisticsAttributeNotFound( + final IOStatistics stats, + final IOStatistics.Attributes attr) { + assertThat(stats) + .describedAs("Statistics %s and attribute %s", stats, attr) + .isNotNull() + .matches(s -> !((IOStatistics) s).hasAttribute(attr), + "Should not have attribute " + attr); + } + + /** + * 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 verifyStatisticValue( + final IOStatistics stats, + final String key, + final long value) { + final Long statistic = stats.getStatistic(key); + assertThat(statistic) + .describedAs("Statistics %s and key %s with expected value %s", stats, + key, value) + .isNotNull() + .isEqualTo(value); + return statistic; + } + + /** + * 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; + } + + /** + * Update IO statistics from the source if they are static; + * dynamic stats are returned as is. + * @param statistics current statistics (or null) + * @param origin origin of the statistics. + * @return the possibly updated statistics + */ + public static IOStatistics maybeUpdate(final IOStatistics statistics, + final Object origin) { + if (statistics == null + || !statistics.hasAttribute(IOStatistics.Attributes.Dynamic)) { + return extractStatistics(origin); + } else { + return statistics; + } + } +} 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..ce9e7f9e5de85 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestEmptyIOStatistics.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics; + +import java.util.Iterator; +import java.util.Map; +import java.util.NoSuchElementException; + +import org.junit.Test; + +import org.apache.hadoop.fs.statistics.impl.EmptyIOStatistics; +import org.apache.hadoop.test.AbstractHadoopTestBase; + +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertIOStatisticsAttributeNotFound; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertIOStatisticsHasAttribute; +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.verifyStatisticValue; +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 stats = EmptyIOStatistics.getInstance(); + + @Test + public void testAttributes() throws Throwable { + assertIOStatisticsHasAttribute(stats, + IOStatistics.Attributes.Static); + assertIOStatisticsAttributeNotFound(stats, + IOStatistics.Attributes.Dynamic); + assertIOStatisticsAttributeNotFound(stats, + IOStatistics.Attributes.Snapshotted); + } + + @Test + public void testSnapshotUnsupported() throws Throwable { + assertThat(stats.snapshot()) + .describedAs("Snapshot of %s", stats) + .isFalse(); + } + + @Test + public void testIterator() throws Throwable { + Iterator> iterator = stats.iterator(); + + assertThat(iterator.hasNext()) + .describedAs("iterator.hasNext()") + .isFalse(); + intercept(NoSuchElementException.class, iterator::next); + } + + @Test + public void testUnknownStatistic() throws Throwable { + assertStatisticIsUnknown(stats, "anything"); + assertStatisticIsUntracked(stats, "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(stats, "anything")) + .isInstanceOf(AssertionError.class); + } + + @Test + public void testStatisticsValueAssertion() throws Throwable { + // expect an exception to be raised when the + // + assertThatThrownBy(() -> + verifyStatisticValue(stats, "anything", 0)) + .isInstanceOf(AssertionError.class); + } + + + +} 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..b57e0e000b7fb 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,25 @@ 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.S3ClientOptions; +import com.amazonaws.services.s3.AmazonS3ClientBuilder; +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; @@ -49,13 +58,22 @@ 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); @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,8 +90,17 @@ 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)); } /** @@ -81,67 +108,107 @@ public AmazonS3 createS3Client(URI name, * 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) { + 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(); + // if this worked life would be so much simpler + // client.setEndpoint(endpoint); + return client; } /** - * Configure S3 client from the Hadoop configuration. - * - * This includes: endpoint, Path Access and possibly other - * options. + * Patch a classically-constructed s3 instance's endpoint. + * @param s3 S3 client + * @param endpoint possibly empty endpoint. * - * @param conf Hadoop configuration - * @return S3 client * @throws IllegalArgumentException if misconfigured */ - private static AmazonS3 configureAmazonS3Client(AmazonS3 s3, - Configuration conf) + protected static AmazonS3 setEndpoint(AmazonS3 s3, + String endpoint) throws IllegalArgumentException { - String endPoint = conf.getTrimmed(ENDPOINT, ""); - if (!endPoint.isEmpty()) { - try { - s3.setEndpoint(endPoint); + if (!endpoint.isEmpty()) { + try { + s3.setEndpoint(endpoint); } catch (IllegalArgumentException e) { - String msg = "Incorrect endpoint: " + e.getMessage(); + String msg = "Incorrect endpoint: " + e.getMessage(); LOG.error(msg); throw new IllegalArgumentException(msg, e); } } - return applyS3ClientOptions(s3, conf); + return s3; } /** - * Perform any tuning of the {@code S3ClientOptions} settings based on - * the Hadoop configuration. - * This is different from the general AWS configuration creation as - * it is unique to S3 connections. + * 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. * - * 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 - * @return the S3 client + * @param endpoint possibly null endpoint. + * @param awsConf config to build the URI from. + * @return a configuration for the S3 client builder. */ - private static AmazonS3 applyS3ClientOptions(AmazonS3 s3, - Configuration conf) { - final boolean pathStyleAccess = conf.getBoolean(PATH_STYLE_ACCESS, false); - if (pathStyleAccess) { - LOG.debug("Enabling path style access!"); - s3.setS3ClientOptions(S3ClientOptions.builder() - .setPathStyleAccess(true) - .build()); + @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; } - return s3; + + 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..96cab1b65be3e 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,7 +20,9 @@ import com.amazonaws.ClientConfiguration; import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.metrics.RequestMetricCollector; import com.amazonaws.services.s3.AmazonS3; +import com.amazonaws.services.s3.S3ClientOptions; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -40,12 +42,27 @@ 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()); + if (pathStyleAccess) { + LOG.debug("Enabling path style access!"); + s3.setS3ClientOptions(S3ClientOptions.builder() + .setPathStyleAccess(true) + .build()); + } + setEndpoint(s3, endpoint); + 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 66cac99de7baa..0bfa069beb1de 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,6 +49,11 @@ 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.fs.statistics.impl.EmptyIOStatistics; import org.apache.hadoop.util.Progressable; import static org.apache.hadoop.fs.s3a.S3AUtils.*; @@ -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.createIOStatistics() + : EmptyIOStatistics.getInstance(); 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(); @@ -466,6 +479,7 @@ public String toString() { if (block != null) { sb.append(", activeBlock=").append(block); } + sb.append(IOStatisticsLogging.iostatisticsSourceToString(this)); sb.append('}'); return sb.toString(); } @@ -486,7 +500,7 @@ private long now() { * Get the statistics for this stream. * @return stream statistics */ - S3AInstrumentation.OutputStreamStatistics getStatistics() { + BlockOutputStreamStatistics getStatistics() { return statistics; } @@ -518,6 +532,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..c0de7bdaa910c 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; + protected final BlockOutputStreamStatistics statistics; protected DataBlock(long index, - S3AInstrumentation.OutputStreamStatistics statistics) { + BlockOutputStreamStatistics statistics) { this.index = index; this.statistics = statistics; } @@ -387,7 +388,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 +433,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 +511,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 +561,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 +806,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 +830,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..3eefaec1bb253 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,14 @@ 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.IntegratedS3AStatisticsContext; +import org.apache.hadoop.fs.s3a.impl.statistics.S3AStatisticsContext; 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.IOStatisticsImplementationHelper; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.Text; import org.apache.hadoop.security.token.TokenIdentifier; @@ -195,7 +201,8 @@ @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 +257,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; @@ -343,6 +353,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 +363,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 +514,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 IntegratedS3AStatisticsContext( + new IntegratedS3AStatisticsContext.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 +618,7 @@ protected void verifyBucketExistsV2() * Get S3A Instrumentation. For test purposes. * @return this instance's instrumentation. */ + @VisibleForTesting public S3AInstrumentation getInstrumentation() { return instrumentation; } @@ -641,7 +679,8 @@ private void bindAWSClient(URI name, boolean dtEnabled) throws IOException { S3ClientFactory.class); s3 = ReflectionUtils.newInstance(s3ClientFactoryClass, conf) - .createS3Client(getUri(), bucket, credentials, uaSuffix); + .createS3Client(getUri(), bucket, credentials, uaSuffix, + statisticsContext.newStatisticsFromAwsSdk()); } /** @@ -1139,7 +1178,7 @@ private S3AReadOpContext createReadContext( invoker, s3guardInvoker, statistics, - instrumentation, + statisticsContext, fileStatus, seekPolicy, changePolicy, @@ -1244,7 +1283,7 @@ public FSDataOutputStream create(Path f, FsPermission permission, progress, partSize, blockFactory, - instrumentation.newOutputStreamStatistics(statistics), + statisticsContext.newOutputStreamStatistics(), getWriteOperationHelper(), putTracker), null); @@ -1706,8 +1745,7 @@ protected void incrementStatistic(Statistic statistic) { * @param count the count to increment */ protected void incrementStatistic(Statistic statistic, long count) { - instrumentation.incrementCounter(statistic, count); - storageStatistics.incrementCounter(statistic, count); + statisticsContext.incrementCounter(statistic, count); } /** @@ -1716,7 +1754,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 +1763,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 +1776,7 @@ public void operationRetried(Exception ex) { if (isThrottleException(ex)) { operationThrottled(false); } else { + incrementStatistic(STORE_IO_RETRY); incrementStatistic(IGNORED_ERRORS); } } @@ -1789,11 +1828,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 +1845,16 @@ public S3AStorageStatistics getStorageStatistics() { return storageStatistics; } + /** + * Get this filesystem's storage statistics as IO Statistics. + * @return statistics + */ + @Override + public IOStatistics getIOStatistics() { + return IOStatisticsImplementationHelper.createFromStorageStatistics( + storageStatistics).getIOStatistics(); + } + /** * Request object metadata; increments counters in the process. * Retry policy: retry untranslated. @@ -3441,8 +3490,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 +4498,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 +4613,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 +4771,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..2596f5bb0cf76 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.createIOStatistics(); 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..6005dfeb48ae2 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,27 @@ 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.IOStatistics; +import org.apache.hadoop.fs.statistics.StreamStatisticNames; +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,12 +56,14 @@ 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.impl.IOStatisticsImplementationHelper.createDynamicIOStatistics; import static org.apache.hadoop.fs.s3a.Statistic.*; /** @@ -63,32 +78,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 +208,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 +579,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 +599,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,23 +608,23 @@ 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.openOperations.get()); + streamCloseOperations.incr(statistics.closeOperations.get()); + streamClosed.incr(statistics.closed.get()); + streamAborted.incr(statistics.aborted.get()); + streamSeekOperations.incr(statistics.seekOperations.get()); + streamReadExceptions.incr(statistics.readExceptions.get()); + streamForwardSeekOperations.incr(statistics.forwardSeekOperations.get()); + streamBytesSkippedOnSeek.incr(statistics.bytesSkippedOnSeek.get()); + streamBackwardSeekOperations.incr(statistics.backwardSeekOperations.get()); + streamBytesBackwardsOnSeek.incr(statistics.bytesBackwardsOnSeek.get()); + streamBytesRead.incr(statistics.bytesRead.get()); + streamReadOperations.incr(statistics.readOperations.get()); + streamReadFullyOperations.incr(statistics.readFullyOperations.get()); + streamReadsIncomplete.incr(statistics.readsIncomplete.get()); + streamBytesReadInClose.incr(statistics.bytesReadInClose.get()); + streamBytesDiscardedInAbort.incr(statistics.bytesDiscardedInAbort.get()); incrementCounter(STREAM_READ_VERSION_MISMATCHES, statistics.versionMismatches.get()); } @@ -622,7 +642,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 +655,44 @@ 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. + * These counters are marked as volatile so that IOStatistics on the stream + * will get the latest values. + * They are only to be incremented within synchronized blocks. */ - @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; + + public final AtomicLong openOperations= new AtomicLong(0); + public final AtomicLong closeOperations= new AtomicLong(0); + public final AtomicLong closed= new AtomicLong(0); + public final AtomicLong aborted= new AtomicLong(0); + public final AtomicLong seekOperations= new AtomicLong(0); + public final AtomicLong readExceptions= new AtomicLong(0); + public final AtomicLong forwardSeekOperations= new AtomicLong(0); + public final AtomicLong backwardSeekOperations= new AtomicLong(0); + public final AtomicLong bytesRead= new AtomicLong(0); + public final AtomicLong bytesSkippedOnSeek= new AtomicLong(0); + public final AtomicLong bytesBackwardsOnSeek= new AtomicLong(0); + public final AtomicLong readOperations= new AtomicLong(0); + public final AtomicLong readFullyOperations= new AtomicLong(0); + public final AtomicLong readsIncomplete= new AtomicLong(0); + public final AtomicLong bytesReadInClose= new AtomicLong(0); + public final AtomicLong bytesDiscardedInAbort= new AtomicLong(0); + public final AtomicLong policySetCount= new AtomicLong(0); + public volatile long inputPolicy; private final AtomicLong versionMismatches = new AtomicLong(0); - private InputStreamStatistics mergedStats; + private InputStreamStatisticsImpl mergedStats; - private InputStreamStatistics() { + private InputStreamStatisticsImpl( + FileSystem.Statistics filesystemStatistics) { + this.filesystemStatistics = filesystemStatistics; } /** @@ -670,10 +700,11 @@ 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; + seekOperations.incrementAndGet(); + backwardSeekOperations.incrementAndGet(); + bytesBackwardsOnSeek.addAndGet(-negativeOffset); } /** @@ -682,11 +713,12 @@ 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++; + seekOperations.incrementAndGet(); + forwardSeekOperations.incrementAndGet(); if (skipped > 0) { - bytesSkippedOnSeek += skipped; + bytesSkippedOnSeek.addAndGet(skipped); } } @@ -694,10 +726,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 openOperations.getAndIncrement(); } /** @@ -707,23 +738,25 @@ public long streamOpened() { * @param remainingInCurrentRequest the number of bytes remaining in * the current request. */ + @Override public void streamClose(boolean abortedConnection, long remainingInCurrentRequest) { - closeOperations++; + closeOperations.incrementAndGet(); if (abortedConnection) { - this.aborted++; - bytesDiscardedInAbort += remainingInCurrentRequest; + aborted.incrementAndGet(); + bytesDiscardedInAbort.addAndGet(remainingInCurrentRequest); } else { - closed++; - bytesReadInClose += remainingInCurrentRequest; + closed.incrementAndGet(); + bytesReadInClose.addAndGet(remainingInCurrentRequest); } } /** * An ignored stream read exception was received. */ + @Override public void readException() { - readExceptions++; + readExceptions.incrementAndGet(); } /** @@ -731,9 +764,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; + bytesRead.addAndGet(bytes); } } @@ -742,8 +776,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++; + readOperations.incrementAndGet(); } /** @@ -752,8 +787,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++; + readFullyOperations.incrementAndGet(); } /** @@ -761,9 +797,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++; + readsIncomplete.incrementAndGet(); } } @@ -780,17 +817,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 +844,26 @@ 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("OpenOperations=").append(openOperations.get()); + sb.append(", CloseOperations=").append(closeOperations.get()); + sb.append(", Closed=").append(closed.get()); + sb.append(", Aborted=").append(aborted.get()); + sb.append(", SeekOperations=").append(seekOperations.get()); + sb.append(", ReadExceptions=").append(readExceptions.get()); sb.append(", ForwardSeekOperations=") - .append(forwardSeekOperations); + .append(forwardSeekOperations.get()); sb.append(", BackwardSeekOperations=") - .append(backwardSeekOperations); - sb.append(", BytesSkippedOnSeek=").append(bytesSkippedOnSeek); - sb.append(", BytesBackwardsOnSeek=").append(bytesBackwardsOnSeek); - sb.append(", BytesRead=").append(bytesRead); + .append(backwardSeekOperations.get()); + sb.append(", BytesSkippedOnSeek=").append(bytesSkippedOnSeek.get()); + sb.append(", BytesBackwardsOnSeek=").append(bytesBackwardsOnSeek.get()); + sb.append(", BytesRead=").append(bytesRead.get()); 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); + .append(bytesRead.get() - bytesSkippedOnSeek.get()); + sb.append(", ReadOperations=").append(readOperations.get()); + sb.append(", ReadFullyOperations=").append(readFullyOperations.get()); + sb.append(", ReadsIncomplete=").append(readsIncomplete.get()); + sb.append(", BytesReadInClose=").append(bytesReadInClose.get()); + sb.append(", BytesDiscardedInAbort=").append(bytesDiscardedInAbort.get()); sb.append(", InputPolicy=").append(inputPolicy); sb.append(", InputPolicySetCount=").append(policySetCount); sb.append(", versionMismatches=").append(versionMismatches.get()); @@ -839,110 +879,251 @@ public String toString() { * 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, + bytesRead.get() + bytesReadInClose.get()); + } } } /** - * 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; - diff.inputPolicy = inputPolicy - inputStats.inputPolicy; + private InputStreamStatisticsImpl setd(InputStreamStatisticsImpl inputStats) { + InputStreamStatisticsImpl diff = + new InputStreamStatisticsImpl(filesystemStatistics); + setd(diff.openOperations, openOperations, inputStats.openOperations); + setd(diff.closeOperations, closeOperations, inputStats.closeOperations); + setd(diff.closed, closed, inputStats.closed); + setd(diff.aborted, aborted, inputStats.aborted); + setd(diff.seekOperations, seekOperations, inputStats.seekOperations); + setd(diff.readExceptions, readExceptions, inputStats.readExceptions); + setd(diff.forwardSeekOperations, forwardSeekOperations, inputStats.forwardSeekOperations); + setd(diff.backwardSeekOperations, backwardSeekOperations, inputStats.backwardSeekOperations); + setd(diff.bytesRead, bytesRead, inputStats.bytesRead); + setd(diff.bytesSkippedOnSeek, bytesSkippedOnSeek, inputStats.bytesSkippedOnSeek); + setd(diff.bytesBackwardsOnSeek, bytesBackwardsOnSeek, inputStats.bytesBackwardsOnSeek); + setd(diff.readOperations, readOperations, inputStats.readOperations); + setd(diff.readFullyOperations, readFullyOperations, inputStats.readFullyOperations); + setd(diff.readsIncomplete, readsIncomplete, inputStats.readsIncomplete); + setd(diff.bytesReadInClose, bytesReadInClose, inputStats.bytesReadInClose); + setd(diff.bytesDiscardedInAbort, bytesDiscardedInAbort, inputStats.bytesDiscardedInAbort); + setd(diff.policySetCount, policySetCount, inputStats.policySetCount); + diff.inputPolicy = inputPolicy -inputStats.inputPolicy; diff.versionMismatches.set(versionMismatches.longValue() - inputStats.versionMismatches.longValue()); return diff; } /** - * 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.openOperations.set(openOperations.get()); + copy.closeOperations.set(closeOperations.get()); + copy.closed.set(closed.get()); + copy.aborted.set(aborted.get()); + copy.seekOperations.set(seekOperations.get()); + copy.readExceptions.set(readExceptions.get()); + copy.forwardSeekOperations.set(forwardSeekOperations.get()); + copy.backwardSeekOperations.set(backwardSeekOperations.get()); + copy.bytesRead.set(bytesRead.get()); + copy.bytesSkippedOnSeek.set(bytesSkippedOnSeek.get()); + copy.bytesBackwardsOnSeek.set(bytesBackwardsOnSeek.get()); + copy.readOperations.set(readOperations.get()); + copy.readFullyOperations.set(readFullyOperations.get()); + copy.readsIncomplete.set(readsIncomplete.get()); + copy.bytesReadInClose.set(bytesReadInClose.get()); + copy.bytesDiscardedInAbort.set(bytesDiscardedInAbort.get()); + copy.policySetCount.set(policySetCount.get()); copy.inputPolicy = inputPolicy; return copy; } + + /** + * Convert to an IOStatistics source which is dynamically updated. + * @return statistics + */ + @Override + public IOStatistics createIOStatistics() { + DynamicIOStatisticsBuilder builder + = createDynamicIOStatistics(); + + builder.add(StreamStatisticNames.STREAM_BYTES_DISCARDED_ABORT, + bytesDiscardedInAbort); + builder.add(StreamStatisticNames.STREAM_CLOSED, + closed); + builder.add(StreamStatisticNames.STREAM_CLOSE_OPERATIONS, + closeOperations); + builder.add(StreamStatisticNames.STREAM_OPENED, + openOperations); + builder.add(StreamStatisticNames.STREAM_READ_BYTES, + bytesRead); + builder.add(StreamStatisticNames.STREAM_READ_EXCEPTIONS, + readExceptions); + builder.add(StreamStatisticNames.STREAM_READ_FULLY_OPERATIONS, + readFullyOperations); + builder.add(StreamStatisticNames.STREAM_READ_OPERATIONS, + readOperations); + builder.add(StreamStatisticNames.STREAM_READ_OPERATIONS_INCOMPLETE, + readsIncomplete); + return builder.build(); + } + + @Override + public long getCloseOperations() { + return closeOperations.get(); + } + + @Override + public long getClosed() { + return closed.get(); + } + + @Override + public long getAborted() { + return aborted.get(); + } + + @Override + public long getForwardSeekOperations() { + return forwardSeekOperations.get(); + } + + @Override + public long getBackwardSeekOperations() { + return backwardSeekOperations.get(); + } + + @Override + public long getBytesRead() { + return bytesRead.get(); + } + + @Override + public long getBytesSkippedOnSeek() { + return bytesSkippedOnSeek.get(); + } + + @Override + public long getBytesBackwardsOnSeek() { + return bytesBackwardsOnSeek.get(); + } + + @Override + public long getBytesReadInClose() { + return bytesReadInClose.get(); + } + + @Override + public long getBytesDiscardedInAbort() { + return bytesDiscardedInAbort.get(); + } + + @Override + public long getOpenOperations() { + return openOperations.get(); + } + + @Override + public long getSeekOperations() { + return seekOperations.get(); + } + + @Override + public long getReadExceptions() { + return readExceptions.get(); + } + + @Override + public long getReadOperations() { + return readOperations.get(); + } + + @Override + public long getReadFullyOperations() { + return readFullyOperations.get(); + } + + @Override + public long getReadsIncomplete() { + return readsIncomplete.get(); + } + + @Override + public long getPolicySetCount() { + return policySetCount.get(); + } + + @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 +1131,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 +1139,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 +1175,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 +1185,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 +1198,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 +1215,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 +1225,8 @@ void exceptionInMultipartComplete(int count) { /** * Note an exception in a multipart abort. */ - void exceptionInMultipartAbort() { + @Override + public void exceptionInMultipartAbort() { exceptionsInMultipartFinalize.incrementAndGet(); } @@ -1043,6 +1234,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 +1244,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 +1260,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,11 +1283,13 @@ 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(); } @@ -1097,10 +1298,28 @@ public int blocksReleased() { * 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,9 +1332,10 @@ 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(", blocksActivelyAllocated=").append(getBlocksActivelyAllocated()); sb.append(", exceptionsInMultipartFinalize=").append( exceptionsInMultipartFinalize); sb.append(", transferDuration=").append(transferDuration).append(" ms"); @@ -1128,6 +1348,23 @@ public String toString() { sb.append('}'); return sb.toString(); } + + /** + * Convert to an IOStatistics source which is dynamically updated. + * @return statistics + */ + @Override + public IOStatistics createIOStatistics() { + DynamicIOStatisticsBuilder builder = createDynamicIOStatistics(); + + builder.add(StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS, + blocksSubmitted); + builder.add(StreamStatisticNames.STREAM_WRITE_BYTES, + bytesWritten); + builder.add(StreamStatisticNames.STREAM_WRITE_FAILURES, + blockUploadsFailed); + return builder.build(); + } } /** @@ -1192,13 +1429,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 +1443,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 +1452,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 +1482,7 @@ public void taskCompleted(boolean success) { 1); } + @Override public void jobCompleted(boolean success) { incrementCounter( success ? COMMITTER_JOBS_SUCCEEDED @@ -1253,20 +1496,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 +1609,66 @@ 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..60149ced38beb 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 * @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..54f17cbdcb7c9 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,8 @@ package org.apache.hadoop.fs.s3a; -import org.apache.hadoop.fs.StorageStatistics.CommonStatisticNames; +import org.apache.hadoop.fs.statistics.StoreStatisticNames; +import org.apache.hadoop.fs.statistics.StreamStatisticNames; import java.util.HashMap; import java.util.Map; @@ -49,39 +50,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 +124,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_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_CLOSED, + "Count of times the TCP stream was closed"), + STREAM_CLOSE_OPERATIONS( + StreamStatisticNames.STREAM_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_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_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_BYTES_DISCARDED_ABORT, "Count of bytes discarded by aborting the stream"), - STREAM_WRITE_FAILURES("stream_write_failures", + STREAM_WRITE_FAILURES( + StreamStatisticNames.STREAM_WRITE_FAILURES, "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 +201,9 @@ 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 +278,17 @@ 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 602732b6d3250..552a5449a55a7 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 @@ -49,6 +49,7 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +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; @@ -104,17 +105,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 8592ad490184e..27bc52591466f 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 @@ -44,12 +44,12 @@ import org.apache.hadoop.fs.PathFilter; 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.statistics.CommitterStatistics; import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.util.DurationInfo; @@ -79,7 +79,7 @@ public class CommitOperations { private final S3AFileSystem fs; /** Statistics. */ - private final S3AInstrumentation.CommitterStatistics statistics; + private final CommitterStatistics statistics; /** * Write operations for the destination fs. @@ -126,7 +126,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..1de2df0278303 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,9 @@ 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..946c27ecc1eb7 --- /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 nextOperationId = 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 nextOperationId.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/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..7b45258f38c97 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/AwsStatisticsCollector.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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; + } + + @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..77d73e115392a --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/BlockOutputStreamStatistics.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.s3a.impl.statistics; + +import java.io.Closeable; + +import org.apache.hadoop.fs.statistics.IOStatistics; + +/** + * Block output stream statistics. + */ +public interface BlockOutputStreamStatistics extends Closeable { + + /** + * 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; my 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(); + + /** + * Convert to an IOStatistics source which is + * dynamically updated. + * @return statistics + */ + IOStatistics createIOStatistics(); + + /** + * 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/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..6b58fa8b06c9b --- /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. + */ +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..03ce2aed5b5a2 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/EmptyS3AStatisticsContext.java @@ -0,0 +1,457 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl.statistics; + +import java.io.IOException; +import java.time.Duration; + +import org.apache.hadoop.fs.s3a.Statistic; +import org.apache.hadoop.fs.s3a.s3guard.MetastoreInstrumentation; +import org.apache.hadoop.fs.s3a.s3guard.MetastoreInstrumentationImpl; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.impl.EmptyIOStatistics; + +/** + * 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 getMetastoreInstrumentation() { + 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 createIOStatistics() { + return EmptyIOStatistics.getInstance(); + } + + @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 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 createIOStatistics() { + return EmptyIOStatistics.getInstance(); + } + + @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/IntegratedS3AStatisticsContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/IntegratedS3AStatisticsContext.java new file mode 100644 index 0000000000000..faeb9b434e331 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/IntegratedS3AStatisticsContext.java @@ -0,0 +1,201 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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 IntegratedS3AStatisticsContext implements S3AStatisticsContext { + + private final S3AFSStatisticsSource statisticsSource; + + /** + * Instantiate. + * @param statisticsSource integration binding + */ + public IntegratedS3AStatisticsContext( + final S3AFSStatisticsSource statisticsSource) { + this.statisticsSource = statisticsSource; + } + + + /** + * Get the instrumentation from the FS integraation. + * @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 getMetastoreInstrumentation() { + 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/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..3c1023f653f1b --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AInputStreamStatistics.java @@ -0,0 +1,164 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.IOStatistics; + +/** + * Statistics updated by an input stream during its actual operation. + * It also contains getters for tests. + */ +public interface S3AInputStreamStatistics extends AutoCloseable { + + /** + * 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); + + /** + * Convert to an IOStatistics source which is + * dynamically updated. + * @return statistics + */ + IOStatistics createIOStatistics(); + + 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(); +} 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..80cb4bd8bd44d --- /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 getMetastoreInstrumentation(); + + /** + * 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..333e2d6649560 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/StatisticsFromAwsSdk.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.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/s3guard/DynamoDBMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java index 38b38fb7f93e7..7bf1b829c92bb 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java @@ -450,7 +450,8 @@ void bindToOwnerFilesystem(final S3AFileSystem fs) { owner = fs; conf = owner.getConf(); StoreContext context = owner.createStoreContext(); - instrumentation = context.getInstrumentation().getS3GuardInstrumentation(); + instrumentation = context.getInstrumentation() + .getMetastoreInstrumentation(); username = context.getUsername(); executor = context.createThrottledExecutor(); ttlTimeProvider = Preconditions.checkNotNull( 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/contract/s3a/ITestS3AContractStreamIOStatistics.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractStreamIOStatistics.java new file mode 100644 index 0000000000000..024c970233fc4 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractStreamIOStatistics.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.s3a; + +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.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 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_FAILURES); + } + +} 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..deb36c7719f24 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 @@ -24,10 +24,13 @@ import java.util.ArrayList; import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.metrics.RequestMetricCollector; import com.amazonaws.services.s3.AmazonS3; 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 +41,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 ff176f58da67d..a15b75c63134f 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 @@ -45,11 +45,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)); } 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..4077257ad764e 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 @@ -305,6 +305,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(); @@ -562,7 +565,7 @@ protected ObjectMetadata readLandsatMetadata(final S3AFileSystem delegatedFS) factory.setConf(conf); String host = landsat.getHost(); AmazonS3 s3 = factory.createS3Client(landsat, host, testing, - "ITestSessionDelegationInFileystem"); + "ITestSessionDelegationInFileystem", null); 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/TestNeworkBinding.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNetworkBinding.java similarity index 52% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNeworkBinding.java rename to hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNetworkBinding.java index eebc3bfdf2fe4..882e12c5665a1 100644 --- 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/TestNetworkBinding.java @@ -18,19 +18,23 @@ package org.apache.hadoop.fs.s3a.impl; -import org.assertj.core.api.Assertions; +import com.amazonaws.ClientConfiguration; +import com.amazonaws.client.builder.AwsClientBuilder; import org.junit.Test; -import org.apache.hadoop.test.HadoopTestBase; +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 TestNeworkBinding extends HadoopTestBase { +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 @@ -54,8 +58,45 @@ public void testRegionNullToUSEast() { } private static void assertRegionFixup(String region, String expected) { - Assertions.assertThat(fixBucketRegion(region)) + assertThat(fixBucketRegion(region)) .describedAs("Fixup of %s", region) .isEqualTo(expected); } + + @Test + public void testNull() throws Throwable { + expectEndpoint("", true, "unused"); + } + + @Test + public void testUSEastEndpoint() throws Throwable { + expectEndpoint(US_EAST_1, false, US_EAST_1); + } + + @Test + 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/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/scale/AbstractSTestS3AHugeFiles.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java index 315d1fe7285be..827e34168f357 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,6 @@ 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.junit.FixMethodOrder; import org.junit.Test; import org.junit.runners.MethodSorters; @@ -36,12 +34,14 @@ 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.util.Progressable; import static org.apache.hadoop.fs.contract.ContractTestUtils.*; @@ -170,7 +170,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, @@ -234,7 +234,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