You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by st...@apache.org on 2020/12/31 21:56:05 UTC

[hadoop] branch trunk updated: HADOOP-17271. S3A connector to support IOStatistics. (#2580)

This is an automated email from the ASF dual-hosted git repository.

stevel pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 617af28  HADOOP-17271. S3A connector to support IOStatistics. (#2580)
617af28 is described below

commit 617af28e80774249b0a52006ca70a4bf0b14451b
Author: Steve Loughran <st...@cloudera.com>
AuthorDate: Thu Dec 31 21:55:39 2020 +0000

    HADOOP-17271. S3A connector to support IOStatistics. (#2580)
    
    
    S3A connector to support the IOStatistics API of HADOOP-16830,
    
    This is a major rework of the S3A Statistics collection to
    
    * Embrace the IOStatistics APIs
    * Move from direct references of S3AInstrumention statistics
      collectors to interface/implementation classes in new packages.
    * Ubiquitous support of IOStatistics, including:
      S3AFileSystem, input and output streams, RemoteIterator instances
      provided in list calls.
    * Adoption of new statistic names from hadoop-common
    
    Regarding statistic collection, as well as all existing
    statistics, the connector now records min/max/mean durations
    of HTTP GET and HEAD requests, and those of LIST operations.
    
    Contributed by Steve Loughran.
---
 .../hadoop-aws/dev-support/findbugs-exclude.xml    |   10 +
 hadoop-tools/hadoop-aws/pom.xml                    |    4 +
 .../java/org/apache/hadoop/fs/s3a/Constants.java   |    7 +
 .../hadoop/fs/s3a/DefaultS3ClientFactory.java      |  207 ++-
 .../hadoop/fs/s3a/InconsistentS3ClientFactory.java |   14 +-
 .../java/org/apache/hadoop/fs/s3a/Invoker.java     |   48 +-
 .../java/org/apache/hadoop/fs/s3a/Listing.java     |  288 ++--
 .../apache/hadoop/fs/s3a/S3ABlockOutputStream.java |   64 +-
 .../org/apache/hadoop/fs/s3a/S3ADataBlocks.java    |   23 +-
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java    |  306 ++--
 .../org/apache/hadoop/fs/s3a/S3AInputStream.java   |   61 +-
 .../apache/hadoop/fs/s3a/S3AInstrumentation.java   | 1514 +++++++++++++-------
 .../org/apache/hadoop/fs/s3a/S3AOpContext.java     |   32 +-
 .../org/apache/hadoop/fs/s3a/S3AReadOpContext.java |   10 +-
 .../apache/hadoop/fs/s3a/S3AStorageStatistics.java |   99 +-
 .../java/org/apache/hadoop/fs/s3a/S3AUtils.java    |    8 +-
 .../org/apache/hadoop/fs/s3a/S3ClientFactory.java  |    5 +-
 .../java/org/apache/hadoop/fs/s3a/Statistic.java   |  640 ++++++---
 .../apache/hadoop/fs/s3a/WriteOperationHelper.java |   17 +-
 .../org/apache/hadoop/fs/s3a/WriteOperations.java  |    5 +-
 .../s3a/auth/delegation/S3ADelegationTokens.java   |   14 +-
 .../hadoop/fs/s3a/commit/AbstractS3ACommitter.java |   61 +-
 .../hadoop/fs/s3a/commit/CommitOperations.java     |   52 +-
 .../fs/s3a/commit/MagicCommitIntegration.java      |   10 +-
 .../apache/hadoop/fs/s3a/commit/PutTracker.java    |    5 +-
 .../hadoop/fs/s3a/commit/files/PendingSet.java     |   38 +-
 .../fs/s3a/commit/files/SinglePendingCommit.java   |   31 +-
 .../hadoop/fs/s3a/commit/files/SuccessData.java    |   31 +-
 .../fs/s3a/commit/magic/MagicCommitTracker.java    |   23 +-
 .../fs/s3a/commit/magic/MagicS3GuardCommitter.java |    6 +
 .../hadoop/fs/s3a/impl/ActiveOperationContext.java |   92 ++
 .../hadoop/fs/s3a/impl/BulkDeleteRetryHandler.java |    5 +-
 .../hadoop/fs/s3a/impl/CallableSupplier.java       |    8 +-
 .../apache/hadoop/fs/s3a/impl/ChangeTracker.java   |   19 +-
 .../hadoop/fs/s3a/impl/ErrorTranslation.java       |   12 +
 .../hadoop/fs/s3a/impl/InternalConstants.java      |    7 +
 .../fs/s3a/impl/ListingOperationCallbacks.java     |   10 +-
 .../apache/hadoop/fs/s3a/impl/NetworkBinding.java  |   18 +-
 .../hadoop/fs/s3a/impl/S3AMultipartUploader.java   |   24 +-
 .../fs/s3a/impl/S3AMultipartUploaderBuilder.java   |    2 +-
 .../apache/hadoop/fs/s3a/impl/StoreContext.java    |   15 +-
 .../hadoop/fs/s3a/impl/StoreContextBuilder.java    |    6 +-
 .../fs/s3a/s3guard/DynamoDBMetadataStore.java      |   20 +-
 .../hadoop/fs/s3a/s3guard/InternalIterators.java   |   91 --
 .../hadoop/fs/s3a/s3guard/RetryingCollection.java  |   10 +-
 .../hadoop/fs/s3a/select/SelectInputStream.java    |   12 +-
 .../statistics/BlockOutputStreamStatistics.java    |  137 ++
 .../ChangeTrackerStatistics.java}                  |   31 +-
 .../fs/s3a/statistics/CommitterStatistics.java     |   66 +
 .../fs/s3a/statistics/CountersAndGauges.java       |   72 +
 .../DelegationTokenStatistics.java}                |   21 +-
 .../s3a/statistics/S3AInputStreamStatistics.java   |  191 +++
 .../statistics/S3AMultipartUploaderStatistics.java |    7 +-
 .../S3AStatisticInterface.java}                    |   21 +-
 .../fs/s3a/statistics/S3AStatisticsContext.java    |   69 +
 .../StatisticTypeEnum.java}                        |   31 +-
 .../fs/s3a/statistics/StatisticsFromAwsSdk.java    |   81 ++
 .../impl/AbstractS3AStatisticsSource.java          |  111 ++
 .../statistics/impl/AwsStatisticsCollector.java    |  129 ++
 .../impl/BondedS3AStatisticsContext.java           |  234 +++
 .../statistics/impl/CountingChangeTracker.java}    |   39 +-
 .../statistics/impl/EmptyS3AStatisticsContext.java |  597 ++++++++
 .../impl}/S3AMultipartUploaderStatisticsImpl.java  |   46 +-
 .../statistics/impl/StatisticsFromAwsSdkImpl.java  |   88 ++
 .../impl/package-info.java}                        |   24 +-
 .../package-info.java}                             |   28 +-
 .../org/apache/hadoop/fs/s3a/tools/MarkerTool.java |   12 +-
 .../fs/contract/s3a/ITestS3AContractDistCp.java    |   12 +-
 .../apache/hadoop/fs/s3a/AbstractS3ATestBase.java  |   23 +
 .../fs/s3a/ITestLocatedFileStatusFetcher.java      |  266 +++-
 .../hadoop/fs/s3a/ITestS3ABlockOutputArray.java    |    9 +-
 .../org/apache/hadoop/fs/s3a/ITestS3AMetrics.java  |   31 +-
 .../hadoop/fs/s3a/ITestS3AMiscOperations.java      |    8 +-
 .../org/apache/hadoop/fs/s3a/ITestS3AUnbuffer.java |  105 +-
 .../hadoop/fs/s3a/ITestS3GuardEmptyDirs.java       |    2 +-
 .../apache/hadoop/fs/s3a/MockS3AFileSystem.java    |   20 +-
 .../apache/hadoop/fs/s3a/MockS3ClientFactory.java  |    5 +-
 .../org/apache/hadoop/fs/s3a/S3ATestUtils.java     |   11 +-
 .../java/org/apache/hadoop/fs/s3a/TestListing.java |   54 +-
 .../hadoop/fs/s3a/TestS3ABlockOutputStream.java    |    8 +-
 .../hadoop/fs/s3a/TestStreamChangeTracker.java     |    7 +-
 .../apache/hadoop/fs/s3a/auth/ITestAssumeRole.java |   13 +-
 .../fs/s3a/auth/ITestRestrictedReadAccess.java     |    2 +-
 .../apache/hadoop/fs/s3a/auth/ProgressCounter.java |    8 +-
 .../ITestSessionDelegationInFileystem.java         |   17 +-
 .../hadoop/fs/s3a/commit/AbstractCommitITest.java  |    9 +-
 .../fs/s3a/commit/AbstractITCommitProtocol.java    |   25 +-
 .../commit/integration/ITestS3ACommitterMRJob.java |    1 -
 .../s3a/commit/staging/MockedStagingCommitter.java |    4 +-
 .../fs/s3a/impl/ITestPartialRenamesDeletes.java    |   11 +-
 .../hadoop/fs/s3a/impl/TestNetworkBinding.java     |  105 ++
 .../hadoop/fs/s3a/impl/TestNeworkBinding.java      |   61 -
 .../fs/s3a/performance/AbstractS3ACostTest.java    |   72 +-
 .../fs/s3a/performance/ITestS3ADeleteCost.java     |   39 +-
 .../fs/s3a/performance/ITestS3ARenameCost.java     |   30 +-
 .../hadoop/fs/s3a/performance/OperationCost.java   |    2 +
 .../fs/s3a/performance/OperationCostValidator.java |   36 +-
 ...TestDynamoDBMetadataStoreAuthoritativeMode.java |    8 +-
 .../s3guard/ITestDynamoDBMetadataStoreScale.java   |    4 +-
 .../fs/s3a/scale/AbstractSTestS3AHugeFiles.java    |   35 +-
 .../scale/ILoadTestS3ABulkDeleteThrottling.java    |   11 -
 .../fs/s3a/scale/ITestS3ADirectoryPerformance.java |   39 +-
 .../s3a/scale/ITestS3AInputStreamPerformance.java  |   94 +-
 .../hadoop/fs/s3a/scale/S3AScaleTestBase.java      |    3 +-
 .../apache/hadoop/fs/s3a/select/ITestS3Select.java |   34 +-
 .../hadoop/fs/s3a/select/ITestS3SelectLandsat.java |    6 +-
 .../hadoop/fs/s3a/select/ITestS3SelectMRJob.java   |    4 +-
 .../s3a/statistics/ITestAggregateIOStatistics.java |  100 ++
 .../ITestS3AContractStreamIOStatistics.java        |   90 ++
 .../s3a/test/MinimalListingOperationCallbacks.java |   12 +-
 .../hadoop-aws/src/test/resources/log4j.properties |    8 +
 111 files changed, 5663 insertions(+), 1800 deletions(-)

diff --git a/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml b/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml
index b510536..1ebf858 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 @@
     <Bug pattern="SF_SWITCH_FALLTHROUGH"/>
   </Match>
 
+  <!--
+  Some of the S3A Instrumentation classes increment volatile references from
+  within synchronized contexts; they use volatile to keep the cost
+  of these updates and reading them down.
+   -->
+  <Match>
+    <Class name="org.apache.hadoop.fs.s3a.S3AInstrumentation$InputStreamStatisticsImpl"/>
+    <Bug pattern="VO_VOLATILE_INCREMENT"/>
+  </Match>
+
 </FindBugsFilter>
diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml
index af8983e..f6d7023 100644
--- a/hadoop-tools/hadoop-aws/pom.xml
+++ b/hadoop-tools/hadoop-aws/pom.xml
@@ -201,6 +201,8 @@
                     <exclude>**/ITestMarkerToolRootOperations.java</exclude>
                     <!-- operations across the metastore -->
                     <exclude>**/ITestS3GuardDDBRootOperations.java</exclude>
+                    <!-- leave this until the end for better statistics -->
+                    <exclude>**/ITestAggregateIOStatistics.java</exclude>
                   </excludes>
                 </configuration>
               </execution>
@@ -250,6 +252,8 @@
                     <!-- operations across the metastore -->
                     <include>**/ITestS3AContractRootDir.java</include>
                     <include>**/ITestS3GuardDDBRootOperations.java</include>
+                    <!-- leave this until the end for better statistics -->
+                    <include>**/ITestAggregateIOStatistics.java</include>
                   </includes>
                 </configuration>
               </execution>
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
index a1c1d96..fcaec50 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
@@ -549,6 +549,13 @@ public final class Constants {
   @InterfaceStability.Unstable
   public static final String INPUT_FADV_RANDOM = "random";
 
+  /**
+   * Gauge name for the input policy : {@value}.
+   * This references an enum currently exclusive to the S3A stream.
+   */
+  public static final String STREAM_READ_GAUGE_INPUT_POLICY =
+      "stream_read_gauge_input_policy";
+
   @InterfaceAudience.Private
   @InterfaceStability.Unstable
   public static final String S3_CLIENT_FACTORY_IMPL =
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java
index 276961b..96d16e8 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java
@@ -23,16 +23,26 @@ import java.net.URI;
 
 import com.amazonaws.ClientConfiguration;
 import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.client.builder.AwsClientBuilder;
+import com.amazonaws.metrics.RequestMetricCollector;
 import com.amazonaws.services.s3.AmazonS3;
 import com.amazonaws.services.s3.AmazonS3Client;
+import com.amazonaws.services.s3.AmazonS3ClientBuilder;
 import com.amazonaws.services.s3.S3ClientOptions;
+import com.amazonaws.services.s3.internal.ServiceUtils;
+import com.amazonaws.util.AwsHostNameUtils;
+import com.amazonaws.util.RuntimeHttpUtils;
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.s3a.statistics.StatisticsFromAwsSdk;
+import org.apache.hadoop.fs.s3a.statistics.impl.AwsStatisticsCollector;
 
 import static org.apache.hadoop.fs.s3a.Constants.EXPERIMENTAL_AWS_INTERNAL_THROTTLING;
 import static org.apache.hadoop.fs.s3a.Constants.ENDPOINT;
@@ -41,7 +51,7 @@ import static org.apache.hadoop.fs.s3a.Constants.PATH_STYLE_ACCESS;
 
 /**
  * The default {@link S3ClientFactory} implementation.
- * This which calls the AWS SDK to configure and create an
+ * This calls the AWS SDK to configure and create an
  * {@link AmazonS3Client} that communicates with the S3 service.
  */
 @InterfaceAudience.Private
@@ -49,13 +59,30 @@ import static org.apache.hadoop.fs.s3a.Constants.PATH_STYLE_ACCESS;
 public class DefaultS3ClientFactory extends Configured
     implements S3ClientFactory {
 
-  protected static final Logger LOG = S3AFileSystem.LOG;
+  private static final String S3_SERVICE_NAME = "s3";
+  private static final String S3_SIGNER = "S3SignerType";
+  private static final String S3_V4_SIGNER = "AWSS3V4SignerType";
 
+  /**
+   * Subclasses refer to this.
+   */
+  protected static final Logger LOG =
+      LoggerFactory.getLogger(DefaultS3ClientFactory.class);
+
+  /**
+   * Create the client.
+   * <p>
+   * If the AWS stats are not null then a {@link AwsStatisticsCollector}.
+   * is created to bind to the two.
+   * <i>Important: until this binding works properly across regions,
+   * this should be null.</i>
+   */
   @Override
   public AmazonS3 createS3Client(URI name,
       final String bucket,
       final AWSCredentialsProvider credentials,
-      final String userAgentSuffix) throws IOException {
+      final String userAgentSuffix,
+      final StatisticsFromAwsSdk statisticsFromAwsSdk) throws IOException {
     Configuration conf = getConf();
     final ClientConfiguration awsConf = S3AUtils
         .createAwsConf(conf, bucket, Constants.AWS_SERVICE_IDENTIFIER_S3);
@@ -72,36 +99,124 @@ public class DefaultS3ClientFactory extends Configured
     if (!StringUtils.isEmpty(userAgentSuffix)) {
       awsConf.setUserAgentSuffix(userAgentSuffix);
     }
-    return configureAmazonS3Client(
-        newAmazonS3Client(credentials, awsConf), conf);
+    // optional metrics
+    RequestMetricCollector metrics = statisticsFromAwsSdk != null
+        ? new AwsStatisticsCollector(statisticsFromAwsSdk)
+        : null;
+
+    return newAmazonS3Client(
+        credentials,
+        awsConf,
+        metrics,
+        conf.getTrimmed(ENDPOINT, ""),
+        conf.getBoolean(PATH_STYLE_ACCESS, false));
   }
 
   /**
-   * Wrapper around constructor for {@link AmazonS3} client.
+   * Create an {@link AmazonS3} client.
    * Override this to provide an extended version of the client
    * @param credentials credentials to use
    * @param awsConf  AWS configuration
-   * @return  new AmazonS3 client
+   * @param metrics metrics collector or null
+   * @param endpoint endpoint string; may be ""
+   * @param pathStyleAccess enable path style access?
+   * @return new AmazonS3 client
    */
   protected AmazonS3 newAmazonS3Client(
-      AWSCredentialsProvider credentials, ClientConfiguration awsConf) {
-    return new AmazonS3Client(credentials, awsConf);
+      final AWSCredentialsProvider credentials,
+      final ClientConfiguration awsConf,
+      final RequestMetricCollector metrics,
+      final String endpoint,
+      final boolean pathStyleAccess) {
+    if (metrics != null) {
+      LOG.debug("Building S3 client using the SDK builder API");
+      return buildAmazonS3Client(credentials, awsConf, metrics, endpoint,
+          pathStyleAccess);
+    } else {
+      LOG.debug("Building S3 client using the SDK builder API");
+      return classicAmazonS3Client(credentials, awsConf, endpoint,
+          pathStyleAccess);
+    }
   }
 
   /**
-   * Configure S3 client from the Hadoop configuration.
-   *
+   * Use the (newer) Builder SDK to create a an AWS S3 client.
+   * <p>
+   * This has a more complex endpoint configuration in a
+   * way which does not yet work in this code in a way
+   * which doesn't trigger regressions. So it is only used
+   * when SDK metrics are supplied.
+   * @param credentials credentials to use
+   * @param awsConf  AWS configuration
+   * @param metrics metrics collector or null
+   * @param endpoint endpoint string; may be ""
+   * @param pathStyleAccess enable path style access?
+   * @return new AmazonS3 client
+   */
+  private AmazonS3 buildAmazonS3Client(
+      final AWSCredentialsProvider credentials,
+      final ClientConfiguration awsConf,
+      final RequestMetricCollector metrics,
+      final String endpoint,
+      final boolean pathStyleAccess) {
+    AmazonS3ClientBuilder b = AmazonS3Client.builder();
+    b.withCredentials(credentials);
+    b.withClientConfiguration(awsConf);
+    b.withPathStyleAccessEnabled(pathStyleAccess);
+    if (metrics != null) {
+      b.withMetricsCollector(metrics);
+    }
+
+    // endpoint set up is a PITA
+    //  client.setEndpoint("") is no longer available
+    AwsClientBuilder.EndpointConfiguration epr
+        = createEndpointConfiguration(endpoint, awsConf);
+    if (epr != null) {
+      // an endpoint binding was constructed: use it.
+      b.withEndpointConfiguration(epr);
+    }
+    final AmazonS3 client = b.build();
+    return client;
+  }
+
+  /**
+   * Wrapper around constructor for {@link AmazonS3} client.
+   * Override this to provide an extended version of the client.
+   * <p>
+   * This uses a deprecated constructor -it is currently
+   * the only one which works for us.
+   * @param credentials credentials to use
+   * @param awsConf  AWS configuration
+   * @param endpoint endpoint string; may be ""
+   * @param pathStyleAccess enable path style access?
+   * @return new AmazonS3 client
+   */
+  @SuppressWarnings("deprecation")
+  private AmazonS3 classicAmazonS3Client(
+      AWSCredentialsProvider credentials,
+      ClientConfiguration awsConf,
+      final String endpoint,
+      final boolean pathStyleAccess) {
+    final AmazonS3 client = new AmazonS3Client(credentials, awsConf);
+    return configureAmazonS3Client(client, endpoint, pathStyleAccess);
+  }
+
+  /**
+   * Configure classic S3 client.
+   * <p>
    * This includes: endpoint, Path Access and possibly other
    * options.
    *
-   * @param conf Hadoop configuration
+   * @param s3 S3 Client.
+   * @param endPoint s3 endpoint, may be empty
+   * @param pathStyleAccess enable path style access?
    * @return S3 client
    * @throws IllegalArgumentException if misconfigured
    */
-  private static AmazonS3 configureAmazonS3Client(AmazonS3 s3,
-      Configuration conf)
+  protected static AmazonS3 configureAmazonS3Client(AmazonS3 s3,
+      final String endPoint,
+      final boolean pathStyleAccess)
       throws IllegalArgumentException {
-    String endPoint = conf.getTrimmed(ENDPOINT, "");
     if (!endPoint.isEmpty()) {
       try {
         s3.setEndpoint(endPoint);
@@ -111,7 +226,7 @@ public class DefaultS3ClientFactory extends Configured
         throw new IllegalArgumentException(msg, e);
       }
     }
-    return applyS3ClientOptions(s3, conf);
+    return applyS3ClientOptions(s3, pathStyleAccess);
   }
 
   /**
@@ -119,23 +234,23 @@ public class DefaultS3ClientFactory extends Configured
    * the Hadoop configuration.
    * This is different from the general AWS configuration creation as
    * it is unique to S3 connections.
-   *
+   * <p>
    * 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}
+   * <p>
    * Enabling path-style access and a
    * region-specific endpoint switches the behavior to use URIs of the form
    * {@code http://s3-eu-west-1.amazonaws.com/bucketname}.
    * It is common to use this when connecting to private S3 servers, as it
    * avoids the need to play with DNS entries.
    * @param s3 S3 client
-   * @param conf Hadoop configuration
+   * @param pathStyleAccess enable path style access?
    * @return the S3 client
    */
-  private static AmazonS3 applyS3ClientOptions(AmazonS3 s3,
-      Configuration conf) {
-    final boolean pathStyleAccess = conf.getBoolean(PATH_STYLE_ACCESS, false);
+  protected static AmazonS3 applyS3ClientOptions(AmazonS3 s3,
+      final boolean pathStyleAccess) {
     if (pathStyleAccess) {
       LOG.debug("Enabling path style access!");
       s3.setS3ClientOptions(S3ClientOptions.builder()
@@ -144,4 +259,54 @@ public class DefaultS3ClientFactory extends Configured
     }
     return s3;
   }
+
+  /**
+   * Given an endpoint string, return an endpoint config, or null, if none
+   * is needed.
+   * <p>
+   * 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.
+   * <p>
+   * Note also that AWS signing is a mystery which nobody fully
+   * understands, especially given all problems surface in a
+   * "400 bad request" response, which, like all security systems,
+   * provides minimal diagnostics out of fear of leaking
+   * secrets.
+   *
+   * @param endpoint possibly null endpoint.
+   * @param awsConf config to build the URI from.
+   * @return a configuration for the S3 client builder.
+   */
+  @VisibleForTesting
+  public static AwsClientBuilder.EndpointConfiguration
+      createEndpointConfiguration(
+          final String endpoint, final ClientConfiguration awsConf) {
+    LOG.debug("Creating endpoint configuration for {}", endpoint);
+    if (endpoint == null || endpoint.isEmpty()) {
+      // the default endpoint...we should be using null at this point.
+      LOG.debug("Using default endpoint -no need to generate a configuration");
+      return null;
+    }
+
+    final URI epr = RuntimeHttpUtils.toUri(endpoint, awsConf);
+    LOG.debug("Endpoint URI = {}", epr);
+
+    String region;
+    if (!ServiceUtils.isS3USStandardEndpoint(endpoint)) {
+      LOG.debug("Endpoint {} is not the default; parsing", epr);
+      region = AwsHostNameUtils.parseRegion(
+          epr.getHost(),
+          S3_SERVICE_NAME);
+    } else {
+      // US-east, set region == null.
+      LOG.debug("Endpoint {} is the standard one; declare region as null", epr);
+      region = null;
+    }
+    LOG.debug("Region for endpoint {}, URI {} is determined as {}",
+        endpoint, epr, region);
+    return new AwsClientBuilder.EndpointConfiguration(endpoint, region);
+  }
 }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java
index 932c472..ddc4922 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.fs.s3a;
 
 import com.amazonaws.ClientConfiguration;
 import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.metrics.RequestMetricCollector;
 import com.amazonaws.services.s3.AmazonS3;
 
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -40,12 +41,21 @@ public class InconsistentS3ClientFactory extends DefaultS3ClientFactory {
    * Logs a warning that this is being done.
    * @param credentials credentials to use
    * @param awsConf  AWS configuration
+   * @param metrics metric collector
+   * @param endpoint AWS endpoint
+   * @param pathStyleAccess should path style access be supported?
    * @return an inconsistent client.
    */
   @Override
   protected AmazonS3 newAmazonS3Client(AWSCredentialsProvider credentials,
-      ClientConfiguration awsConf) {
+      ClientConfiguration awsConf,
+      final RequestMetricCollector metrics,
+      final String endpoint,
+      final boolean pathStyleAccess) {
     LOG.warn("** FAILURE INJECTION ENABLED.  Do not run in production! **");
-    return new InconsistentAmazonS3Client(credentials, awsConf, getConf());
+    InconsistentAmazonS3Client s3
+        = new InconsistentAmazonS3Client(credentials, awsConf, getConf());
+    configureAmazonS3Client(s3, endpoint, pathStyleAccess);
+    return s3;
   }
 }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java
index a79737d..19cd6c9 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java
@@ -30,17 +30,19 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.util.DurationInfo;
+import org.apache.hadoop.util.functional.CallableRaisingIOE;
 
 /**
  * Class to provide lambda expression invocation of AWS operations.
  *
  * The core retry logic is in
- * {@link #retryUntranslated(String, boolean, Retried, Operation)};
+ * {@link #retryUntranslated(String, boolean, Retried, CallableRaisingIOE)};
  * the other {@code retry() and retryUntranslated()} calls are wrappers.
  *
- * The static {@link #once(String, String, Operation)} and
+ * The static {@link #once(String, String, CallableRaisingIOE)} and
  * {@link #once(String, String, VoidOperation)} calls take an operation and
  * return it with AWS exceptions translated to IOEs of some form.
  *
@@ -56,11 +58,13 @@ import org.apache.hadoop.util.DurationInfo;
  * These callbacks can be used for reporting and incrementing statistics.
  *
  * The static {@link #quietly(String, String, VoidOperation)} and
- * {@link #quietlyEval(String, String, Operation)} calls exist to take any
- * operation and quietly catch and log at debug. The return value of
- * {@link #quietlyEval(String, String, Operation)} is a java 8 optional,
+ * {@link #quietlyEval(String, String, CallableRaisingIOE)} calls exist to
+ * take any operation and quietly catch and log at debug.
+ * The return value of {@link #quietlyEval(String, String, CallableRaisingIOE)}
+ * is a java 8 optional,
  * which can then be used in java8-expressions.
  */
+@InterfaceAudience.Private
 public class Invoker {
   private static final Logger LOG = LoggerFactory.getLogger(Invoker.class);
 
@@ -104,10 +108,11 @@ public class Invoker {
    * @throws IOException any IOE raised, or translated exception
    */
   @Retries.OnceTranslated
-  public static <T> T once(String action, String path, Operation<T> operation)
+  public static <T> T once(String action, String path,
+      CallableRaisingIOE<T> operation)
       throws IOException {
     try (DurationInfo ignored = new DurationInfo(LOG, false, "%s", action)) {
-      return operation.execute();
+      return operation.apply();
     } catch (AmazonClientException e) {
       throw S3AUtils.translateException(action, path, e);
     }
@@ -143,7 +148,7 @@ public class Invoker {
       Logger log,
       String action,
       String path,
-      Operation<T> operation) {
+      CallableRaisingIOE<T> operation) {
     try {
       once(action, path, operation);
     } catch (IOException e) {
@@ -280,7 +285,7 @@ public class Invoker {
   public <T> T retry(String action,
       @Nullable String path,
       boolean idempotent,
-      Operation<T> operation)
+      CallableRaisingIOE<T> operation)
       throws IOException {
 
     return retry(action, path, idempotent, retryCallback, operation);
@@ -288,7 +293,7 @@ public class Invoker {
 
   /**
    * Execute a function with retry processing.
-   * Uses {@link #once(String, String, Operation)} as the inner
+   * Uses {@link #once(String, String, CallableRaisingIOE)} as the inner
    * invocation mechanism before retry logic is performed.
    * @param <T> type of return value
    * @param action action to execute (used in error messages)
@@ -306,7 +311,7 @@ public class Invoker {
       @Nullable String path,
       boolean idempotent,
       Retried retrying,
-      Operation<T> operation)
+      CallableRaisingIOE<T> operation)
       throws IOException {
     return retryUntranslated(
         toDescription(action, path),
@@ -317,7 +322,7 @@ public class Invoker {
 
   /**
    * Execute a function with retry processing when doRetry=true, else just once.
-   * Uses {@link #once(String, String, Operation)} as the inner
+   * Uses {@link #once(String, String, CallableRaisingIOE)} as the inner
    * invocation mechanism before retry logic is performed.
    * @param <T> type of return value
    * @param doRetry true if retries should be performed
@@ -337,7 +342,7 @@ public class Invoker {
       @Nullable String path,
       boolean idempotent,
       Retried retrying,
-      Operation<T> operation)
+      CallableRaisingIOE<T> operation)
       throws IOException {
     if (doRetry) {
       return retryUntranslated(
@@ -366,7 +371,7 @@ public class Invoker {
   public <T> T retryUntranslated(
       String text,
       boolean idempotent,
-      Operation<T> operation) throws IOException {
+      CallableRaisingIOE<T> operation) throws IOException {
     return retryUntranslated(text, idempotent,
         retryCallback, operation);
   }
@@ -391,7 +396,7 @@ public class Invoker {
       String text,
       boolean idempotent,
       Retried retrying,
-      Operation<T> operation) throws IOException {
+      CallableRaisingIOE<T> operation) throws IOException {
 
     Preconditions.checkArgument(retrying != null, "null retrying argument");
     int retryCount = 0;
@@ -404,7 +409,7 @@ public class Invoker {
           LOG.debug("retry #{}", retryCount);
         }
         // execute the operation, returning if successful
-        return operation.execute();
+        return operation.apply();
       } catch (IOException | SdkBaseException e) {
         caught = e;
       }
@@ -490,7 +495,7 @@ public class Invoker {
    */
   public static <T> Optional<T> quietlyEval(String action,
       String path,
-      Operation<T> operation) {
+      CallableRaisingIOE<T> operation) {
     try {
       return Optional.of(once(action, path, operation));
     } catch (Exception e) {
@@ -511,15 +516,6 @@ public class Invoker {
   }
 
   /**
-   * Arbitrary operation throwing an IOException.
-   * @param <T> return type
-   */
-  @FunctionalInterface
-  public interface Operation<T> {
-    T execute() throws IOException;
-  }
-
-  /**
    * Void operation which may raise an IOException.
    */
   @FunctionalInterface
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java
index 8b35eda..3cb3d5d 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java
@@ -38,8 +38,11 @@ import org.apache.hadoop.fs.s3a.s3guard.DirListingMetadata;
 import org.apache.hadoop.fs.s3a.s3guard.MetadataStoreListFilesIterator;
 import org.apache.hadoop.fs.s3a.s3guard.PathMetadata;
 import org.apache.hadoop.fs.s3a.s3guard.S3Guard;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
+import org.apache.hadoop.util.functional.RemoteIterators;
 
-import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
 import org.slf4j.Logger;
 
 import java.io.FileNotFoundException;
@@ -48,7 +51,6 @@ import java.time.Instant;
 import java.time.OffsetDateTime;
 import java.time.ZoneOffset;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
@@ -57,6 +59,7 @@ import java.util.Map;
 import java.util.NoSuchElementException;
 import java.util.Set;
 import java.util.concurrent.CompletableFuture;
+import java.util.StringJoiner;
 
 import static org.apache.hadoop.fs.impl.FutureIOSupport.awaitFuture;
 import static org.apache.hadoop.fs.s3a.Constants.S3N_FOLDER_SUFFIX;
@@ -67,6 +70,12 @@ import static org.apache.hadoop.fs.s3a.S3AUtils.objectRepresentsDirectory;
 import static org.apache.hadoop.fs.s3a.S3AUtils.stringify;
 import static org.apache.hadoop.fs.s3a.S3AUtils.translateException;
 import static org.apache.hadoop.fs.s3a.auth.RoleModel.pathToKey;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_CONTINUE_LIST_REQUEST;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_LIST_REQUEST;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore;
+import static org.apache.hadoop.util.functional.RemoteIterators.filteringRemoteIterator;
+import static org.apache.hadoop.util.functional.RemoteIterators.remoteIteratorFromArray;
+import static org.apache.hadoop.util.functional.RemoteIterators.remoteIteratorFromSingleton;
 
 /**
  * Place for the S3A listing classes; keeps all the small classes under control.
@@ -96,11 +105,14 @@ public class Listing extends AbstractStoreOperation {
    * @param acceptor the file status acceptor
    * @return the file status iterator
    */
-  ProvidedFileStatusIterator createProvidedFileStatusIterator(
+  RemoteIterator<S3AFileStatus> createProvidedFileStatusIterator(
       S3AFileStatus[] fileStatuses,
       PathFilter filter,
       FileStatusAcceptor acceptor) {
-    return new ProvidedFileStatusIterator(fileStatuses, filter, acceptor);
+    return filteringRemoteIterator(
+        remoteIteratorFromArray(fileStatuses),
+        status ->
+            filter.accept(status.getPath()) && acceptor.accept(status));
   }
 
   /**
@@ -109,11 +121,11 @@ public class Listing extends AbstractStoreOperation {
    * @return the file status iterator.
    */
   @VisibleForTesting
-  public static ProvidedFileStatusIterator toProvidedFileStatusIterator(
+  public static RemoteIterator<S3AFileStatus> toProvidedFileStatusIterator(
           S3AFileStatus[] fileStatuses) {
-    return new ProvidedFileStatusIterator(fileStatuses,
-            ACCEPT_ALL,
-            Listing.ACCEPT_ALL_BUT_S3N);
+    return filteringRemoteIterator(
+        remoteIteratorFromArray(fileStatuses),
+        Listing.ACCEPT_ALL_BUT_S3N::accept);
   }
 
   /**
@@ -185,9 +197,11 @@ public class Listing extends AbstractStoreOperation {
    * @return a new remote iterator
    */
   @VisibleForTesting
-  public LocatedFileStatusIterator createLocatedFileStatusIterator(
+  public RemoteIterator<S3ALocatedFileStatus> createLocatedFileStatusIterator(
       RemoteIterator<S3AFileStatus> statusIterator) {
-    return new LocatedFileStatusIterator(statusIterator);
+    return RemoteIterators.mappingRemoteIterator(
+        statusIterator,
+        listingOperationCallbacks::toLocatedFileStatus);
   }
 
   /**
@@ -199,11 +213,28 @@ public class Listing extends AbstractStoreOperation {
    * @return a new remote iterator.
    */
   @VisibleForTesting
-  TombstoneReconcilingIterator createTombstoneReconcilingIterator(
-      RemoteIterator<S3ALocatedFileStatus> iterator, Set<Path> tombstones) {
-    return new TombstoneReconcilingIterator(iterator, tombstones);
+  RemoteIterator<S3ALocatedFileStatus> createTombstoneReconcilingIterator(
+      RemoteIterator<S3ALocatedFileStatus> iterator,
+      @Nullable Set<Path> tombstones) {
+    if (tombstones == null || tombstones.isEmpty()) {
+      // no need to filter.
+      return iterator;
+    } else {
+      return filteringRemoteIterator(
+          iterator,
+          candidate -> !tombstones.contains(candidate.getPath()));
+    }
   }
 
+  /**
+   * Create a remote iterator from a single status entry.
+   * @param status status
+   * @return iterator.
+   */
+  public RemoteIterator<S3ALocatedFileStatus> createSingleStatusIterator(
+      S3ALocatedFileStatus status) {
+    return remoteIteratorFromSingleton(status);
+  }
 
   /**
    * List files under a path assuming the path to be a directory.
@@ -369,7 +400,7 @@ public class Listing extends AbstractStoreOperation {
                     allowAuthoritative);
     // In auth mode return directly with auth flag.
     if (allowAuthoritative && dirMeta != null && dirMeta.isAuthoritative()) {
-      ProvidedFileStatusIterator mfsItr = createProvidedFileStatusIterator(
+      RemoteIterator<S3AFileStatus> mfsItr = createProvidedFileStatusIterator(
               S3Guard.dirMetaToStatuses(dirMeta),
               ACCEPT_ALL,
               Listing.ACCEPT_ALL_BUT_S3N);
@@ -430,105 +461,6 @@ public class Listing extends AbstractStoreOperation {
   }
 
   /**
-   * A remote iterator which only iterates over a single `LocatedFileStatus`
-   * value.
-   *
-   * If the status value is null, the iterator declares that it has no data.
-   * This iterator is used to handle {@link S3AFileSystem#listStatus(Path)}
-   * calls where the path handed in refers to a file, not a directory:
-   * this is the iterator returned.
-   */
-  static final class SingleStatusRemoteIterator
-      implements RemoteIterator<S3ALocatedFileStatus> {
-
-    /**
-     * The status to return; set to null after the first iteration.
-     */
-    private S3ALocatedFileStatus status;
-
-    /**
-     * Constructor.
-     * @param status status value: may be null, in which case
-     * the iterator is empty.
-     */
-    SingleStatusRemoteIterator(S3ALocatedFileStatus status) {
-      this.status = status;
-    }
-
-    /**
-     * {@inheritDoc}
-     * @return true if there is a file status to return: this is always false
-     * for the second iteration, and may be false for the first.
-     * @throws IOException never
-     */
-    @Override
-    public boolean hasNext() throws IOException {
-      return status != null;
-    }
-
-    /**
-     * {@inheritDoc}
-     * @return the non-null status element passed in when the instance was
-     * constructed, if it ha not already been retrieved.
-     * @throws IOException never
-     * @throws NoSuchElementException if this is the second call, or it is
-     * the first call and a null {@link LocatedFileStatus} entry was passed
-     * to the constructor.
-     */
-    @Override
-    public S3ALocatedFileStatus next() throws IOException {
-      if (hasNext()) {
-        S3ALocatedFileStatus s = this.status;
-        status = null;
-        return s;
-      } else {
-        throw new NoSuchElementException();
-      }
-    }
-  }
-
-  /**
-   * This wraps up a provided non-null list of file status as a remote iterator.
-   *
-   * It firstly filters the provided list and later {@link #next} call will get
-   * from the filtered list. This suffers from scalability issues if the
-   * provided list is too large.
-   *
-   * There is no remote data to fetch.
-   */
-  static class ProvidedFileStatusIterator
-      implements RemoteIterator<S3AFileStatus> {
-    private final ArrayList<S3AFileStatus> filteredStatusList;
-    private int index = 0;
-
-    ProvidedFileStatusIterator(S3AFileStatus[] fileStatuses, PathFilter filter,
-        FileStatusAcceptor acceptor) {
-      Preconditions.checkArgument(fileStatuses != null, "Null status list!");
-
-      filteredStatusList = new ArrayList<>(fileStatuses.length);
-      for (S3AFileStatus status : fileStatuses) {
-        if (filter.accept(status.getPath()) && acceptor.accept(status)) {
-          filteredStatusList.add(status);
-        }
-      }
-      filteredStatusList.trimToSize();
-    }
-
-    @Override
-    public boolean hasNext() throws IOException {
-      return index < filteredStatusList.size();
-    }
-
-    @Override
-    public S3AFileStatus next() throws IOException {
-      if (!hasNext()) {
-        throw new NoSuchElementException();
-      }
-      return filteredStatusList.get(index++);
-    }
-  }
-
-  /**
    * Wraps up object listing into a remote iterator which will ask for more
    * listing data if needed.
    *
@@ -555,7 +487,7 @@ public class Listing extends AbstractStoreOperation {
    * Thread safety: None.
    */
   class FileStatusListingIterator
-      implements RemoteIterator<S3AFileStatus> {
+      implements RemoteIterator<S3AFileStatus>, IOStatisticsSource {
 
     /** Source of objects. */
     private final ObjectListingIterator source;
@@ -758,6 +690,23 @@ public class Listing extends AbstractStoreOperation {
     public int getBatchSize() {
       return batchSize;
     }
+
+    /**
+     * Return any IOStatistics provided by the underlying stream.
+     * @return IO stats from the inner stream.
+     */
+    @Override
+    public IOStatistics getIOStatistics() {
+      return source.getIOStatistics();
+    }
+
+    @Override
+    public String toString() {
+      return new StringJoiner(", ",
+          FileStatusListingIterator.class.getSimpleName() + "[", "]")
+          .add(source.toString())
+          .toString();
+    }
   }
 
   /**
@@ -780,7 +729,8 @@ public class Listing extends AbstractStoreOperation {
    *
    * Thread safety: none.
    */
-  class ObjectListingIterator implements RemoteIterator<S3ListResult> {
+  class ObjectListingIterator implements RemoteIterator<S3ListResult>,
+      IOStatisticsSource {
 
     /** The path listed. */
     private final Path listPath;
@@ -805,6 +755,8 @@ public class Listing extends AbstractStoreOperation {
      */
     private int maxKeys;
 
+    private final IOStatisticsStore iostats;
+
     /**
      * Future to store current batch listing result.
      */
@@ -828,10 +780,14 @@ public class Listing extends AbstractStoreOperation {
         S3ListRequest request) throws IOException {
       this.listPath = listPath;
       this.maxKeys = listingOperationCallbacks.getMaxKeys();
-      this.s3ListResultFuture = listingOperationCallbacks
-              .listObjectsAsync(request);
       this.request = request;
       this.objectsPrev = null;
+      this.iostats = iostatisticsStore()
+          .withDurationTracking(OBJECT_LIST_REQUEST)
+          .withDurationTracking(OBJECT_CONTINUE_LIST_REQUEST)
+          .build();
+      this.s3ListResultFuture = listingOperationCallbacks
+          .listObjectsAsync(request, iostats);
     }
 
     /**
@@ -895,7 +851,7 @@ public class Listing extends AbstractStoreOperation {
         LOG.debug("[{}], Requesting next {} objects under {}",
                 listingCount, maxKeys, listPath);
         s3ListResultFuture = listingOperationCallbacks
-                .continueListObjectsAsync(request, objects);
+                .continueListObjectsAsync(request, objects, iostats);
       }
     }
 
@@ -903,7 +859,13 @@ public class Listing extends AbstractStoreOperation {
     public String toString() {
       return "Object listing iterator against " + listPath
           + "; listing count "+ listingCount
-          + "; isTruncated=" + objects.isTruncated();
+          + "; isTruncated=" + objects.isTruncated()
+          + "; " + iostats;
+    }
+
+    @Override
+    public IOStatistics getIOStatistics() {
+      return iostats;
     }
 
     /**
@@ -967,89 +929,6 @@ public class Listing extends AbstractStoreOperation {
   }
 
   /**
-   * Take a remote iterator over a set of {@link FileStatus} instances and
-   * return a remote iterator of {@link LocatedFileStatus} instances.
-   */
-  class LocatedFileStatusIterator
-      implements RemoteIterator<S3ALocatedFileStatus> {
-    private final RemoteIterator<S3AFileStatus> statusIterator;
-
-    /**
-     * Constructor.
-     * @param statusIterator an iterator over the remote status entries
-     */
-    LocatedFileStatusIterator(RemoteIterator<S3AFileStatus> statusIterator) {
-      this.statusIterator = statusIterator;
-    }
-
-    @Override
-    public boolean hasNext() throws IOException {
-      return statusIterator.hasNext();
-    }
-
-    @Override
-    public S3ALocatedFileStatus next() throws IOException {
-      return listingOperationCallbacks
-              .toLocatedFileStatus(statusIterator.next());
-    }
-  }
-
-  /**
-   * Wraps another iterator and filters out files that appear in the provided
-   * set of tombstones.  Will read ahead in the iterator when necessary to
-   * ensure that emptiness is detected early enough if only deleted objects
-   * remain in the source iterator.
-   */
-  static class TombstoneReconcilingIterator implements
-      RemoteIterator<S3ALocatedFileStatus> {
-    private S3ALocatedFileStatus next = null;
-    private final RemoteIterator<S3ALocatedFileStatus> iterator;
-    private final Set<Path> tombstones;
-
-    /**
-     * @param iterator Source iterator to filter
-     * @param tombstones set of tombstone markers to filter out of results
-     */
-    TombstoneReconcilingIterator(RemoteIterator<S3ALocatedFileStatus>
-        iterator, Set<Path> tombstones) {
-      this.iterator = iterator;
-      if (tombstones != null) {
-        this.tombstones = tombstones;
-      } else {
-        this.tombstones = Collections.emptySet();
-      }
-    }
-
-    private boolean fetch() throws IOException {
-      while (next == null && iterator.hasNext()) {
-        S3ALocatedFileStatus candidate = iterator.next();
-        if (!tombstones.contains(candidate.getPath())) {
-          next = candidate;
-          return true;
-        }
-      }
-      return false;
-    }
-
-    public boolean hasNext() throws IOException {
-      if (next != null) {
-        return true;
-      }
-      return fetch();
-    }
-
-    public S3ALocatedFileStatus next() throws IOException {
-      if (hasNext()) {
-        S3ALocatedFileStatus result = next;
-        next = null;
-        fetch();
-        return result;
-      }
-      throw new NoSuchElementException();
-    }
-  }
-
-  /**
    * Accept all entries except those which map to S3N pseudo directory markers.
    */
   static class AcceptAllButS3nDirs implements FileStatusAcceptor {
@@ -1117,4 +996,9 @@ public class Listing extends AbstractStoreOperation {
     }
   }
 
+  public static RemoteIterator<LocatedFileStatus> toLocatedFileStatusIterator(
+      RemoteIterator<? extends LocatedFileStatus> iterator) {
+    return (RemoteIterator < LocatedFileStatus >) iterator;
+  }
+
 }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
index b17412f..0fdad21 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
@@ -20,6 +20,8 @@ package org.apache.hadoop.fs.s3a;
 
 import java.io.IOException;
 import java.io.OutputStream;
+import java.time.Duration;
+import java.time.Instant;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Locale;
@@ -49,10 +51,15 @@ import org.apache.hadoop.fs.PathIOException;
 import org.apache.hadoop.fs.StreamCapabilities;
 import org.apache.hadoop.fs.s3a.commit.CommitConstants;
 import org.apache.hadoop.fs.s3a.commit.PutTracker;
+import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsLogging;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
 import org.apache.hadoop.util.Progressable;
 
 import static org.apache.hadoop.fs.s3a.S3AUtils.*;
 import static org.apache.hadoop.fs.s3a.Statistic.*;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.emptyStatistics;
 import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
 
 /**
@@ -67,7 +74,7 @@ import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 class S3ABlockOutputStream extends OutputStream implements
-    StreamCapabilities {
+    StreamCapabilities, IOStatisticsSource {
 
   private static final Logger LOG =
       LoggerFactory.getLogger(S3ABlockOutputStream.class);
@@ -81,6 +88,9 @@ class S3ABlockOutputStream extends OutputStream implements
   /** Size of all blocks. */
   private final int blockSize;
 
+  /** IO Statistics. */
+  private final IOStatistics iostatistics;
+
   /** Total bytes for uploads submitted so far. */
   private long bytesSubmitted;
 
@@ -109,7 +119,7 @@ class S3ABlockOutputStream extends OutputStream implements
   private long blockCount = 0;
 
   /** Statistics to build up. */
-  private final S3AInstrumentation.OutputStreamStatistics statistics;
+  private final BlockOutputStreamStatistics statistics;
 
   /**
    * Write operation helper; encapsulation of the filesystem operations.
@@ -146,7 +156,7 @@ class S3ABlockOutputStream extends OutputStream implements
       Progressable progress,
       long blockSize,
       S3ADataBlocks.BlockFactory blockFactory,
-      S3AInstrumentation.OutputStreamStatistics statistics,
+      BlockOutputStreamStatistics statistics,
       WriteOperationHelper writeOperationHelper,
       PutTracker putTracker)
       throws IOException {
@@ -155,6 +165,10 @@ class S3ABlockOutputStream extends OutputStream implements
     this.blockFactory = blockFactory;
     this.blockSize = (int) blockSize;
     this.statistics = statistics;
+    // test instantiations may not provide statistics;
+    this.iostatistics = statistics != null
+        ? statistics.getIOStatistics()
+        : emptyStatistics();
     this.writeOperationHelper = writeOperationHelper;
     this.putTracker = putTracker;
     Preconditions.checkArgument(blockSize >= Constants.MULTIPART_MIN_SIZE,
@@ -282,6 +296,7 @@ class S3ABlockOutputStream extends OutputStream implements
     if (len == 0) {
       return;
     }
+    statistics.writeBytes(len);
     S3ADataBlocks.DataBlock block = createBlockIfNeeded();
     int written = block.write(source, offset, len);
     int remainingCapacity = block.remainingCapacity();
@@ -382,7 +397,8 @@ class S3ABlockOutputStream extends OutputStream implements
         // then complete the operation
         if (putTracker.aboutToComplete(multiPartUpload.getUploadId(),
             partETags,
-            bytes)) {
+            bytes,
+            iostatistics)) {
           multiPartUpload.complete(partETags);
         } else {
           LOG.info("File {} will be visible when the job is committed", key);
@@ -432,10 +448,9 @@ class S3ABlockOutputStream extends OutputStream implements
         writeOperationHelper.createPutObjectRequest(key, uploadData.getFile())
         : writeOperationHelper.createPutObjectRequest(key,
             uploadData.getUploadStream(), size);
-    long transferQueueTime = now();
     BlockUploadProgress callback =
         new BlockUploadProgress(
-            block, progressListener, transferQueueTime);
+            block, progressListener,  now());
     putObjectRequest.setGeneralProgressListener(callback);
     statistics.blockUploadQueued(size);
     ListenableFuture<PutObjectResult> putObjectResult =
@@ -473,6 +488,8 @@ class S3ABlockOutputStream extends OutputStream implements
     if (block != null) {
       sb.append(", activeBlock=").append(block);
     }
+    sb.append(" Statistics=")
+        .append(IOStatisticsLogging.ioStatisticsSourceToString(this));
     sb.append('}');
     return sb.toString();
   }
@@ -485,15 +502,15 @@ class S3ABlockOutputStream extends OutputStream implements
    * Current time in milliseconds.
    * @return time
    */
-  private long now() {
-    return System.currentTimeMillis();
+  private Instant now() {
+    return Instant.now();
   }
 
   /**
    * Get the statistics for this stream.
    * @return stream statistics
    */
-  S3AInstrumentation.OutputStreamStatistics getStatistics() {
+  BlockOutputStreamStatistics getStatistics() {
     return statistics;
   }
 
@@ -520,11 +537,20 @@ class S3ABlockOutputStream extends OutputStream implements
     case StreamCapabilities.HSYNC:
       return false;
 
+      // yes, we do statistics.
+    case StreamCapabilities.IOSTATISTICS:
+      return true;
+
     default:
       return false;
     }
   }
 
+  @Override
+  public IOStatistics getIOStatistics() {
+    return iostatistics;
+  }
+
   /**
    * Multiple partition upload.
    */
@@ -636,10 +662,9 @@ class S3ABlockOutputStream extends OutputStream implements
         noteUploadFailure(e);
         throw e;
       }
-      long transferQueueTime = now();
       BlockUploadProgress callback =
           new BlockUploadProgress(
-              block, progressListener, transferQueueTime);
+              block, progressListener, now());
       request.setGeneralProgressListener(callback);
       statistics.blockUploadQueued(block.dataSize());
       ListenableFuture<PartETag> partETagFuture =
@@ -754,8 +779,8 @@ class S3ABlockOutputStream extends OutputStream implements
   private final class BlockUploadProgress implements ProgressListener {
     private final S3ADataBlocks.DataBlock block;
     private final ProgressListener nextListener;
-    private final long transferQueueTime;
-    private long transferStartTime;
+    private final Instant transferQueueTime;
+    private Instant transferStartTime;
 
     /**
      * Track the progress of a single block upload.
@@ -766,7 +791,7 @@ class S3ABlockOutputStream extends OutputStream implements
      */
     private BlockUploadProgress(S3ADataBlocks.DataBlock block,
         ProgressListener nextListener,
-        long transferQueueTime) {
+        Instant transferQueueTime) {
       this.block = block;
       this.transferQueueTime = transferQueueTime;
       this.nextListener = nextListener;
@@ -787,17 +812,22 @@ class S3ABlockOutputStream extends OutputStream implements
 
       case TRANSFER_PART_STARTED_EVENT:
         transferStartTime = now();
-        statistics.blockUploadStarted(transferStartTime - transferQueueTime,
+        statistics.blockUploadStarted(
+            Duration.between(transferQueueTime, transferStartTime),
             size);
         incrementWriteOperations();
         break;
 
       case TRANSFER_PART_COMPLETED_EVENT:
-        statistics.blockUploadCompleted(now() - transferStartTime, size);
+        statistics.blockUploadCompleted(
+            Duration.between(transferStartTime, now()),
+            size);
         break;
 
       case TRANSFER_PART_FAILED_EVENT:
-        statistics.blockUploadFailed(now() - transferStartTime, size);
+        statistics.blockUploadFailed(
+            Duration.between(transferStartTime, now()),
+            size);
         LOG.warn("Transfer failure of block {}", block);
         break;
 
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java
index fa38f24..2503177 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.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.fs.FSExceptionMessages;
+import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics;
 import org.apache.hadoop.util.DirectBufferPool;
 
 import static org.apache.hadoop.fs.s3a.S3ADataBlocks.DataBlock.DestState.*;
@@ -180,7 +181,7 @@ final class S3ADataBlocks {
      * @return a new block.
      */
     abstract DataBlock create(long index, int limit,
-        S3AInstrumentation.OutputStreamStatistics statistics)
+        BlockOutputStreamStatistics statistics)
         throws IOException;
 
     /**
@@ -210,10 +211,10 @@ final class S3ADataBlocks {
 
     private volatile DestState state = Writing;
     protected final long index;
-    protected final S3AInstrumentation.OutputStreamStatistics statistics;
+    private final BlockOutputStreamStatistics statistics;
 
     protected DataBlock(long index,
-        S3AInstrumentation.OutputStreamStatistics statistics) {
+        BlockOutputStreamStatistics statistics) {
       this.index = index;
       this.statistics = statistics;
     }
@@ -372,6 +373,10 @@ final class S3ADataBlocks {
         statistics.blockReleased();
       }
     }
+
+    protected BlockOutputStreamStatistics getStatistics() {
+      return statistics;
+    }
   }
 
   // ====================================================================
@@ -387,7 +392,7 @@ final class S3ADataBlocks {
 
     @Override
     DataBlock create(long index, int limit,
-        S3AInstrumentation.OutputStreamStatistics statistics)
+        BlockOutputStreamStatistics statistics)
         throws IOException {
       return new ByteArrayBlock(0, limit, statistics);
     }
@@ -432,7 +437,7 @@ final class S3ADataBlocks {
 
     ByteArrayBlock(long index,
         int limit,
-        S3AInstrumentation.OutputStreamStatistics statistics) {
+        BlockOutputStreamStatistics statistics) {
       super(index, statistics);
       this.limit = limit;
       buffer = new S3AByteArrayOutputStream(limit);
@@ -510,7 +515,7 @@ final class S3ADataBlocks {
 
     @Override
     ByteBufferBlock create(long index, int limit,
-        S3AInstrumentation.OutputStreamStatistics statistics)
+        BlockOutputStreamStatistics statistics)
         throws IOException {
       return new ByteBufferBlock(index, limit, statistics);
     }
@@ -560,7 +565,7 @@ final class S3ADataBlocks {
        */
       ByteBufferBlock(long index,
           int bufferSize,
-          S3AInstrumentation.OutputStreamStatistics statistics) {
+          BlockOutputStreamStatistics statistics) {
         super(index, statistics);
         this.bufferSize = bufferSize;
         blockBuffer = requestBuffer(bufferSize);
@@ -805,7 +810,7 @@ final class S3ADataBlocks {
     @Override
     DataBlock create(long index,
         int limit,
-        S3AInstrumentation.OutputStreamStatistics statistics)
+        BlockOutputStreamStatistics statistics)
         throws IOException {
       File destFile = getOwner()
           .createTmpFileForWrite(String.format("s3ablock-%04d-", index),
@@ -829,7 +834,7 @@ final class S3ADataBlocks {
     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 a02947b..f60ff75 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
@@ -116,11 +116,14 @@ import org.apache.hadoop.fs.s3a.impl.S3AMultipartUploaderBuilder;
 import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum;
 import org.apache.hadoop.fs.s3a.impl.StoreContext;
 import org.apache.hadoop.fs.s3a.impl.StoreContextBuilder;
-import org.apache.hadoop.fs.s3a.impl.statistics.S3AMultipartUploaderStatisticsImpl;
 import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
 import org.apache.hadoop.fs.s3a.select.InternalSelectConstants;
 import org.apache.hadoop.fs.s3a.tools.MarkerToolOperations;
 import org.apache.hadoop.fs.s3a.tools.MarkerToolOperationsImpl;
+import org.apache.hadoop.fs.statistics.DurationTracker;
+import org.apache.hadoop.fs.statistics.DurationTrackerFactory;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.token.DelegationTokenIssuer;
@@ -158,6 +161,11 @@ import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
 import org.apache.hadoop.fs.s3a.s3guard.PathMetadata;
 import org.apache.hadoop.fs.s3a.s3guard.S3Guard;
 import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider;
+import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics;
+import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics;
+import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
+import org.apache.hadoop.fs.s3a.statistics.StatisticsFromAwsSdk;
+import org.apache.hadoop.fs.s3a.statistics.impl.BondedS3AStatisticsContext;
 import org.apache.hadoop.fs.s3native.S3xLoginHelper;
 import org.apache.hadoop.io.retry.RetryPolicies;
 import org.apache.hadoop.fs.store.EtagChecksum;
@@ -169,10 +177,12 @@ import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.SemaphoredDelegatingExecutor;
 import org.apache.hadoop.util.concurrent.HadoopExecutors;
 
+import static java.util.Objects.requireNonNull;
 import static org.apache.hadoop.fs.impl.AbstractFSBuilderImpl.rejectUnknownMandatoryKeys;
 import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs;
 import static org.apache.hadoop.fs.s3a.Constants.*;
 import static org.apache.hadoop.fs.s3a.Invoker.*;
+import static org.apache.hadoop.fs.s3a.Listing.toLocatedFileStatusIterator;
 import static org.apache.hadoop.fs.s3a.S3AUtils.*;
 import static org.apache.hadoop.fs.s3a.Statistic.*;
 import static org.apache.commons.lang3.StringUtils.isNotEmpty;
@@ -184,12 +194,20 @@ import static org.apache.hadoop.fs.s3a.commit.CommitConstants.FS_S3A_COMMITTER_A
 import static org.apache.hadoop.fs.s3a.commit.CommitConstants.FS_S3A_COMMITTER_STAGING_ABORT_PENDING_UPLOADS;
 import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit;
 import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletionIgnoringExceptions;
+import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isObjectNotFound;
 import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isUnknownBucket;
+import static org.apache.hadoop.fs.s3a.impl.InternalConstants.AWS_SDK_METRICS_ENABLED;
 import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_404;
 import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.fixBucketRegion;
 import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.logDnsLookup;
 import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.dirMetaToStatuses;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_CONTINUE_LIST_REQUEST;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_LIST_REQUEST;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.pairedTrackerFactory;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfInvocation;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfOperation;
 import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
+import static org.apache.hadoop.util.functional.RemoteIterators.typeCastingRemoteIterator;
 
 /**
  * The core S3A Filesystem implementation.
@@ -207,7 +225,7 @@ import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
 @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.
    */
@@ -259,9 +277,13 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * is no encryption.
    */
   private EncryptionSecrets encryptionSecrets = new EncryptionSecrets();
+  /** The core instrumentation. */
   private S3AInstrumentation instrumentation;
-  private final S3AStorageStatistics storageStatistics =
-      createStorageStatistics();
+  /** Accessors to statistics for this FS. */
+  private S3AStatisticsContext statisticsContext;
+  /** Storage Statistics Bonded to the instrumentation. */
+  private S3AStorageStatistics storageStatistics;
+
   private long readAhead;
   private S3AInputPolicy inputPolicy;
   private ChangeDetectionPolicy changeDetectionPolicy;
@@ -368,6 +390,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
 
       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();
@@ -377,7 +400,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
 
       s3guardInvoker = new Invoker(new S3GuardExistsRetryPolicy(getConf()),
           onRetry);
-      writeHelper = new WriteOperationHelper(this, getConf());
+      writeHelper = new WriteOperationHelper(this, getConf(),
+          statisticsContext);
 
       failOnMetadataWriteError = conf.getBoolean(FAIL_ON_METADATA_WRITE_ERROR,
           FAIL_ON_METADATA_WRITE_ERROR_DEFAULT);
@@ -532,6 +556,33 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   }
 
   /**
+   * Initialize the statistics binding.
+   * This is done by creating an {@code IntegratedS3AStatisticsContext}
+   * with callbacks to get the FS's instrumentation and FileSystem.statistics
+   * field; the latter may change after {@link #initialize(URI, Configuration)},
+   * so needs to be dynamically adapted.
+   * Protected so that (mock) subclasses can replace it with a
+   * different statistics binding, if desired.
+   */
+  protected void initializeStatisticsBinding() {
+    storageStatistics = createStorageStatistics(
+        requireNonNull(getIOStatistics()));
+    statisticsContext = new BondedS3AStatisticsContext(
+        new BondedS3AStatisticsContext.S3AFSStatisticsSource() {
+
+          @Override
+          public S3AInstrumentation getInstrumentation() {
+            return S3AFileSystem.this.getInstrumentation();
+          }
+
+          @Override
+          public Statistics getInstanceStatistics() {
+            return S3AFileSystem.this.statistics;
+          }
+        });
+  }
+
+  /**
    * Initialize the thread pool.
    * This must be re-invoked after replacing the S3Client during test
    * runs.
@@ -565,13 +616,15 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
 
   /**
    * Create the storage statistics or bind to an existing one.
-   * @return a storage statistics instance.
+   * @param ioStatistics IOStatistics to build the storage statistics from.
+   * @return a storage statistics instance; expected to be that of the FS.
    */
-  protected static S3AStorageStatistics createStorageStatistics() {
+  protected static S3AStorageStatistics createStorageStatistics(
+      final IOStatistics ioStatistics) {
     return (S3AStorageStatistics)
         GlobalStorageStatistics.INSTANCE
             .put(S3AStorageStatistics.NAME,
-                () -> new S3AStorageStatistics());
+                () -> new S3AStorageStatistics(ioStatistics));
   }
 
   /**
@@ -610,6 +663,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * Get S3A Instrumentation. For test purposes.
    * @return this instance's instrumentation.
    */
+  @VisibleForTesting
   public S3AInstrumentation getInstrumentation() {
     return instrumentation;
   }
@@ -677,8 +731,16 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
         S3_CLIENT_FACTORY_IMPL, DEFAULT_S3_CLIENT_FACTORY_IMPL,
         S3ClientFactory.class);
 
+    StatisticsFromAwsSdk awsStats = null;
+    //  TODO: HADOOP-16830 when the S3 client building code works
+    //   with different regions,
+    //   then non-null stats can be passed in here.
+    if (AWS_SDK_METRICS_ENABLED) {
+      awsStats = statisticsContext.newStatisticsFromAwsSdk();
+    }
+
     s3 = ReflectionUtils.newInstance(s3ClientFactoryClass, conf)
-        .createS3Client(getUri(), bucket, credentials, uaSuffix);
+        .createS3Client(getUri(), bucket, credentials, uaSuffix, awsStats);
   }
 
   /**
@@ -1176,7 +1238,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
         invoker,
         s3guardInvoker,
         statistics,
-        instrumentation,
+        statisticsContext,
         fileStatus,
         seekPolicy,
         changePolicy,
@@ -1273,15 +1335,20 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     PutTracker putTracker =
         committerIntegration.createTracker(path, key);
     String destKey = putTracker.getDestKey();
+    final BlockOutputStreamStatistics outputStreamStatistics
+        = statisticsContext.newOutputStreamStatistics();
     return new FSDataOutputStream(
         new S3ABlockOutputStream(this,
             destKey,
-            new SemaphoredDelegatingExecutor(boundedThreadPool,
-                blockOutputActiveBlocks, true),
+            new SemaphoredDelegatingExecutor(
+                boundedThreadPool,
+                blockOutputActiveBlocks,
+                true,
+                outputStreamStatistics),
             progress,
             partSize,
             blockFactory,
-            instrumentation.newOutputStreamStatistics(statistics),
+            outputStreamStatistics,
             getWriteOperationHelper(),
             putTracker),
         null);
@@ -1655,20 +1722,26 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     @Override
     @Retries.RetryRaw
     public CompletableFuture<S3ListResult> listObjectsAsync(
-            S3ListRequest request)
+        S3ListRequest request,
+        DurationTrackerFactory trackerFactory)
             throws IOException {
-      return submit(unboundedThreadPool,
-        () -> listObjects(request));
+      return submit(unboundedThreadPool, () ->
+          listObjects(request,
+              pairedTrackerFactory(trackerFactory,
+                  getDurationTrackerFactory())));
     }
 
     @Override
     @Retries.RetryRaw
     public CompletableFuture<S3ListResult> continueListObjectsAsync(
-            S3ListRequest request,
-            S3ListResult prevResult)
+        S3ListRequest request,
+        S3ListResult prevResult,
+        DurationTrackerFactory trackerFactory)
             throws IOException {
       return submit(unboundedThreadPool,
-        () -> continueListObjects(request, prevResult));
+          () -> continueListObjects(request, prevResult,
+              pairedTrackerFactory(trackerFactory,
+                  getDurationTrackerFactory())));
     }
 
     @Override
@@ -1817,8 +1890,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * @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);
   }
 
   /**
@@ -1827,7 +1899,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * @param count the count to decrement
    */
   protected void decrementGauge(Statistic statistic, long count) {
-    instrumentation.decrementGauge(statistic, count);
+    statisticsContext.decrementGauge(statistic, count);
   }
 
   /**
@@ -1836,7 +1908,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * @param count the count to increment
    */
   protected void incrementGauge(Statistic statistic, long count) {
-    instrumentation.incrementGauge(statistic, count);
+    statisticsContext.incrementGauge(statistic, count);
   }
 
   /**
@@ -1849,6 +1921,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     if (isThrottleException(ex)) {
       operationThrottled(false);
     } else {
+      incrementStatistic(STORE_IO_RETRY);
       incrementStatistic(IGNORED_ERRORS);
     }
   }
@@ -1900,11 +1973,11 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     LOG.debug("Request throttled on {}", metastore ? "S3": "DynamoDB");
     if (metastore) {
       incrementStatistic(S3GUARD_METADATASTORE_THROTTLED);
-      instrumentation.addValueToQuantiles(S3GUARD_METADATASTORE_THROTTLE_RATE,
+      statisticsContext.addValueToQuantiles(S3GUARD_METADATASTORE_THROTTLE_RATE,
           1);
     } else {
       incrementStatistic(STORE_IO_THROTTLED);
-      instrumentation.addValueToQuantiles(STORE_IO_THROTTLE_RATE, 1);
+      statisticsContext.addValueToQuantiles(STORE_IO_THROTTLE_RATE, 1);
     }
   }
 
@@ -1918,6 +1991,27 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   }
 
   /**
+   * Get the instrumentation's IOStatistics.
+   * @return statistics
+   */
+  @Override
+  public IOStatistics getIOStatistics() {
+    return instrumentation != null
+        ? instrumentation.getIOStatistics()
+        : null;
+  }
+
+  /**
+   * Get the factory for duration tracking.
+   * @return a factory from the instrumentation.
+   */
+  protected DurationTrackerFactory getDurationTrackerFactory() {
+    return instrumentation != null ?
+        instrumentation.getDurationTrackerFactory()
+        : null;
+  }
+
+  /**
    * Request object metadata; increments counters in the process.
    * Retry policy: retry untranslated.
    * @param key key
@@ -1954,15 +2048,30 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     ObjectMetadata meta = changeInvoker.retryUntranslated("GET " + key, true,
         () -> {
           incrementStatistic(OBJECT_METADATA_REQUESTS);
-          LOG.debug("HEAD {} with change tracker {}", key, changeTracker);
-          if (changeTracker != null) {
-            changeTracker.maybeApplyConstraint(request);
-          }
-          ObjectMetadata objectMetadata = s3.getObjectMetadata(request);
-          if (changeTracker != null) {
-            changeTracker.processMetadata(objectMetadata, operation);
+          DurationTracker duration = getDurationTrackerFactory()
+              .trackDuration(ACTION_HTTP_HEAD_REQUEST.getSymbol());
+          try {
+            LOG.debug("HEAD {} with change tracker {}", key, changeTracker);
+            if (changeTracker != null) {
+              changeTracker.maybeApplyConstraint(request);
+            }
+            ObjectMetadata objectMetadata = s3.getObjectMetadata(request);
+            if (changeTracker != null) {
+              changeTracker.processMetadata(objectMetadata, operation);
+            }
+            return objectMetadata;
+          } catch(AmazonServiceException ase) {
+            if (!isObjectNotFound(ase)) {
+              // file not found is not considered a failure of the call,
+              // so only switch the duration tracker to update failure
+              // metrics on other exception outcomes.
+              duration.failed();
+            }
+            throw ase;
+          } finally {
+            // update the tracker.
+            duration.close();
           }
-          return objectMetadata;
         });
     incrementReadOperations();
     return meta;
@@ -1974,13 +2083,15 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    *
    * Retry policy: retry untranslated.
    * @param request request to initiate
+   * @param trackerFactory duration tracking
    * @return the results
    * @throws IOException if the retry invocation raises one (it shouldn't).
    */
   @Retries.RetryRaw
-  protected S3ListResult listObjects(S3ListRequest request) throws IOException {
+  protected S3ListResult listObjects(S3ListRequest request,
+      @Nullable final DurationTrackerFactory trackerFactory)
+      throws IOException {
     incrementReadOperations();
-    incrementStatistic(OBJECT_LIST_REQUESTS);
     LOG.debug("LIST {}", request);
     validateListArguments(request);
     try(DurationInfo ignored =
@@ -1988,13 +2099,15 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       return invoker.retryUntranslated(
           request.toString(),
           true,
-          () -> {
-            if (useListV1) {
-              return S3ListResult.v1(s3.listObjects(request.getV1()));
-            } else {
-              return S3ListResult.v2(s3.listObjectsV2(request.getV2()));
-            }
-          });
+          trackDurationOfOperation(trackerFactory,
+              OBJECT_LIST_REQUEST,
+              () -> {
+                if (useListV1) {
+                  return S3ListResult.v1(s3.listObjects(request.getV1()));
+                } else {
+                  return S3ListResult.v2(s3.listObjectsV2(request.getV2()));
+                }
+              }));
     }
   }
 
@@ -2015,12 +2128,14 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * Retry policy: retry untranslated.
    * @param request last list objects request to continue
    * @param prevResult last paged result to continue from
+   * @param trackerFactory duration tracking
    * @return the next result object
    * @throws IOException none, just there for retryUntranslated.
    */
   @Retries.RetryRaw
   protected S3ListResult continueListObjects(S3ListRequest request,
-      S3ListResult prevResult) throws IOException {
+      S3ListResult prevResult,
+      final DurationTrackerFactory trackerFactory) throws IOException {
     incrementReadOperations();
     validateListArguments(request);
     try(DurationInfo ignored =
@@ -2028,17 +2143,19 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       return invoker.retryUntranslated(
           request.toString(),
           true,
-          () -> {
-            incrementStatistic(OBJECT_CONTINUE_LIST_REQUESTS);
-            if (useListV1) {
-              return S3ListResult.v1(
-                  s3.listNextBatchOfObjects(prevResult.getV1()));
-            } else {
-              request.getV2().setContinuationToken(prevResult.getV2()
-                  .getNextContinuationToken());
-              return S3ListResult.v2(s3.listObjectsV2(request.getV2()));
-            }
-          });
+          trackDurationOfOperation(
+              trackerFactory,
+              OBJECT_CONTINUE_LIST_REQUEST,
+              () -> {
+                if (useListV1) {
+                  return S3ListResult.v1(
+                      s3.listNextBatchOfObjects(prevResult.getV1()));
+                } else {
+                  request.getV2().setContinuationToken(prevResult.getV2()
+                      .getNextContinuationToken());
+                  return S3ListResult.v2(s3.listObjectsV2(request.getV2()));
+                }
+              }));
     }
   }
 
@@ -2083,9 +2200,10 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       invoker.retryUntranslated(String.format("Delete %s:/%s", bucket, key),
           DELETE_CONSIDERED_IDEMPOTENT,
           ()-> {
-            incrementStatistic(OBJECT_DELETE_REQUESTS);
             incrementStatistic(OBJECT_DELETE_OBJECTS);
-            s3.deleteObject(bucket, key);
+            trackDurationOfInvocation(getDurationTrackerFactory(),
+                OBJECT_DELETE_REQUEST.getSymbol(),
+                () -> s3.deleteObject(bucket, key));
             return null;
           });
     }
@@ -2169,11 +2287,12 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
             // handle the failure
             retryHandler.bulkDeleteRetried(deleteRequest, e);
           },
-          () -> {
-            incrementStatistic(OBJECT_DELETE_REQUESTS, 1);
-            incrementStatistic(OBJECT_DELETE_OBJECTS, keyCount);
-            return s3.deleteObjects(deleteRequest);
-          });
+          // duration is tracked in the bulk delete counters
+          trackDurationOfOperation(getDurationTrackerFactory(),
+              OBJECT_BULK_DELETE_REQUEST.getSymbol(), () -> {
+                incrementStatistic(OBJECT_DELETE_OBJECTS, keyCount);
+                return s3.deleteObjects(deleteRequest);
+            }));
     } catch (MultiObjectDeleteException e) {
       // one or more of the keys could not be deleted.
       // log and rethrow
@@ -2665,17 +2784,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
           throws FileNotFoundException, IOException {
     RemoteIterator<S3AFileStatus> listStatusItr = once("listStatus",
             p.toString(), () -> innerListStatus(p));
-    return new RemoteIterator<FileStatus>() {
-      @Override
-      public boolean hasNext() throws IOException {
-        return listStatusItr.hasNext();
-      }
-
-      @Override
-      public FileStatus next() throws IOException {
-        return listStatusItr.next();
-      }
-    };
+    return typeCastingRemoteIterator(listStatusItr);
   }
 
   /**
@@ -3201,8 +3310,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
         S3ListRequest request = createListObjectsRequest(dirKey, "/",
             listSize);
         // execute the request
-        S3ListResult listResult = listObjects(request);
-
+        S3ListResult listResult = listObjects(request,
+            getDurationTrackerFactory());
 
         if (listResult.hasPrefixesOrObjects(contextAccessors, tombstones)) {
           if (LOG.isDebugEnabled()) {
@@ -3607,8 +3716,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     ChangeTracker changeTracker = new ChangeTracker(
         keyToQualifiedPath(srcKey).toString(),
         changeDetectionPolicy,
-        readContext.instrumentation.newInputStreamStatistics()
-            .getVersionMismatchCounter(),
+        readContext.getS3AStatisticsContext()
+            .newInputStreamStatistics()
+            .getChangeTrackerStatistics(),
         srcAttributes);
 
     String action = "copyFile(" + srcKey + ", " + dstKey + ")";
@@ -4102,12 +4212,10 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     sb.append(", delegation tokens=")
         .append(delegationTokens.map(Objects::toString).orElse("disabled"));
     sb.append(", ").append(directoryPolicy);
-    sb.append(", statistics {")
-        .append(statistics)
-        .append("}");
-    if (instrumentation != null) {
-      sb.append(", metrics {")
-          .append(instrumentation.dump("{", "=", "} ", true))
+    // if logging at debug, toString returns the entire IOStatistics set.
+    if (getInstrumentation() != null) {
+      sb.append(", instrumentation {")
+          .append(getInstrumentation().toString())
           .append("}");
     }
     sb.append('}');
@@ -4307,21 +4415,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
         new Listing.AcceptFilesOnly(qualify(f)), null, true, false));
   }
 
-  private static RemoteIterator<LocatedFileStatus> toLocatedFileStatusIterator(
-      RemoteIterator<? extends LocatedFileStatus> iterator) {
-    return new RemoteIterator<LocatedFileStatus>() {
-      @Override
-      public boolean hasNext() throws IOException {
-        return iterator.hasNext();
-      }
-
-      @Override
-      public LocatedFileStatus next() throws IOException {
-        return iterator.next();
-      }
-    };
-  }
-
   /**
    * Recursive List of files and empty directories.
    * @param f path to list from
@@ -4404,7 +4497,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       if (status != null && status.isFile()) {
         // simple case: File
         LOG.debug("Path is a file: {}", path);
-        return new Listing.SingleStatusRemoteIterator(
+        return listing.createSingleStatusIterator(
             toLocatedFileStatus(status));
       }
       // Assuming the path to be a directory
@@ -4424,7 +4517,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
                 ? status
                 : (S3AFileStatus) getFileStatus(path);
         if (fileStatus.isFile()) {
-          return new Listing.SingleStatusRemoteIterator(
+          return listing.createSingleStatusIterator(
                   toLocatedFileStatus(fileStatus));
         }
       }
@@ -4483,7 +4576,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
               if (fileStatus.isFile()) {
                 // simple case: File
                 LOG.debug("Path is a file");
-                return new Listing.SingleStatusRemoteIterator(
+                return listing.createSingleStatusIterator(
                         filter.accept(path)
                                 ? toLocatedFileStatus(fileStatus)
                                 : null);
@@ -4630,8 +4723,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * 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")
@@ -4765,8 +4858,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       ChangeTracker changeTracker =
           new ChangeTracker(uri.toString(),
               changeDetectionPolicy,
-              readContext.instrumentation.newInputStreamStatistics()
-                  .getVersionMismatchCounter(),
+              readContext.getS3AStatisticsContext()
+                  .newInputStreamStatistics()
+                  .getChangeTrackerStatistics(),
               objectAttributes);
 
       // will retry internally if wrong version detected
@@ -4913,7 +5007,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
         getWriteOperationHelper(),
         ctx,
         basePath,
-        new S3AMultipartUploaderStatisticsImpl(ctx::incrementStatistic));
+        statisticsContext.createMultipartUploaderStatistics());
   }
 
   /**
@@ -4933,7 +5027,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
         .setExecutor(boundedThreadPool)
         .setExecutorCapacity(executorCapacity)
         .setInvoker(invoker)
-        .setInstrumentation(getInstrumentation())
+        .setInstrumentation(statisticsContext)
         .setStorageStatistics(getStorageStatistics())
         .setInputPolicy(getInputPolicy())
         .setChangeDetectionPolicy(changeDetectionPolicy)
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
index bd8adad..c725fdf 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
@@ -33,10 +33,14 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.CanSetReadahead;
 import org.apache.hadoop.fs.CanUnbuffer;
 import org.apache.hadoop.fs.FSExceptionMessages;
-import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
+import org.apache.hadoop.fs.s3a.impl.ChangeTracker;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
 import org.apache.hadoop.fs.PathIOException;
 import org.apache.hadoop.fs.StreamCapabilities;
-import org.apache.hadoop.fs.s3a.impl.ChangeTracker;
+import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.fs.statistics.DurationTracker;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -68,7 +72,7 @@ import static org.apache.hadoop.util.StringUtils.toLowerCase;
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
-        CanUnbuffer, StreamCapabilities {
+        CanUnbuffer, StreamCapabilities, IOStatisticsSource {
 
   public static final String E_NEGATIVE_READAHEAD_VALUE
       = "Negative readahead value";
@@ -105,7 +109,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
   private final String uri;
   private static final Logger LOG =
       LoggerFactory.getLogger(S3AInputStream.class);
-  private final S3AInstrumentation.InputStreamStatistics streamStatistics;
+  private final S3AInputStreamStatistics streamStatistics;
   private S3AEncryptionMethods serverSideEncryptionAlgorithm;
   private String serverSideEncryptionKey;
   private S3AInputPolicy inputPolicy;
@@ -132,6 +136,11 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
   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
    * actual read() operation.
@@ -154,13 +163,15 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
     this.contentLength = l;
     this.client = client;
     this.uri = "s3a://" + this.bucket + "/" + this.key;
-    this.streamStatistics = ctx.instrumentation.newInputStreamStatistics();
+    this.streamStatistics = ctx.getS3AStatisticsContext()
+        .newInputStreamStatistics();
+    this.ioStatistics = streamStatistics.getIOStatistics();
     this.serverSideEncryptionAlgorithm =
         s3Attributes.getServerSideEncryptionAlgorithm();
     this.serverSideEncryptionKey = s3Attributes.getServerSideEncryptionKey();
     this.changeTracker = new ChangeTracker(uri,
         ctx.getChangeDetectionPolicy(),
-        streamStatistics.getVersionMismatchCounter(),
+        streamStatistics.getChangeTrackerStatistics(),
         s3Attributes);
     setInputPolicy(ctx.getInputPolicy());
     setReadahead(ctx.getReadahead());
@@ -210,8 +221,21 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
     String text = String.format("%s %s at %d",
         operation, uri, targetPos);
     changeTracker.maybeApplyConstraint(request);
-    object = Invoker.once(text, uri,
-        () -> client.getObject(request));
+    DurationTracker tracker = streamStatistics.initiateGetRequest();
+    try {
+      object = Invoker.once(text, uri,
+          () -> client.getObject(request));
+    } catch(IOException e) {
+      // input function failed: note it
+      tracker.failed();
+      // and rethrow
+      throw e;
+    } finally {
+      // update the tracker.
+      // this is called after any catch() call will have
+      // set the failed flag.
+      tracker.close();
+    }
 
     changeTracker.processResponse(object, operation,
         targetPos);
@@ -294,13 +318,11 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
       if (skipForward) {
         // the forward seek range is within the limits
         LOG.debug("Forward seek on {}, of {} bytes", uri, diff);
-        streamStatistics.seekForwards(diff);
         long skipped = wrappedStream.skip(diff);
         if (skipped > 0) {
           pos += skipped;
-          // as these bytes have been read, they are included in the counter
-          incrementBytesRead(diff);
         }
+        streamStatistics.seekForwards(diff, skipped);
 
         if (pos == targetPos) {
           // all is well
@@ -312,6 +334,9 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
           LOG.warn("Failed to seek on {} to {}. Current position {}",
               uri, targetPos,  pos);
         }
+      } else {
+        // not attempting to read any bytes from the stream
+        streamStatistics.seekForwards(diff, 0);
       }
     } else if (diff < 0) {
       // backwards seek
@@ -356,7 +381,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
     // open.  After that, an exception generally means the file has changed
     // and there is no point retrying anymore.
     Invoker invoker = context.getReadInvoker();
-    invoker.maybeRetry(streamStatistics.openOperations == 0,
+    invoker.maybeRetry(streamStatistics.getOpenOperations() == 0,
         "lazySeek", pathStr, true,
         () -> {
           //For lazy seek
@@ -720,7 +745,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
       sb.append(" contentRangeFinish=").append(contentRangeFinish);
       sb.append(" remainingInCurrentRequest=")
           .append(remainingInCurrentRequest());
-      sb.append(changeTracker);
+      sb.append(" ").append(changeTracker);
       sb.append('\n').append(s);
       sb.append('}');
       return sb.toString();
@@ -774,7 +799,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
    */
   @InterfaceAudience.Private
   @InterfaceStability.Unstable
-  public S3AInstrumentation.InputStreamStatistics getS3AStreamStatistics() {
+  public S3AInputStreamStatistics getS3AStreamStatistics() {
     return streamStatistics;
   }
 
@@ -858,13 +883,14 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
     try {
       closeStream("unbuffer()", contentRangeFinish, false);
     } finally {
-      streamStatistics.merge(false);
+      streamStatistics.unbuffered();
     }
   }
 
   @Override
   public boolean hasCapability(String capability) {
     switch (toLowerCase(capability)) {
+    case StreamCapabilities.IOSTATISTICS:
     case StreamCapabilities.READAHEAD:
     case StreamCapabilities.UNBUFFER:
       return true;
@@ -877,4 +903,9 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
   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 19f42cf..982611a 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
@@ -18,14 +18,34 @@
 
 package org.apache.hadoop.fs.s3a;
 
+import javax.annotation.Nullable;
+
 import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.FileSystem.Statistics;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.s3a.s3guard.MetastoreInstrumentation;
+import org.apache.hadoop.fs.s3a.statistics.impl.AbstractS3AStatisticsSource;
+import org.apache.hadoop.fs.s3a.statistics.ChangeTrackerStatistics;
+import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics;
+import org.apache.hadoop.fs.s3a.statistics.CountersAndGauges;
+import org.apache.hadoop.fs.s3a.statistics.impl.CountingChangeTracker;
+import org.apache.hadoop.fs.s3a.statistics.DelegationTokenStatistics;
+import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
+import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics;
+import org.apache.hadoop.fs.s3a.statistics.StatisticTypeEnum;
+import org.apache.hadoop.fs.statistics.DurationTrackerFactory;
+import org.apache.hadoop.fs.statistics.IOStatisticsLogging;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
+import org.apache.hadoop.fs.statistics.StreamStatisticNames;
+import org.apache.hadoop.fs.statistics.DurationTracker;
+import org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding;
+import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
+import org.apache.hadoop.fs.statistics.impl.IOStatisticsStoreBuilder;
 import org.apache.hadoop.metrics2.AbstractMetric;
 import org.apache.hadoop.metrics2.MetricStringBuilder;
 import org.apache.hadoop.metrics2.MetricsCollector;
@@ -43,216 +63,176 @@ import org.apache.hadoop.metrics2.lib.MutableQuantiles;
 
 import java.io.Closeable;
 import java.net.URI;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.EnumSet;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
+import static org.apache.hadoop.fs.s3a.Constants.STREAM_READ_GAUGE_INPUT_POLICY;
+import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.demandStringifyIOStatistics;
+import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.snapshotIOStatistics;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.ACTION_EXECUTOR_ACQUIRED;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.ACTION_HTTP_GET_REQUEST;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_FAILURES;
+import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_UNBUFFERED;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore;
 import static org.apache.hadoop.fs.s3a.Statistic.*;
 
 /**
- * Instrumentation of S3a.
- * Derived from the {@code AzureFileSystemInstrumentation}.
- *
+ * Instrumentation of S3A.
+ * <p>
+ * History
+ * </p>
+ * <ol>
+ *   <li>
+ *    HADOOP-13028. Initial implementation.
+ *    Derived from the {@code AzureFileSystemInstrumentation}.
+ *   </li>
+ *   <li>
+ *    Broadly (and directly) used in S3A.
+ *    The use of direct references causes "problems" in mocking tests.
+ *   </li>
+ *   <li>
+ *     HADOOP-16830. IOStatistics. Move to an interface and implementation
+ *     design for the different inner classes.
+ *   </li>
+ * </ol>
+ * <p>
  * Counters and metrics are generally addressed in code by their name or
  * {@link Statistic} key. There <i>may</i> be some Statistics which do
  * not have an entry here. To avoid attempts to access such counters failing,
  * the operations to increment/query metric values are designed to handle
  * lookup failures.
+ * </p>
+ * <p>
+ *   S3AFileSystem StorageStatistics are dynamically derived from
+ *   the IOStatistics.
+ * </p>
+ * <p>
+ *   The toString() operation includes the entire IOStatistics when this
+ *   class's log is set to DEBUG. This keeps the logs somewhat manageable
+ *   on normal runs, but allows for more reporting.
+ * </p>
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public class S3AInstrumentation implements Closeable, MetricsSource {
+public class S3AInstrumentation implements Closeable, MetricsSource,
+    CountersAndGauges, IOStatisticsSource, DurationTrackerFactory {
   private static final Logger LOG = LoggerFactory.getLogger(
       S3AInstrumentation.class);
 
   private static final String METRICS_SOURCE_BASENAME = "S3AMetrics";
 
   /**
-   * {@value #METRICS_SYSTEM_NAME} The name of the s3a-specific metrics
+   * {@value} The name of the s3a-specific metrics
    * system instance used for s3a metrics.
    */
   public static final String METRICS_SYSTEM_NAME = "s3a-file-system";
 
   /**
-   * {@value #CONTEXT} Currently all s3a metrics are placed in a single
+   * {@value} Currently all s3a metrics are placed in a single
    * "context". Distinct contexts may be used in the future.
    */
   public static final String CONTEXT = "s3aFileSystem";
 
   /**
-   * {@value #METRIC_TAG_FILESYSTEM_ID} The name of a field added to metrics
+   * {@value} The name of a field added to metrics
    * records that uniquely identifies a specific FileSystem instance.
    */
   public static final String METRIC_TAG_FILESYSTEM_ID = "s3aFileSystemId";
 
   /**
-   * {@value #METRIC_TAG_BUCKET} The name of a field added to metrics records
+   * {@value} The name of a field added to metrics records
    * that indicates the hostname portion of the FS URL.
    */
   public static final String METRIC_TAG_BUCKET = "bucket";
 
   // metricsSystemLock must be used to synchronize modifications to
   // metricsSystem and the following counters.
-  private static Object metricsSystemLock = new Object();
+  private static final Object METRICS_SYSTEM_LOCK = new Object();
   private static MetricsSystem metricsSystem = null;
   private static int metricsSourceNameCounter = 0;
   private static int metricsSourceActiveCounter = 0;
 
+  private final DurationTrackerFactory durationTrackerFactory;
+
   private String metricsSourceName;
 
   private final MetricsRegistry registry =
       new MetricsRegistry("s3aFileSystem").setContext(CONTEXT);
-  private final MutableCounterLong streamOpenOperations;
-  private final MutableCounterLong streamCloseOperations;
-  private final MutableCounterLong streamClosed;
-  private final MutableCounterLong streamAborted;
-  private final MutableCounterLong streamSeekOperations;
-  private final MutableCounterLong streamReadExceptions;
-  private final MutableCounterLong streamForwardSeekOperations;
-  private final MutableCounterLong streamBackwardSeekOperations;
-  private final MutableCounterLong streamBytesSkippedOnSeek;
-  private final MutableCounterLong streamBytesBackwardsOnSeek;
-  private final MutableCounterLong streamBytesRead;
-  private final MutableCounterLong streamReadOperations;
-  private final MutableCounterLong streamReadFullyOperations;
-  private final MutableCounterLong streamReadsIncomplete;
-  private final MutableCounterLong streamBytesReadInClose;
-  private final MutableCounterLong streamBytesDiscardedInAbort;
-  private final MutableCounterLong ignoredErrors;
   private final MutableQuantiles putLatencyQuantile;
   private final MutableQuantiles throttleRateQuantile;
   private final MutableQuantiles s3GuardThrottleRateQuantile;
-  private final MutableCounterLong numberOfFilesCreated;
-  private final MutableCounterLong numberOfFilesCopied;
-  private final MutableCounterLong bytesOfFilesCopied;
-  private final MutableCounterLong numberOfFilesDeleted;
-  private final MutableCounterLong numberOfFakeDirectoryDeletes;
-  private final MutableCounterLong numberOfDirectoriesCreated;
-  private final MutableCounterLong numberOfDirectoriesDeleted;
 
   /** Instantiate this without caring whether or not S3Guard is enabled. */
   private final S3GuardInstrumentation s3GuardInstrumentation
       = new S3GuardInstrumentation();
 
-  private static final Statistic[] COUNTERS_TO_CREATE = {
-      INVOCATION_COPY_FROM_LOCAL_FILE,
-      INVOCATION_CREATE,
-      INVOCATION_CREATE_NON_RECURSIVE,
-      INVOCATION_DELETE,
-      INVOCATION_EXISTS,
-      INVOCATION_GET_DELEGATION_TOKEN,
-      INVOCATION_GET_FILE_CHECKSUM,
-      INVOCATION_GET_FILE_STATUS,
-      INVOCATION_GLOB_STATUS,
-      INVOCATION_IS_DIRECTORY,
-      INVOCATION_IS_FILE,
-      INVOCATION_LIST_FILES,
-      INVOCATION_LIST_LOCATED_STATUS,
-      INVOCATION_LIST_STATUS,
-      INVOCATION_MKDIRS,
-      INVOCATION_OPEN,
-      INVOCATION_RENAME,
-      OBJECT_COPY_REQUESTS,
-      OBJECT_DELETE_REQUESTS,
-      OBJECT_DELETE_OBJECTS,
-      OBJECT_LIST_REQUESTS,
-      OBJECT_CONTINUE_LIST_REQUESTS,
-      OBJECT_METADATA_REQUESTS,
-      OBJECT_MULTIPART_UPLOAD_ABORTED,
-      OBJECT_PUT_BYTES,
-      OBJECT_PUT_REQUESTS,
-      OBJECT_PUT_REQUESTS_COMPLETED,
-      OBJECT_SELECT_REQUESTS,
-      STREAM_READ_VERSION_MISMATCHES,
-      STREAM_WRITE_FAILURES,
-      STREAM_WRITE_BLOCK_UPLOADS,
-      STREAM_WRITE_BLOCK_UPLOADS_COMMITTED,
-      STREAM_WRITE_BLOCK_UPLOADS_ABORTED,
-      STREAM_WRITE_TOTAL_TIME,
-      STREAM_WRITE_TOTAL_DATA,
-      COMMITTER_COMMITS_CREATED,
-      COMMITTER_COMMITS_COMPLETED,
-      COMMITTER_JOBS_SUCCEEDED,
-      COMMITTER_JOBS_FAILED,
-      COMMITTER_TASKS_SUCCEEDED,
-      COMMITTER_TASKS_FAILED,
-      COMMITTER_BYTES_COMMITTED,
-      COMMITTER_BYTES_UPLOADED,
-      COMMITTER_COMMITS_FAILED,
-      COMMITTER_COMMITS_ABORTED,
-      COMMITTER_COMMITS_REVERTED,
-      COMMITTER_MAGIC_FILES_CREATED,
-      S3GUARD_METADATASTORE_PUT_PATH_REQUEST,
-      S3GUARD_METADATASTORE_INITIALIZATION,
-      S3GUARD_METADATASTORE_RECORD_DELETES,
-      S3GUARD_METADATASTORE_RECORD_READS,
-      S3GUARD_METADATASTORE_RECORD_WRITES,
-      S3GUARD_METADATASTORE_RETRY,
-      S3GUARD_METADATASTORE_THROTTLED,
-      S3GUARD_METADATASTORE_AUTHORITATIVE_DIRECTORIES_UPDATED,
-      STORE_IO_THROTTLED,
-      DELEGATION_TOKENS_ISSUED,
-      FILES_DELETE_REJECTED,
-      MULTIPART_INSTANTIATED,
-      MULTIPART_PART_PUT,
-      MULTIPART_PART_PUT_BYTES,
-      MULTIPART_UPLOAD_ABORTED,
-      MULTIPART_UPLOAD_ABORT_UNDER_PATH_INVOKED,
-      MULTIPART_UPLOAD_COMPLETED,
-      MULTIPART_UPLOAD_STARTED
-  };
+  /**
+   * This is the IOStatistics store for the S3AFileSystem
+   * instance.
+   * It is not kept in sync with the rest of the S3A instrumentation.
+   * Most inner statistics implementation classes only update this
+   * store when it is pushed back, such as as in close().
+   */
+  private final IOStatisticsStore instanceIOStatistics;
 
+  /**
+   * Gauges to create.
+   * <p></p>
+   * All statistics which are not gauges or quantiles
+   * are registered as counters.
+   */
   private static final Statistic[] GAUGES_TO_CREATE = {
       OBJECT_PUT_REQUESTS_ACTIVE,
       OBJECT_PUT_BYTES_PENDING,
       STREAM_WRITE_BLOCK_UPLOADS_ACTIVE,
       STREAM_WRITE_BLOCK_UPLOADS_PENDING,
-      STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING,
+      STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING,
   };
 
+  /**
+   * Construct the instrumentation for a filesystem.
+   * @param name URI of filesystem.
+   */
   public S3AInstrumentation(URI name) {
     UUID fileSystemInstanceId = UUID.randomUUID();
     registry.tag(METRIC_TAG_FILESYSTEM_ID,
         "A unique identifier for the instance",
         fileSystemInstanceId.toString());
     registry.tag(METRIC_TAG_BUCKET, "Hostname from the FS URL", name.getHost());
-    streamOpenOperations = counter(STREAM_OPENED);
-    streamCloseOperations = counter(STREAM_CLOSE_OPERATIONS);
-    streamClosed = counter(STREAM_CLOSED);
-    streamAborted = counter(STREAM_ABORTED);
-    streamSeekOperations = counter(STREAM_SEEK_OPERATIONS);
-    streamReadExceptions = counter(STREAM_READ_EXCEPTIONS);
-    streamForwardSeekOperations =
-        counter(STREAM_FORWARD_SEEK_OPERATIONS);
-    streamBackwardSeekOperations =
-        counter(STREAM_BACKWARD_SEEK_OPERATIONS);
-    streamBytesSkippedOnSeek = counter(STREAM_SEEK_BYTES_SKIPPED);
-    streamBytesBackwardsOnSeek =
-        counter(STREAM_SEEK_BYTES_BACKWARDS);
-    streamBytesRead = counter(STREAM_SEEK_BYTES_READ);
-    streamReadOperations = counter(STREAM_READ_OPERATIONS);
-    streamReadFullyOperations =
-        counter(STREAM_READ_FULLY_OPERATIONS);
-    streamReadsIncomplete =
-        counter(STREAM_READ_OPERATIONS_INCOMPLETE);
-    streamBytesReadInClose = counter(STREAM_CLOSE_BYTES_READ);
-    streamBytesDiscardedInAbort = counter(STREAM_ABORT_BYTES_DISCARDED);
-    numberOfFilesCreated = counter(FILES_CREATED);
-    numberOfFilesCopied = counter(FILES_COPIED);
-    bytesOfFilesCopied = counter(FILES_COPIED_BYTES);
-    numberOfFilesDeleted = counter(FILES_DELETED);
-    numberOfFakeDirectoryDeletes = counter(FAKE_DIRECTORIES_DELETED);
-    numberOfDirectoriesCreated = counter(DIRECTORIES_CREATED);
-    numberOfDirectoriesDeleted = counter(DIRECTORIES_DELETED);
-    ignoredErrors = counter(IGNORED_ERRORS);
-    for (Statistic statistic : COUNTERS_TO_CREATE) {
-      counter(statistic);
-    }
-    for (Statistic statistic : GAUGES_TO_CREATE) {
-      gauge(statistic.getSymbol(), statistic.getDescription());
-    }
+
+    // now set up the instance IOStatistics.
+    // create the builder
+    IOStatisticsStoreBuilder storeBuilder = iostatisticsStore();
+
+    // add the gauges
+    List<Statistic> gauges = Arrays.asList(GAUGES_TO_CREATE);
+    gauges.forEach(this::gauge);
+
+    // declare all counter statistics
+    EnumSet.allOf(Statistic.class).stream()
+        .filter(statistic ->
+            statistic.getType() == StatisticTypeEnum.TYPE_COUNTER)
+        .forEach(stat -> {
+          counter(stat);
+          storeBuilder.withCounters(stat.getSymbol());
+        });
+
+    // and durations
+    EnumSet.allOf(Statistic.class).stream()
+        .filter(statistic ->
+            statistic.getType() == StatisticTypeEnum.TYPE_DURATION)
+        .forEach(stat -> {
+          duration(stat);
+          storeBuilder.withDurationTracking(stat.getSymbol());
+        });
+
     //todo need a config for the quantiles interval?
     int interval = 1;
     putLatencyQuantile = quantiles(S3GUARD_METADATASTORE_PUT_PATH_LATENCY,
@@ -262,12 +242,21 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
     throttleRateQuantile = quantiles(STORE_IO_THROTTLE_RATE,
         "events", "frequency (Hz)", interval);
 
+    // register with Hadoop metrics
     registerAsMetricsSource(name);
+
+    // and build the IO Statistics
+    instanceIOStatistics = storeBuilder.build();
+
+    // duration track metrics (Success/failure) and IOStatistics.
+    durationTrackerFactory = IOStatisticsBinding.pairedTrackerFactory(
+        instanceIOStatistics,
+        new MetricDurationTrackerFactory());
   }
 
   @VisibleForTesting
   public MetricsSystem getMetricsSystem() {
-    synchronized (metricsSystemLock) {
+    synchronized (METRICS_SYSTEM_LOCK) {
       if (metricsSystem == null) {
         metricsSystem = new MetricsSystemImpl();
         metricsSystem.init(METRICS_SYSTEM_NAME);
@@ -282,7 +271,7 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
    */
   private void registerAsMetricsSource(URI name) {
     int number;
-    synchronized(metricsSystemLock) {
+    synchronized(METRICS_SYSTEM_LOCK) {
       getMetricsSystem();
 
       metricsSourceActiveCounter++;
@@ -313,6 +302,15 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
   }
 
   /**
+   * Registering a duration adds the success and failure counters.
+   * @param op statistic to track
+   */
+  protected final void duration(Statistic op) {
+    counter(op.getSymbol(), op.getDescription());
+    counter(op.getSymbol() + SUFFIX_FAILURES, op.getDescription());
+  }
+
+  /**
    * Create a gauge in the registry.
    * @param name name gauge name
    * @param desc description
@@ -323,6 +321,15 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
   }
 
   /**
+   * Create a gauge in the registry.
+   * @param op statistic to count
+   * @return the gauge
+   */
+  protected final MutableGaugeLong gauge(Statistic op) {
+    return gauge(op.getSymbol(), op.getDescription());
+  }
+
+  /**
    * Create a quantiles in the registry.
    * @param op  statistic to collect
    * @param sampleName sample name of the quantiles
@@ -398,7 +405,8 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
     }
     if (!(metric instanceof MutableCounterLong)) {
       throw new IllegalStateException("Metric " + name
-          + " is not a MutableCounterLong: " + metric);
+          + " is not a MutableCounterLong: " + metric
+          + " (type: " + metric.getClass() +")");
     }
     return (MutableCounterLong) metric;
   }
@@ -443,10 +451,56 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
   }
 
   /**
+   * Get the instance IO Statistics.
+   * @return statistics.
+   */
+  @Override
+  public IOStatisticsStore getIOStatistics() {
+    return instanceIOStatistics;
+  }
+
+  /**
+   * Get the duration tracker factory.
+   * @return duration tracking for the instrumentation.
+   */
+  public DurationTrackerFactory getDurationTrackerFactory() {
+    return durationTrackerFactory;
+  }
+
+  /**
+   * The duration tracker updates the metrics with the count
+   * and IOStatistics will full duration information.
+   * @param key statistic key prefix
+   * @param count  #of times to increment the matching counter in this
+   * operation.
+   * @return a duration tracker.
+   */
+  @Override
+  public DurationTracker trackDuration(final String key, final long count) {
+    return durationTrackerFactory.trackDuration(key, count);
+  }
+
+  /**
+   * String representation. Includes the IOStatistics
+   * when logging is at DEBUG.
+   * @return a string form.
+   */
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "S3AInstrumentation{");
+    if (LOG.isDebugEnabled()) {
+      sb.append("instanceIOStatistics=").append(instanceIOStatistics);
+    }
+    sb.append('}');
+    return sb.toString();
+  }
+
+  /**
    * Indicate that S3A created a file.
    */
   public void fileCreated() {
-    numberOfFilesCreated.incr();
+    incrementCounter(FILES_CREATED, 1);
   }
 
   /**
@@ -454,7 +508,7 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
    * @param count number of files.
    */
   public void fileDeleted(int count) {
-    numberOfFilesDeleted.incr(count);
+    incrementCounter(FILES_DELETED, count);
   }
 
   /**
@@ -462,21 +516,21 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
    * @param count number of directory entries included in the delete request.
    */
   public void fakeDirsDeleted(int count) {
-    numberOfFakeDirectoryDeletes.incr(count);
+    incrementCounter(FAKE_DIRECTORIES_DELETED, count);
   }
 
   /**
    * Indicate that S3A created a directory.
    */
   public void directoryCreated() {
-    numberOfDirectoriesCreated.incr();
+    incrementCounter(DIRECTORIES_CREATED, 1);
   }
 
   /**
    * Indicate that S3A just deleted a directory.
    */
   public void directoryDeleted() {
-    numberOfDirectoriesDeleted.incr();
+    incrementCounter(DIRECTORIES_DELETED, 1);
   }
 
   /**
@@ -486,27 +540,44 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
    * @param size total size in bytes
    */
   public void filesCopied(int files, long size) {
-    numberOfFilesCopied.incr(files);
-    bytesOfFilesCopied.incr(size);
+    incrementCounter(FILES_COPIED, files);
+    incrementCounter(FILES_COPIED_BYTES, size);
   }
 
   /**
    * Note that an error was ignored.
    */
   public void errorIgnored() {
-    ignoredErrors.incr();
+    incrementCounter(IGNORED_ERRORS, 1);
   }
 
   /**
-   * Increment a specific counter.
-   * No-op if not defined.
+   * Increments a mutable counter and the matching
+   * instance IOStatistics counter.
+   * No-op if the counter is not defined, or the count == 0.
    * @param op operation
    * @param count increment value
    */
   public void incrementCounter(Statistic op, long count) {
-    MutableCounterLong counter = lookupCounter(op.getSymbol());
-    if (counter != null) {
-      counter.incr(count);
+    String name = op.getSymbol();
+    if (count != 0) {
+      incrementMutableCounter(name, count);
+      instanceIOStatistics.incrementCounter(name, count);
+    }
+  }
+
+  /**
+   * Increments a Mutable counter.
+   * No-op if not a positive integer.
+   * @param name counter name.
+   * @param count increment value
+   */
+  private void incrementMutableCounter(final String name, final long count) {
+    if (count > 0) {
+      MutableCounterLong counter = lookupCounter(name);
+      if (counter != null) {
+        counter.incr(count);
+      }
     }
   }
 
@@ -525,8 +596,10 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
   }
 
   /**
-   * Increment a specific counter.
-   * No-op if not defined.
+   * Increments a mutable counter and the matching
+   * instance IOStatistics counter with the value of
+   * the atomic long.
+   * No-op if the counter is not defined, or the count == 0.
    * @param op operation
    * @param count atomic long containing value
    */
@@ -567,11 +640,29 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
   }
 
   /**
+   * Add the duration as a timed statistic, deriving
+   * statistic name from the operation symbol and the outcome.
+   * @param op operation
+   * @param success was the operation a success?
+   * @param duration how long did it take
+   */
+  @Override
+  public void recordDuration(final Statistic op,
+      final boolean success,
+      final Duration duration) {
+    String name = op.getSymbol()
+        + (success ? "" : SUFFIX_FAILURES);
+    instanceIOStatistics.addTimedOperation(name, duration);
+  }
+
+  /**
    * Create a stream input statistics instance.
    * @return the new instance
+   * @param filesystemStatistics FS Statistics to update in close().
    */
-  public InputStreamStatistics newInputStreamStatistics() {
-    return new InputStreamStatistics();
+  public S3AInputStreamStatistics newInputStreamStatistics(
+      @Nullable final FileSystem.Statistics filesystemStatistics) {
+    return new InputStreamStatistics(filesystemStatistics);
   }
 
   /**
@@ -587,34 +678,8 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
    * Create a new instance of the committer statistics.
    * @return a new committer statistics instance
    */
-  CommitterStatistics newCommitterStatistics() {
-    return new CommitterStatistics();
-  }
-
-  /**
-   * Merge in the statistics of a single input stream into
-   * the filesystem-wide statistics.
-   * @param statistics stream statistics
-   */
-  private void mergeInputStreamStatistics(InputStreamStatistics statistics) {
-    streamOpenOperations.incr(statistics.openOperations);
-    streamCloseOperations.incr(statistics.closeOperations);
-    streamClosed.incr(statistics.closed);
-    streamAborted.incr(statistics.aborted);
-    streamSeekOperations.incr(statistics.seekOperations);
-    streamReadExceptions.incr(statistics.readExceptions);
-    streamForwardSeekOperations.incr(statistics.forwardSeekOperations);
-    streamBytesSkippedOnSeek.incr(statistics.bytesSkippedOnSeek);
-    streamBackwardSeekOperations.incr(statistics.backwardSeekOperations);
-    streamBytesBackwardsOnSeek.incr(statistics.bytesBackwardsOnSeek);
-    streamBytesRead.incr(statistics.bytesRead);
-    streamReadOperations.incr(statistics.readOperations);
-    streamReadFullyOperations.incr(statistics.readFullyOperations);
-    streamReadsIncomplete.incr(statistics.readsIncomplete);
-    streamBytesReadInClose.incr(statistics.bytesReadInClose);
-    streamBytesDiscardedInAbort.incr(statistics.bytesDiscardedInAbort);
-    incrementCounter(STREAM_READ_VERSION_MISMATCHES,
-        statistics.versionMismatches.get());
+  public CommitterStatistics newCommitterStatistics() {
+    return new CommitterStatisticsImpl();
   }
 
   @Override
@@ -623,14 +688,15 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
   }
 
   public void close() {
-    synchronized (metricsSystemLock) {
+    synchronized (METRICS_SYSTEM_LOCK) {
       // it is critical to close each quantile, as they start a scheduled
       // task in a shared thread pool.
       putLatencyQuantile.stop();
       throttleRateQuantile.stop();
       s3GuardThrottleRateQuantile.stop();
       metricsSystem.unregisterSource(metricsSourceName);
-      int activeSources = --metricsSourceActiveCounter;
+      metricsSourceActiveCounter--;
+      int activeSources = metricsSourceActiveCounter;
       if (activeSources == 0) {
         LOG.debug("Shutting down metrics publisher");
         metricsSystem.publishMetricsNow();
@@ -641,164 +707,364 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
   }
 
   /**
-   * Statistics updated by an input stream during its actual operation.
-   * These counters not thread-safe and are for use in a single instance
-   * of a stream.
+   * A duration tracker which updates a mutable counter with a metric.
+   * The metric is updated with the count on start; after a failure
+   * the failures count is incremented by one.
    */
-  @InterfaceAudience.Private
-  @InterfaceStability.Unstable
-  public final class InputStreamStatistics implements AutoCloseable {
-    public long openOperations;
-    public long closeOperations;
-    public long closed;
-    public long aborted;
-    public long seekOperations;
-    public long readExceptions;
-    public long forwardSeekOperations;
-    public long backwardSeekOperations;
-    public long bytesRead;
-    public long bytesSkippedOnSeek;
-    public long bytesBackwardsOnSeek;
-    public long readOperations;
-    public long readFullyOperations;
-    public long readsIncomplete;
-    public long bytesReadInClose;
-    public long bytesDiscardedInAbort;
-    public long policySetCount;
-    public long inputPolicy;
-    /** This is atomic so that it can be passed as a reference. */
-    private final AtomicLong versionMismatches = new AtomicLong(0);
-    private InputStreamStatistics mergedStats;
-
-    private InputStreamStatistics() {
+  private final class MetricUpdatingDurationTracker
+      implements DurationTracker {
+
+    private final String symbol;
+
+    private boolean failed;
+
+    private MetricUpdatingDurationTracker(
+        final String symbol,
+        final long count) {
+      this.symbol = symbol;
+      incrementMutableCounter(symbol, count);
+    }
+
+    @Override
+    public void failed() {
+      failed = true;
     }
 
     /**
-     * Seek backwards, incrementing the seek and backward seek counters.
-     * @param negativeOffset how far was the seek?
-     * This is expected to be negative.
+     * Close: on failure increment any mutable counter of
+     * failures.
      */
+    @Override
+    public void close() {
+      if (failed) {
+        incrementMutableCounter(symbol + SUFFIX_FAILURES, 1);
+      }
+    }
+  }
+
+  /**
+   * Duration Tracker Factory for updating metrics.
+   */
+  private final class MetricDurationTrackerFactory
+      implements DurationTrackerFactory {
+
+    @Override
+    public DurationTracker trackDuration(final String key, final long count) {
+      return new MetricUpdatingDurationTracker(key, count);
+    }
+
+  }
+
+  /**
+   * Statistics updated by an S3AInputStream during its actual operation.
+   * <p>
+   * When {@code unbuffer()} is called, the changed numbers are propagated
+   * to the S3AFileSystem metrics.
+   * </p>
+   * <p>
+   * When {@code close()} is called, the final set of numbers are propagated
+   * to the S3AFileSystem metrics.
+   * </p>
+   * The {@link FileSystem.Statistics} statistics passed in are also
+   * updated. This ensures that whichever thread calls close() gets the
+   * total count of bytes read, even if any work is done in other
+   * threads.
+   *
+   */
+  private final class InputStreamStatistics
+      extends AbstractS3AStatisticsSource
+      implements S3AInputStreamStatistics {
+
+    /**
+     * Distance used when incrementing FS stats.
+     */
+    private static final int DISTANCE = 5;
+
+    /**
+     * FS statistics for the thread creating the stream.
+     */
+    private final FileSystem.Statistics filesystemStatistics;
+
+    /**
+     * The statistics from the last merge.
+     */
+    private IOStatisticsSnapshot mergedStats;
+
+    /*
+    The core counters are extracted to atomic longs for slightly
+    faster resolution on the critical paths, especially single byte
+    reads and the like.
+     */
+    private final AtomicLong aborted;
+    private final AtomicLong backwardSeekOperations;
+    private final AtomicLong bytesBackwardsOnSeek;
+    private final AtomicLong bytesDiscardedInAbort;
+    /** Bytes read by the application. */
+    private final AtomicLong bytesRead;
+    private final AtomicLong bytesDiscardedInClose;
+    private final AtomicLong bytesDiscardedOnSeek;
+    private final AtomicLong bytesSkippedOnSeek;
+    private final AtomicLong closed;
+    private final AtomicLong forwardSeekOperations;
+    private final AtomicLong openOperations;
+    private final AtomicLong readExceptions;
+    private final AtomicLong readsIncomplete;
+    private final AtomicLong readOperations;
+    private final AtomicLong readFullyOperations;
+    private final AtomicLong seekOperations;
+
+    /** Bytes read by the application and any when draining streams . */
+    private final AtomicLong totalBytesRead;
+
+    /**
+     * Instantiate.
+     * @param filesystemStatistics FS Statistics to update in close().
+     */
+    private InputStreamStatistics(
+        @Nullable FileSystem.Statistics filesystemStatistics) {
+      this.filesystemStatistics = filesystemStatistics;
+      IOStatisticsStore st = iostatisticsStore()
+          .withCounters(
+              StreamStatisticNames.STREAM_READ_ABORTED,
+              StreamStatisticNames.STREAM_READ_BYTES_DISCARDED_ABORT,
+              StreamStatisticNames.STREAM_READ_CLOSED,
+              StreamStatisticNames.STREAM_READ_BYTES_DISCARDED_CLOSE,
+              StreamStatisticNames.STREAM_READ_CLOSE_OPERATIONS,
+              StreamStatisticNames.STREAM_READ_OPENED,
+              StreamStatisticNames.STREAM_READ_BYTES,
+              StreamStatisticNames.STREAM_READ_EXCEPTIONS,
+              StreamStatisticNames.STREAM_READ_FULLY_OPERATIONS,
+              StreamStatisticNames.STREAM_READ_OPERATIONS,
+              StreamStatisticNames.STREAM_READ_OPERATIONS_INCOMPLETE,
+              StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS,
+              StreamStatisticNames.STREAM_READ_SEEK_POLICY_CHANGED,
+              StreamStatisticNames.STREAM_READ_SEEK_BACKWARD_OPERATIONS,
+              StreamStatisticNames.STREAM_READ_SEEK_FORWARD_OPERATIONS,
+              StreamStatisticNames.STREAM_READ_SEEK_BYTES_BACKWARDS,
+              StreamStatisticNames.STREAM_READ_SEEK_BYTES_DISCARDED,
+              StreamStatisticNames.STREAM_READ_SEEK_BYTES_SKIPPED,
+              StreamStatisticNames.STREAM_READ_TOTAL_BYTES,
+              StreamStatisticNames.STREAM_READ_UNBUFFERED,
+              StreamStatisticNames.STREAM_READ_VERSION_MISMATCHES)
+          .withGauges(STREAM_READ_GAUGE_INPUT_POLICY)
+          .withDurationTracking(ACTION_HTTP_GET_REQUEST)
+          .build();
+      setIOStatistics(st);
+      aborted = st.getCounterReference(
+          StreamStatisticNames.STREAM_READ_ABORTED);
+      backwardSeekOperations = st.getCounterReference(
+          StreamStatisticNames.STREAM_READ_SEEK_BACKWARD_OPERATIONS);
+      bytesBackwardsOnSeek = st.getCounterReference(
+          StreamStatisticNames.STREAM_READ_SEEK_BYTES_BACKWARDS);
+      bytesDiscardedInAbort = st.getCounterReference(
+          StreamStatisticNames.STREAM_READ_BYTES_DISCARDED_ABORT);
+      bytesRead = st.getCounterReference(
+          StreamStatisticNames.STREAM_READ_BYTES);
+      bytesDiscardedInClose = st.getCounterReference(
+          StreamStatisticNames.STREAM_READ_BYTES_DISCARDED_CLOSE);
+      bytesDiscardedOnSeek = st.getCounterReference(
+          StreamStatisticNames.STREAM_READ_SEEK_BYTES_DISCARDED);
+      bytesSkippedOnSeek = st.getCounterReference(
+          StreamStatisticNames.STREAM_READ_SEEK_BYTES_SKIPPED);
+      closed = st.getCounterReference(
+          StreamStatisticNames.STREAM_READ_CLOSED);
+      forwardSeekOperations = st.getCounterReference(
+          StreamStatisticNames.STREAM_READ_SEEK_FORWARD_OPERATIONS);
+      openOperations = st.getCounterReference(
+          StreamStatisticNames.STREAM_READ_OPENED);
+      readExceptions = st.getCounterReference(
+          StreamStatisticNames.STREAM_READ_EXCEPTIONS);
+      readsIncomplete = st.getCounterReference(
+          StreamStatisticNames.STREAM_READ_OPERATIONS_INCOMPLETE);
+      readOperations = st.getCounterReference(
+          StreamStatisticNames.STREAM_READ_OPERATIONS);
+      readFullyOperations = st.getCounterReference(
+          StreamStatisticNames.STREAM_READ_FULLY_OPERATIONS);
+      seekOperations = st.getCounterReference(
+          StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS);
+      totalBytesRead = st.getCounterReference(
+          StreamStatisticNames.STREAM_READ_TOTAL_BYTES);
+      setIOStatistics(st);
+      // create initial snapshot of merged statistics
+      mergedStats = snapshotIOStatistics(st);
+    }
+
+    /**
+     * Increment a named counter by one.
+     * @param name counter name
+     * @return the new value
+     */
+    private long increment(String name) {
+      return increment(name, 1);
+    }
+
+    /**
+     * Increment a named counter by a given value.
+     * @param name counter name
+     * @param value value to increment by.
+     * @return the new value
+     */
+    private long increment(String name, long value) {
+      return incCounter(name, value);
+    }
+
+    /**
+     * {@inheritDoc}.
+     * Increments the number of seek operations,
+     * and backward seek operations.
+     * The offset is inverted and used as the increment
+     * of {@link #bytesBackwardsOnSeek}.
+     */
+    @Override
     public void seekBackwards(long negativeOffset) {
-      seekOperations++;
-      backwardSeekOperations++;
-      bytesBackwardsOnSeek -= negativeOffset;
+      seekOperations.incrementAndGet();
+      backwardSeekOperations.incrementAndGet();
+      bytesBackwardsOnSeek.addAndGet(-negativeOffset);
     }
 
     /**
-     * Record a forward seek, adding a seek operation, a forward
-     * seek operation, and any bytes skipped.
-     * @param skipped number of bytes skipped by reading from the stream.
-     * If the seek was implemented by a close + reopen, set this to zero.
+     * {@inheritDoc}.
+     * Increment the number of seek and forward seek
+     * operations, as well as counters of bytes skipped
+     * and bytes read in seek, where appropriate.
+     * Bytes read in seek are also added to the totalBytesRead
+     * counter.
      */
-    public void seekForwards(long skipped) {
-      seekOperations++;
-      forwardSeekOperations++;
+    @Override
+    public void seekForwards(final long skipped,
+        long bytesReadInSeek) {
+      seekOperations.incrementAndGet();
+      forwardSeekOperations.incrementAndGet();
       if (skipped > 0) {
-        bytesSkippedOnSeek += skipped;
+        bytesSkippedOnSeek.addAndGet(skipped);
+      }
+      if (bytesReadInSeek > 0) {
+        bytesDiscardedOnSeek.addAndGet(bytesReadInSeek);
+        totalBytesRead.addAndGet(bytesReadInSeek);
       }
     }
 
     /**
-     * The inner stream was opened.
-     * @return the previous count
+     * {@inheritDoc}.
+     * Use {@code getAnIncrement()} on {@link #openOperations}
+     * so that on invocation 1 it returns 0.
+     * The caller will know that this is the first invocation.
      */
+    @Override
     public long streamOpened() {
-      long count = openOperations;
-      openOperations++;
-      return count;
+      return openOperations.getAndIncrement();
     }
 
     /**
-     * The inner stream was closed.
-     * @param abortedConnection flag to indicate the stream was aborted,
-     * rather than closed cleanly
-     * @param remainingInCurrentRequest the number of bytes remaining in
-     * the current request.
+     * {@inheritDoc}.
+     * If the connection was aborted, increment {@link #aborted}
+     * and add the byte's remaining count to {@link #bytesDiscardedInAbort}.
+     * If not aborted, increment {@link #closed} and
+     * then {@link #bytesDiscardedInClose} and {@link #totalBytesRead}
+     * with the bytes remaining value.
      */
+    @Override
     public void streamClose(boolean abortedConnection,
         long remainingInCurrentRequest) {
-      closeOperations++;
       if (abortedConnection) {
-        this.aborted++;
-        bytesDiscardedInAbort += remainingInCurrentRequest;
+        // the connection was aborted.
+        // update the counter of abort() calls and bytes discarded
+        aborted.incrementAndGet();
+        bytesDiscardedInAbort.addAndGet(remainingInCurrentRequest);
       } else {
-        closed++;
-        bytesReadInClose += remainingInCurrentRequest;
+        // connection closed, possibly draining the stream of surplus
+        // bytes.
+        closed.incrementAndGet();
+        bytesDiscardedInClose.addAndGet(remainingInCurrentRequest);
+        totalBytesRead.addAndGet(remainingInCurrentRequest);
       }
     }
 
     /**
-     * An ignored stream read exception was received.
+     * {@inheritDoc}.
      */
+    @Override
     public void readException() {
-      readExceptions++;
+      readExceptions.incrementAndGet();
     }
 
     /**
-     * Increment the bytes read counter by the number of bytes;
-     * no-op if the argument is negative.
-     * @param bytes number of bytes read
+     * {@inheritDoc}.
+     * If the byte counter is positive, increment bytesRead and totalBytesRead.
      */
+    @Override
     public void bytesRead(long bytes) {
       if (bytes > 0) {
-        bytesRead += bytes;
+        bytesRead.addAndGet(bytes);
+        totalBytesRead.addAndGet(bytes);
       }
     }
 
-    /**
-     * A {@code read(byte[] buf, int off, int len)} operation has started.
-     * @param pos starting position of the read
-     * @param len length of bytes to read
-     */
+    @Override
     public void readOperationStarted(long pos, long len) {
-      readOperations++;
+      readOperations.incrementAndGet();
     }
 
-    /**
-     * A {@code PositionedRead.read(position, buffer, offset, length)}
-     * operation has just started.
-     * @param pos starting position of the read
-     * @param len length of bytes to read
-     */
+    @Override
     public void readFullyOperationStarted(long pos, long len) {
-      readFullyOperations++;
+      readFullyOperations.incrementAndGet();
     }
 
     /**
-     * A read operation has completed.
-     * @param requested number of requested bytes
-     * @param actual the actual number of bytes
+     * {@inheritDoc}.
+     * If more data was requested than was actually returned, this
+     * was an incomplete read. Increment {@link #readsIncomplete}.
      */
+    @Override
     public void readOperationCompleted(int requested, int actual) {
       if (requested > actual) {
-        readsIncomplete++;
+        readsIncomplete.incrementAndGet();
       }
     }
 
     /**
-     * Close triggers the merge of statistics into the filesystem's
+     * {@code close()} merges the stream statistics into the filesystem's
      * instrumentation instance.
      */
     @Override
     public void close() {
+      increment(StreamStatisticNames.STREAM_READ_CLOSE_OPERATIONS);
       merge(true);
     }
 
     /**
-     * The input policy has been switched.
-     * @param updatedPolicy enum value of new policy.
+     * {@inheritDoc}.
+     * As well as incrementing the {@code STREAM_READ_SEEK_POLICY_CHANGED}
+     * counter, the
+     * {@code STREAM_READ_GAUGE_INPUT_POLICY} gauge is set to the new value.
+     *
      */
+    @Override
     public void inputPolicySet(int updatedPolicy) {
-      policySetCount++;
-      inputPolicy = updatedPolicy;
+      increment(StreamStatisticNames.STREAM_READ_SEEK_POLICY_CHANGED);
+      localIOStatistics().setGauge(STREAM_READ_GAUGE_INPUT_POLICY,
+          updatedPolicy);
     }
 
     /**
-     * Get a reference to the version mismatch counter.
-     * @return a counter which can be incremented.
+     * Get the inner class's IO Statistics. This is
+     * needed to avoid findbugs warnings about ambiguity.
+     * @return the Input Stream's statistics.
      */
-    public AtomicLong getVersionMismatchCounter() {
-      return versionMismatches;
+    private IOStatisticsStore localIOStatistics() {
+      return InputStreamStatistics.super.getIOStatistics();
+    }
+
+    /**
+     * The change tracker increments {@code versionMismatches} on any
+     * mismatch.
+     * @return change tracking.
+     */
+    @Override
+    public ChangeTrackerStatistics getChangeTrackerStatistics() {
+      return new CountingChangeTracker(
+          localIOStatistics().getCounterReference(
+              StreamStatisticNames.STREAM_READ_VERSION_MISMATCHES));
     }
 
     /**
@@ -812,323 +1078,510 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
     public String toString() {
       final StringBuilder sb = new StringBuilder(
           "StreamStatistics{");
-      sb.append("OpenOperations=").append(openOperations);
-      sb.append(", CloseOperations=").append(closeOperations);
-      sb.append(", Closed=").append(closed);
-      sb.append(", Aborted=").append(aborted);
-      sb.append(", SeekOperations=").append(seekOperations);
-      sb.append(", ReadExceptions=").append(readExceptions);
-      sb.append(", ForwardSeekOperations=")
-          .append(forwardSeekOperations);
-      sb.append(", BackwardSeekOperations=")
-          .append(backwardSeekOperations);
-      sb.append(", BytesSkippedOnSeek=").append(bytesSkippedOnSeek);
-      sb.append(", BytesBackwardsOnSeek=").append(bytesBackwardsOnSeek);
-      sb.append(", BytesRead=").append(bytesRead);
-      sb.append(", BytesRead excluding skipped=")
-          .append(bytesRead - bytesSkippedOnSeek);
-      sb.append(", ReadOperations=").append(readOperations);
-      sb.append(", ReadFullyOperations=").append(readFullyOperations);
-      sb.append(", ReadsIncomplete=").append(readsIncomplete);
-      sb.append(", BytesReadInClose=").append(bytesReadInClose);
-      sb.append(", BytesDiscardedInAbort=").append(bytesDiscardedInAbort);
-      sb.append(", InputPolicy=").append(inputPolicy);
-      sb.append(", InputPolicySetCount=").append(policySetCount);
-      sb.append(", versionMismatches=").append(versionMismatches.get());
+      sb.append(IOStatisticsLogging.ioStatisticsToString(
+          localIOStatistics()));
       sb.append('}');
       return sb.toString();
     }
 
     /**
+     * {@inheritDoc}
+     * Increment the counter {@code STREAM_READ_UNBUFFERED}
+     * and then merge the current set of statistics into the
+     * FileSystem's statistics through {@link #merge(boolean)}.
+     */
+    @Override
+    public void unbuffered() {
+      increment(STREAM_READ_UNBUFFERED);
+      merge(false);
+    }
+
+    /**
      * Merge the statistics into the filesystem's instrumentation instance.
-     * Takes a diff between the current version of the stats and the
-     * version of the stats when merge was last called, and merges the diff
-     * into the instrumentation instance. Used to periodically merge the
-     * stats into the fs-wide stats. <b>Behavior is undefined if called on a
-     * closed instance.</b>
+     * <p>
+     *   If the merge is invoked because the stream has been closed,
+     *   then all statistics are merged, and the filesystem
+     *   statistics of {@link #filesystemStatistics} updated
+     *   with the bytes read values.
+     * </p>
+     * <p>
+     *   Whichever thread close()d the stream will have its counters
+     *   updated.
+     * </p>
+     * <p>
+     *   If the merge is due to an unbuffer() call, the change in all
+     *   counters since the last merge will be pushed to the Instrumentation's
+     *   counters.
+     * </p>
+     *
+     * @param isClosed is this merge invoked because the stream is closed?
      */
-    void merge(boolean isClosed) {
-      if (mergedStats != null) {
-        mergeInputStreamStatistics(diff(mergedStats));
-      } else {
-        mergeInputStreamStatistics(this);
-      }
-      // If stats are closed, no need to create another copy
-      if (!isClosed) {
-        mergedStats = copy();
+    private void merge(boolean isClosed) {
+
+      IOStatisticsStore ioStatistics = localIOStatistics();
+      LOG.debug("Merging statistics into FS statistics in {}: {}",
+          (isClosed ? "close()" : "unbuffer()"),
+          demandStringifyIOStatistics(ioStatistics));
+      promoteInputStreamCountersToMetrics();
+      mergedStats = snapshotIOStatistics(localIOStatistics());
+
+      if (isClosed) {
+        // stream is being closed.
+        // merge in all the IOStatistics
+        S3AInstrumentation.this.getIOStatistics().aggregate(ioStatistics);
+
+        // increment the filesystem statistics for this thread.
+        if (filesystemStatistics != null) {
+          long t = getTotalBytesRead();
+          filesystemStatistics.incrementBytesRead(t);
+          filesystemStatistics.incrementBytesReadByDistance(DISTANCE, t);
+        }
       }
     }
 
     /**
-     * Returns a diff between this {@link InputStreamStatistics} instance and
-     * the given {@link InputStreamStatistics} instance.
+     * Propagate a counter from the instance-level statistics
+     * to the S3A instrumentation, subtracting the previous merged value.
+     * @param name statistic to promote
      */
-    private InputStreamStatistics diff(InputStreamStatistics inputStats) {
-      InputStreamStatistics diff = new InputStreamStatistics();
-      diff.openOperations = openOperations - inputStats.openOperations;
-      diff.closeOperations = closeOperations - inputStats.closeOperations;
-      diff.closed = closed - inputStats.closed;
-      diff.aborted = aborted - inputStats.aborted;
-      diff.seekOperations = seekOperations - inputStats.seekOperations;
-      diff.readExceptions = readExceptions - inputStats.readExceptions;
-      diff.forwardSeekOperations =
-              forwardSeekOperations - inputStats.forwardSeekOperations;
-      diff.backwardSeekOperations =
-              backwardSeekOperations - inputStats.backwardSeekOperations;
-      diff.bytesRead = bytesRead - inputStats.bytesRead;
-      diff.bytesSkippedOnSeek =
-              bytesSkippedOnSeek - inputStats.bytesSkippedOnSeek;
-      diff.bytesBackwardsOnSeek =
-              bytesBackwardsOnSeek - inputStats.bytesBackwardsOnSeek;
-      diff.readOperations = readOperations - inputStats.readOperations;
-      diff.readFullyOperations =
-              readFullyOperations - inputStats.readFullyOperations;
-      diff.readsIncomplete = readsIncomplete - inputStats.readsIncomplete;
-      diff.bytesReadInClose = bytesReadInClose - inputStats.bytesReadInClose;
-      diff.bytesDiscardedInAbort =
-              bytesDiscardedInAbort - inputStats.bytesDiscardedInAbort;
-      diff.policySetCount = policySetCount - inputStats.policySetCount;
-      diff.inputPolicy = inputPolicy - inputStats.inputPolicy;
-      diff.versionMismatches.set(versionMismatches.longValue() -
-              inputStats.versionMismatches.longValue());
-      return diff;
+    void promoteIOCounter(String name) {
+      incrementMutableCounter(name,
+          lookupCounterValue(name)
+              - mergedStats.counters().get(name));
     }
 
     /**
-     * Returns a new {@link InputStreamStatistics} instance with all the same
-     * values as this {@link InputStreamStatistics}.
+     * Merge in the statistics of a single input stream into
+     * the filesystem-wide metrics counters.
+     * This does not update the FS IOStatistics values.
      */
-    private InputStreamStatistics copy() {
-      InputStreamStatistics copy = new InputStreamStatistics();
-      copy.openOperations = openOperations;
-      copy.closeOperations = closeOperations;
-      copy.closed = closed;
-      copy.aborted = aborted;
-      copy.seekOperations = seekOperations;
-      copy.readExceptions = readExceptions;
-      copy.forwardSeekOperations = forwardSeekOperations;
-      copy.backwardSeekOperations = backwardSeekOperations;
-      copy.bytesRead = bytesRead;
-      copy.bytesSkippedOnSeek = bytesSkippedOnSeek;
-      copy.bytesBackwardsOnSeek = bytesBackwardsOnSeek;
-      copy.readOperations = readOperations;
-      copy.readFullyOperations = readFullyOperations;
-      copy.readsIncomplete = readsIncomplete;
-      copy.bytesReadInClose = bytesReadInClose;
-      copy.bytesDiscardedInAbort = bytesDiscardedInAbort;
-      copy.policySetCount = policySetCount;
-      copy.inputPolicy = inputPolicy;
-      return copy;
+    private void promoteInputStreamCountersToMetrics() {
+      // iterate through all the counters
+      localIOStatistics().counters()
+          .keySet().stream()
+          .forEach(e -> promoteIOCounter(e));
+    }
+
+    @Override
+    public long getCloseOperations() {
+      return lookupCounterValue(
+          StreamStatisticNames.STREAM_READ_CLOSE_OPERATIONS);
+    }
+
+    @Override
+    public long getClosed() {
+      return lookupCounterValue(StreamStatisticNames.STREAM_READ_CLOSED);
     }
+
+    @Override
+    public long getAborted() {
+      return lookupCounterValue(StreamStatisticNames.STREAM_READ_ABORTED);
+    }
+
+    @Override
+    public long getForwardSeekOperations() {
+      return lookupCounterValue(
+          StreamStatisticNames.STREAM_READ_SEEK_FORWARD_OPERATIONS);
+    }
+
+    @Override
+    public long getBackwardSeekOperations() {
+      return lookupCounterValue(
+          StreamStatisticNames.STREAM_READ_SEEK_BACKWARD_OPERATIONS);
+    }
+
+    @Override
+    public long getBytesRead() {
+      return lookupCounterValue(StreamStatisticNames.STREAM_READ_BYTES);
+    }
+
+    @Override
+    public long getTotalBytesRead() {
+      return lookupCounterValue(StreamStatisticNames.STREAM_READ_TOTAL_BYTES);
+    }
+
+    @Override
+    public long getBytesSkippedOnSeek() {
+      return lookupCounterValue(
+          StreamStatisticNames.STREAM_READ_SEEK_BYTES_SKIPPED);
+    }
+
+    @Override
+    public long getBytesBackwardsOnSeek() {
+      return lookupCounterValue(
+          StreamStatisticNames.STREAM_READ_SEEK_BYTES_BACKWARDS);
+    }
+
+    @Override
+    public long getBytesReadInClose() {
+      return lookupCounterValue(
+          StreamStatisticNames.STREAM_READ_BYTES_DISCARDED_CLOSE);
+    }
+
+    @Override
+    public long getBytesDiscardedInAbort() {
+      return lookupCounterValue(
+          StreamStatisticNames.STREAM_READ_BYTES_DISCARDED_ABORT);
+    }
+
+    @Override
+    public long getOpenOperations() {
+      return lookupCounterValue(StreamStatisticNames.STREAM_READ_OPENED);
+    }
+
+    @Override
+    public long getSeekOperations() {
+      return lookupCounterValue(
+          StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS);
+    }
+
+    @Override
+    public long getReadExceptions() {
+      return lookupCounterValue(
+          StreamStatisticNames.STREAM_READ_EXCEPTIONS);
+    }
+
+    @Override
+    public long getReadOperations() {
+      return lookupCounterValue(StreamStatisticNames.STREAM_READ_OPERATIONS);
+    }
+
+    @Override
+    public long getReadFullyOperations() {
+      return lookupCounterValue(
+          StreamStatisticNames.STREAM_READ_FULLY_OPERATIONS);
+    }
+
+    @Override
+    public long getReadsIncomplete() {
+      return lookupCounterValue(
+          StreamStatisticNames.STREAM_READ_OPERATIONS_INCOMPLETE);
+    }
+
+    @Override
+    public long getPolicySetCount() {
+      return lookupCounterValue(
+          StreamStatisticNames.STREAM_READ_SEEK_POLICY_CHANGED);
+    }
+
+    @Override
+    public long getVersionMismatches() {
+      return lookupCounterValue(
+          StreamStatisticNames.STREAM_READ_VERSION_MISMATCHES);
+    }
+
+    @Override
+    public long getInputPolicy() {
+      return localIOStatistics().gauges()
+          .get(STREAM_READ_GAUGE_INPUT_POLICY);
+    }
+
+    @Override
+    public DurationTracker initiateGetRequest() {
+      return trackDuration(ACTION_HTTP_GET_REQUEST);
+    }
+
   }
 
   /**
    * Create a stream output statistics instance.
+   * @param filesystemStatistics thread-local FS statistics.
    * @return the new instance
    */
-  OutputStreamStatistics newOutputStreamStatistics(Statistics statistics) {
-    return new OutputStreamStatistics(statistics);
+  public BlockOutputStreamStatistics newOutputStreamStatistics(
+      FileSystem.Statistics filesystemStatistics) {
+    return new OutputStreamStatistics(filesystemStatistics);
   }
 
   /**
    * Merge in the statistics of a single output stream into
    * the filesystem-wide statistics.
-   * @param statistics stream statistics
+   * @param source stream statistics
    */
-  private void mergeOutputStreamStatistics(OutputStreamStatistics statistics) {
-    incrementCounter(STREAM_WRITE_TOTAL_TIME, statistics.totalUploadDuration());
-    incrementCounter(STREAM_WRITE_QUEUE_DURATION, statistics.queueDuration);
-    incrementCounter(STREAM_WRITE_TOTAL_DATA, statistics.bytesUploaded);
+  private void mergeOutputStreamStatistics(
+      OutputStreamStatistics source) {
+    incrementCounter(STREAM_WRITE_TOTAL_TIME, source.totalUploadDuration());
+    incrementCounter(STREAM_WRITE_QUEUE_DURATION, source.queueDuration);
+    incrementCounter(STREAM_WRITE_TOTAL_DATA, source.bytesUploaded);
     incrementCounter(STREAM_WRITE_BLOCK_UPLOADS,
-        statistics.blockUploadsCompleted);
+        source.blockUploadsCompleted);
+    incrementCounter(STREAM_WRITE_EXCEPTIONS,
+        source.lookupCounterValue(
+            StreamStatisticNames.STREAM_WRITE_EXCEPTIONS));
+    // merge in all the IOStatistics
+    this.getIOStatistics().aggregate(source.getIOStatistics());
   }
 
   /**
    * Statistics updated by an output stream during its actual operation.
-   * Some of these stats may be relayed. However, as block upload is
-   * spans multiple
+   * <p>
+   * Some of these stats are propagated to any passed in
+   * {@link FileSystem.Statistics} instance; this is done
+   * in close() for better cross-thread accounting.
+   * </p>
+   * <p>
+   *   Some of the collected statistics are not directly served via
+   *   IOStatistics.
+   *   They are added to the instrumentation IOStatistics and metric counters
+   *   during the {@link #mergeOutputStreamStatistics(OutputStreamStatistics)}
+   *   operation.
+   * </p>
    */
-  @InterfaceAudience.Private
-  @InterfaceStability.Unstable
-  public final class OutputStreamStatistics implements Closeable {
-    private final AtomicLong blocksSubmitted = new AtomicLong(0);
-    private final AtomicLong blocksInQueue = new AtomicLong(0);
+  private final class OutputStreamStatistics
+      extends AbstractS3AStatisticsSource
+      implements BlockOutputStreamStatistics {
+
     private final AtomicLong blocksActive = new AtomicLong(0);
     private final AtomicLong blockUploadsCompleted = new AtomicLong(0);
-    private final AtomicLong blockUploadsFailed = new AtomicLong(0);
-    private final AtomicLong bytesPendingUpload = new AtomicLong(0);
 
-    private final AtomicLong bytesUploaded = new AtomicLong(0);
+    private final AtomicLong bytesWritten;
+    private final AtomicLong bytesUploaded;
     private final AtomicLong transferDuration = new AtomicLong(0);
     private final AtomicLong queueDuration = new AtomicLong(0);
-    private final AtomicLong exceptionsInMultipartFinalize = new AtomicLong(0);
     private final AtomicInteger blocksAllocated = new AtomicInteger(0);
     private final AtomicInteger blocksReleased = new AtomicInteger(0);
 
-    private Statistics statistics;
+    private final FileSystem.Statistics filesystemStatistics;
 
-    public OutputStreamStatistics(Statistics statistics){
-      this.statistics = statistics;
+    /**
+     * Instantiate.
+     * @param filesystemStatistics FS Statistics to update in close().
+     */
+    private OutputStreamStatistics(
+        @Nullable FileSystem.Statistics filesystemStatistics) {
+      this.filesystemStatistics = filesystemStatistics;
+      IOStatisticsStore st = iostatisticsStore()
+          .withCounters(
+              StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS,
+              STREAM_WRITE_BYTES.getSymbol(),
+              STREAM_WRITE_EXCEPTIONS.getSymbol(),
+              StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING,
+              STREAM_WRITE_TOTAL_TIME.getSymbol(),
+              STREAM_WRITE_QUEUE_DURATION.getSymbol(),
+              STREAM_WRITE_TOTAL_DATA.getSymbol(),
+              STREAM_WRITE_EXCEPTIONS.getSymbol(),
+              STREAM_WRITE_EXCEPTIONS_COMPLETING_UPLOADS.getSymbol())
+          .withGauges(
+              STREAM_WRITE_BLOCK_UPLOADS_PENDING.getSymbol(),
+              STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING.getSymbol())
+          .withDurationTracking(ACTION_EXECUTOR_ACQUIRED)
+          .build();
+      setIOStatistics(st);
+      // these are extracted to avoid lookups on heavily used counters.
+      bytesUploaded = st.getCounterReference(
+          STREAM_WRITE_TOTAL_DATA.getSymbol());
+      bytesWritten = st.getCounterReference(
+          StreamStatisticNames.STREAM_WRITE_BYTES);
     }
 
     /**
-     * A block has been allocated.
+     * Increment the Statistic gauge and the local IOStatistics
+     * equivalent.
+     * @param statistic statistic
+     * @param v value.
+     * @return local IOStatistic value
      */
-    void blockAllocated() {
+    private long incAllGauges(Statistic statistic, long v) {
+      incrementGauge(statistic, v);
+      return incGauge(statistic.getSymbol(), v);
+    }
+
+    @Override
+    public void blockAllocated() {
       blocksAllocated.incrementAndGet();
     }
 
+    @Override
+    public void blockReleased() {
+      blocksReleased.incrementAndGet();
+    }
+
     /**
-     * A block has been released.
+     * {@inheritDoc}
+     * Increments the counter of block uplaods, and the gauges
+     * of block uploads pending (1) and the bytes pending (blockSize).
      */
-    void blockReleased() {
-      blocksReleased.incrementAndGet();
+    @Override
+    public void blockUploadQueued(int blockSize) {
+      incCounter(StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS);
+      incAllGauges(STREAM_WRITE_BLOCK_UPLOADS_PENDING, 1);
+      incAllGauges(STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING, blockSize);
     }
 
     /**
-     * Block is queued for upload.
+     * {@inheritDoc}
+     * Update {@link #queueDuration} with queue duration, decrement
+     * {@code STREAM_WRITE_BLOCK_UPLOADS_PENDING} gauge and increment
+     * {@code STREAM_WRITE_BLOCK_UPLOADS_ACTIVE}.
      */
-    void blockUploadQueued(int blockSize) {
-      blocksSubmitted.incrementAndGet();
-      blocksInQueue.incrementAndGet();
-      bytesPendingUpload.addAndGet(blockSize);
-      incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_PENDING, 1);
-      incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING, blockSize);
-    }
-
-    /** Queued block has been scheduled for upload. */
-    void blockUploadStarted(long duration, int blockSize) {
-      queueDuration.addAndGet(duration);
-      blocksInQueue.decrementAndGet();
-      blocksActive.incrementAndGet();
-      incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_PENDING, -1);
-      incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_ACTIVE, 1);
-    }
-
-    /** A block upload has completed. */
-    void blockUploadCompleted(long duration, int blockSize) {
-      this.transferDuration.addAndGet(duration);
-      incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_ACTIVE, -1);
-      blocksActive.decrementAndGet();
+    @Override
+    public void blockUploadStarted(Duration timeInQueue, int blockSize) {
+      // the local counter is used in toString reporting.
+      queueDuration.addAndGet(timeInQueue.toMillis());
+      // update the duration fields in the IOStatistics.
+      localIOStatistics().addTimedOperation(
+          ACTION_EXECUTOR_ACQUIRED,
+          timeInQueue);
+      incAllGauges(STREAM_WRITE_BLOCK_UPLOADS_PENDING, -1);
+      incAllGauges(STREAM_WRITE_BLOCK_UPLOADS_ACTIVE, 1);
+    }
+
+    /**
+     * Get the inner class's IO Statistics. This is
+     * needed to avoid findbugs warnings about ambiguity.
+     * @return the Input Stream's statistics.
+     */
+    private IOStatisticsStore localIOStatistics() {
+      return OutputStreamStatistics.super.getIOStatistics();
+    }
+
+    /**
+     * {@inheritDoc}
+     * Increment the transfer duration; decrement the
+     * {@code STREAM_WRITE_BLOCK_UPLOADS_ACTIVE} gauge.
+     */
+    @Override
+    public void blockUploadCompleted(
+        Duration timeSinceUploadStarted,
+        int blockSize) {
+      transferDuration.addAndGet(timeSinceUploadStarted.toMillis());
+      incAllGauges(STREAM_WRITE_BLOCK_UPLOADS_ACTIVE, -1);
       blockUploadsCompleted.incrementAndGet();
     }
 
     /**
      *  A block upload has failed.
      *  A final transfer completed event is still expected, so this
-     *  does not decrement the active block counter.
+     *  does not decrement any gauges.
      */
-    void blockUploadFailed(long duration, int blockSize) {
-      blockUploadsFailed.incrementAndGet();
+    @Override
+    public void blockUploadFailed(
+        Duration timeSinceUploadStarted,
+        int blockSize) {
+      incCounter(StreamStatisticNames.STREAM_WRITE_EXCEPTIONS);
     }
 
-    /** Intermediate report of bytes uploaded. */
-    void bytesTransferred(long byteCount) {
+    /**
+     * Intermediate report of bytes uploaded.
+     * Increment counters of bytes upload, reduce the counter and
+     * gauge of pending bytes.;
+     * @param byteCount bytes uploaded
+     */
+    @Override
+    public void bytesTransferred(long byteCount) {
       bytesUploaded.addAndGet(byteCount);
-      statistics.incrementBytesWritten(byteCount);
-      bytesPendingUpload.addAndGet(-byteCount);
-      incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING, -byteCount);
+      incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING, -byteCount);
     }
 
-    /**
-     * Note exception in a multipart complete.
-     * @param count count of exceptions
-     */
-    void exceptionInMultipartComplete(int count) {
+    @Override
+    public void exceptionInMultipartComplete(int count) {
       if (count > 0) {
-        exceptionsInMultipartFinalize.addAndGet(count);
+        incCounter(
+            STREAM_WRITE_EXCEPTIONS_COMPLETING_UPLOADS.getSymbol(),
+            count);
       }
     }
 
-    /**
-     * Note an exception in a multipart abort.
-     */
-    void exceptionInMultipartAbort() {
-      exceptionsInMultipartFinalize.incrementAndGet();
+    @Override
+    public void exceptionInMultipartAbort() {
+      incCounter(
+          STREAM_WRITE_EXCEPTIONS_COMPLETING_UPLOADS.getSymbol());
     }
 
-    /**
-     * Get the number of bytes pending upload.
-     * @return the number of bytes in the pending upload state.
-     */
+    @Override
     public long getBytesPendingUpload() {
-      return bytesPendingUpload.get();
+      return lookupGaugeValue(
+          STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING.getSymbol());
     }
 
-    /**
-     * Data has been uploaded to be committed in a subsequent operation;
-     * to be called at the end of the write.
-     * @param size size in bytes
-     */
+    @Override
     public void commitUploaded(long size) {
       incrementCounter(COMMITTER_BYTES_UPLOADED, size);
     }
 
-    /**
-     * Output stream has closed.
-     * Trigger merge in of all statistics not updated during operation.
-     */
     @Override
     public void close() {
-      if (bytesPendingUpload.get() > 0) {
+      if (getBytesPendingUpload() > 0) {
         LOG.warn("Closing output stream statistics while data is still marked" +
             " as pending upload in {}", this);
       }
       mergeOutputStreamStatistics(this);
+      // and patch the FS statistics.
+      // provided the stream is closed in the worker thread, this will
+      // ensure that the thread-specific worker stats are updated.
+      if (filesystemStatistics != null) {
+        filesystemStatistics.incrementBytesWritten(bytesUploaded.get());
+      }
     }
 
-    long averageQueueTime() {
-      return blocksSubmitted.get() > 0 ?
-          (queueDuration.get() / blocksSubmitted.get()) : 0;
-    }
-
-    double effectiveBandwidth() {
+    /**
+     * What is the effective bandwidth of this stream's write.
+     * @return the bytes uploaded divided by the total duration.
+     */
+    private double effectiveBandwidth() {
       double duration = totalUploadDuration() / 1000.0;
       return duration > 0 ?
           (bytesUploaded.get() / duration) : 0;
     }
 
-    long totalUploadDuration() {
+    /**
+     * Total of time spend uploading bytes.
+     * @return the transfer duration plus queue duration.
+     */
+    private long totalUploadDuration() {
       return queueDuration.get() + transferDuration.get();
     }
 
-    public int blocksAllocated() {
+    @Override
+    public int getBlocksAllocated() {
       return blocksAllocated.get();
     }
 
-    public int blocksReleased() {
+    @Override
+    public int getBlocksReleased() {
       return blocksReleased.get();
     }
 
     /**
-     * Get counters of blocks actively allocated; my be inaccurate
+     * Get counters of blocks actively allocated; may be inaccurate
      * if the numbers change during the (non-synchronized) calculation.
      * @return the number of actively allocated blocks.
      */
-    public int blocksActivelyAllocated() {
+    @Override
+    public int getBlocksActivelyAllocated() {
       return blocksAllocated.get() - blocksReleased.get();
     }
 
+    /**
+     * Record bytes written.
+     * @param count number of bytes
+     */
+    @Override
+    public void writeBytes(long count) {
+      bytesWritten.addAndGet(count);
+    }
+
+    /**
+     * Get the current count of bytes written.
+     * @return the counter value.
+     */
+    @Override
+    public long getBytesWritten() {
+      return bytesWritten.get();
+    }
 
     @Override
     public String toString() {
       final StringBuilder sb = new StringBuilder(
           "OutputStreamStatistics{");
-      sb.append("blocksSubmitted=").append(blocksSubmitted);
-      sb.append(", blocksInQueue=").append(blocksInQueue);
+      sb.append(localIOStatistics().toString());
       sb.append(", blocksActive=").append(blocksActive);
       sb.append(", blockUploadsCompleted=").append(blockUploadsCompleted);
-      sb.append(", blockUploadsFailed=").append(blockUploadsFailed);
-      sb.append(", bytesPendingUpload=").append(bytesPendingUpload);
-      sb.append(", bytesUploaded=").append(bytesUploaded);
       sb.append(", blocksAllocated=").append(blocksAllocated);
       sb.append(", blocksReleased=").append(blocksReleased);
-      sb.append(", blocksActivelyAllocated=").append(blocksActivelyAllocated());
-      sb.append(", exceptionsInMultipartFinalize=").append(
-          exceptionsInMultipartFinalize);
+      sb.append(", blocksActivelyAllocated=")
+          .append(getBlocksActivelyAllocated());
       sb.append(", transferDuration=").append(transferDuration).append(" ms");
-      sb.append(", queueDuration=").append(queueDuration).append(" ms");
-      sb.append(", averageQueueTime=").append(averageQueueTime()).append(" ms");
       sb.append(", totalUploadDuration=").append(totalUploadDuration())
           .append(" ms");
       sb.append(", effectiveBandwidth=").append(effectiveBandwidth())
@@ -1174,10 +1627,6 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
       incrementCounter(S3GUARD_METADATASTORE_RECORD_READS, count);
     }
 
-    /**
-     * records have been written (including deleted).
-     * @param count number of records written.
-     */
     @Override
     public void recordsWritten(int count) {
       incrementCounter(S3GUARD_METADATASTORE_RECORD_WRITES, count);
@@ -1185,7 +1634,8 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
 
     @Override
     public void directoryMarkedAuthoritative() {
-      incrementCounter(S3GUARD_METADATASTORE_AUTHORITATIVE_DIRECTORIES_UPDATED,
+      incrementCounter(
+          S3GUARD_METADATASTORE_AUTHORITATIVE_DIRECTORIES_UPDATED,
           1);
     }
 
@@ -1200,60 +1650,95 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
   }
 
   /**
-   * Instrumentation exported to S3Guard Committers.
+   * Instrumentation exported to S3A Committers.
+   * The S3AInstrumentation metrics and
+   * {@link #instanceIOStatistics} are updated continuously.
    */
-  @InterfaceAudience.Private
-  @InterfaceStability.Unstable
-  public final class CommitterStatistics {
+  private final class CommitterStatisticsImpl
+      extends AbstractS3AStatisticsSource
+      implements CommitterStatistics {
+
+    private CommitterStatisticsImpl() {
+      IOStatisticsStore st = iostatisticsStore()
+          .withCounters(
+              COMMITTER_BYTES_COMMITTED.getSymbol(),
+              COMMITTER_BYTES_UPLOADED.getSymbol(),
+              COMMITTER_COMMITS_CREATED.getSymbol(),
+              COMMITTER_COMMITS_ABORTED.getSymbol(),
+              COMMITTER_COMMITS_COMPLETED.getSymbol(),
+              COMMITTER_COMMITS_FAILED.getSymbol(),
+              COMMITTER_COMMITS_REVERTED.getSymbol(),
+              COMMITTER_JOBS_FAILED.getSymbol(),
+              COMMITTER_JOBS_SUCCEEDED.getSymbol(),
+              COMMITTER_TASKS_FAILED.getSymbol(),
+              COMMITTER_TASKS_SUCCEEDED.getSymbol())
+          .withDurationTracking(
+              COMMITTER_COMMIT_JOB.getSymbol(),
+              COMMITTER_MATERIALIZE_FILE.getSymbol(),
+              COMMITTER_STAGE_FILE_UPLOAD.getSymbol())
+          .build();
+      setIOStatistics(st);
+    }
+
+    /**
+     * Increment both the local counter and the S3AInstrumentation counters.
+     * @param stat statistic
+     * @param value value
+     * @return the new value
+     */
+    private long increment(Statistic stat, long value) {
+      incrementCounter(stat, value);
+      return incCounter(stat.getSymbol(), value);
+    }
 
     /** A commit has been created. */
+    @Override
     public void commitCreated() {
-      incrementCounter(COMMITTER_COMMITS_CREATED, 1);
+      increment(COMMITTER_COMMITS_CREATED, 1);
     }
 
-    /**
-     * Data has been uploaded to be committed in a subsequent operation.
-     * @param size size in bytes
-     */
+    @Override
     public void commitUploaded(long size) {
-      incrementCounter(COMMITTER_BYTES_UPLOADED, size);
+      increment(COMMITTER_BYTES_UPLOADED, size);
     }
 
-    /**
-     * A commit has been completed.
-     * @param size size in bytes
-     */
+    @Override
     public void commitCompleted(long size) {
-      incrementCounter(COMMITTER_COMMITS_COMPLETED, 1);
-      incrementCounter(COMMITTER_BYTES_COMMITTED, size);
+      increment(COMMITTER_COMMITS_COMPLETED, 1);
+      increment(COMMITTER_BYTES_COMMITTED, size);
     }
 
-    /** A commit has been aborted. */
+    @Override
     public void commitAborted() {
-      incrementCounter(COMMITTER_COMMITS_ABORTED, 1);
+      increment(COMMITTER_COMMITS_ABORTED, 1);
     }
 
+    @Override
     public void commitReverted() {
-      incrementCounter(COMMITTER_COMMITS_REVERTED, 1);
+      increment(COMMITTER_COMMITS_REVERTED, 1);
     }
 
+    @Override
     public void commitFailed() {
-      incrementCounter(COMMITTER_COMMITS_FAILED, 1);
+      increment(COMMITTER_COMMITS_FAILED, 1);
     }
 
+    @Override
     public void taskCompleted(boolean success) {
-      incrementCounter(
-          success ? COMMITTER_TASKS_SUCCEEDED
+      increment(success
+              ? COMMITTER_TASKS_SUCCEEDED
               : COMMITTER_TASKS_FAILED,
           1);
     }
 
+    @Override
     public void jobCompleted(boolean success) {
-      incrementCounter(
-          success ? COMMITTER_JOBS_SUCCEEDED
+      increment(success
+              ? COMMITTER_JOBS_SUCCEEDED
               : COMMITTER_JOBS_FAILED,
           1);
     }
+
   }
 
   /**
@@ -1261,26 +1746,33 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
    * @return an instance of delegation token statistics
    */
   public DelegationTokenStatistics newDelegationTokenStatistics() {
-    return new DelegationTokenStatistics();
+    return new DelegationTokenStatisticsImpl();
   }
 
   /**
    * Instrumentation exported to S3A Delegation Token support.
+   * The {@link #tokenIssued()} call is a no-op;
+   * This statistics class doesn't collect any local statistics.
+   * Instead it directly updates the S3A Instrumentation.
    */
-  @InterfaceAudience.Private
-  @InterfaceStability.Unstable
-  public final class DelegationTokenStatistics {
+  private final class DelegationTokenStatisticsImpl implements
+      DelegationTokenStatistics {
 
-    private DelegationTokenStatistics() {
+    private DelegationTokenStatisticsImpl() {
     }
 
-    /** A token has been issued. */
+    @Override
     public void tokenIssued() {
-      incrementCounter(DELEGATION_TOKENS_ISSUED, 1);
+    }
+
+    @Override
+    public DurationTracker trackDuration(final String key, final long count) {
+      return getDurationTrackerFactory()
+          .trackDuration(key, count);
     }
   }
 
-    /**
+  /**
    * Copy all the metrics to a map of (name, long-value).
    * @return a map of the metrics
    */
@@ -1296,7 +1788,7 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
   private static class MetricsToMap extends MetricsRecordBuilder {
     private final MetricsCollector parent;
     private final Map<String, Long> map =
-        new HashMap<>(COUNTERS_TO_CREATE.length * 2);
+        new HashMap<>();
 
     MetricsToMap(MetricsCollector parent) {
       this.parent = parent;
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOpContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOpContext.java
index f09aa82..aca1fa5 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.thirdparty.com.google.common.base.Preconditions;
 
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.s3a.impl.ActiveOperationContext;
+import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
 
 /**
- * Base class for operation context struct passed through codepaths for main
+ * Class for operation context struct passed through codepaths for main
  * S3AFileSystem operations.
  * Anything op-specific should be moved to a subclass of this.
+ *
+ * This was originally a base class, but {@link ActiveOperationContext} was
+ * created to be more minimal and cover many more operation type.
  */
-@SuppressWarnings("visibilitymodifier")  // I want a struct of finals, for real.
-public class S3AOpContext {
+@SuppressWarnings("visibilitymodifier")
+public class S3AOpContext extends ActiveOperationContext {
 
   final boolean isS3GuardEnabled;
   final Invoker invoker;
   @Nullable final FileSystem.Statistics stats;
-  final S3AInstrumentation instrumentation;
   @Nullable final Invoker s3guardInvoker;
 
   /** FileStatus for "destination" path being operated on. */
@@ -53,9 +57,14 @@ public class S3AOpContext {
    * @param dstFileStatus file status from existence check
    */
   public S3AOpContext(boolean isS3GuardEnabled, Invoker invoker,
-      Invoker s3guardInvoker, @Nullable FileSystem.Statistics stats,
-      S3AInstrumentation instrumentation, FileStatus dstFileStatus) {
+      @Nullable Invoker s3guardInvoker,
+      @Nullable FileSystem.Statistics stats,
+      S3AStatisticsContext instrumentation,
+      FileStatus dstFileStatus) {
 
+    super(newOperationId(),
+        instrumentation,
+        null);
     Preconditions.checkNotNull(invoker, "Null invoker arg");
     Preconditions.checkNotNull(instrumentation, "Null instrumentation arg");
     Preconditions.checkNotNull(dstFileStatus, "Null dstFileStatus arg");
@@ -65,7 +74,6 @@ public class S3AOpContext {
     this.invoker = invoker;
     this.s3guardInvoker = s3guardInvoker;
     this.stats = stats;
-    this.instrumentation = instrumentation;
     this.dstFileStatus = dstFileStatus;
   }
 
@@ -77,8 +85,10 @@ public class S3AOpContext {
    * @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 class S3AOpContext {
     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 12be3a5..3729341 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.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy;
+import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
 
 import javax.annotation.Nullable;
 
@@ -60,8 +61,8 @@ public class S3AReadOpContext extends S3AOpContext {
    * @param isS3GuardEnabled true iff S3Guard is enabled.
    * @param invoker invoker for normal retries.
    * @param s3guardInvoker S3Guard-specific retry invoker.
-   * @param stats statistics (may be null)
-   * @param instrumentation FS instrumentation
+   * @param stats Fileystem statistics (may be null)
+   * @param instrumentation statistics context
    * @param dstFileStatus target file status
    * @param inputPolicy the input policy
    * @param readahead readahead for GET operations/skip, etc.
@@ -71,13 +72,14 @@ public class S3AReadOpContext extends S3AOpContext {
       final Path path,
       boolean isS3GuardEnabled,
       Invoker invoker,
-      Invoker s3guardInvoker,
+      @Nullable Invoker s3guardInvoker,
       @Nullable FileSystem.Statistics stats,
-      S3AInstrumentation instrumentation,
+      S3AStatisticsContext instrumentation,
       FileStatus dstFileStatus,
       S3AInputPolicy inputPolicy,
       ChangeDetectionPolicy changeDetectionPolicy,
       final long readahead) {
+
     super(isS3GuardEnabled, invoker, s3guardInvoker, stats, instrumentation,
         dstFileStatus);
     this.path = checkNotNull(path);
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStorageStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStorageStatistics.java
index 4b12667..33ae091 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStorageStatistics.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStorageStatistics.java
@@ -20,106 +20,27 @@ package org.apache.hadoop.fs.s3a;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.StorageStatistics;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.impl.StorageStatisticsFromIOStatistics;
 
-import java.util.Collections;
-import java.util.EnumMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.NoSuchElementException;
-import java.util.concurrent.atomic.AtomicLong;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.emptyStatistics;
 
 /**
- * Storage statistics for S3A.
+ * Storage statistics for S3A, dynamically generated from the IOStatistics.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public class S3AStorageStatistics extends StorageStatistics
-    implements Iterable<StorageStatistics.LongStatistic> {
-  private static final Logger LOG =
-      LoggerFactory.getLogger(S3AStorageStatistics.class);
+public class S3AStorageStatistics
+    extends StorageStatisticsFromIOStatistics {
 
   public static final String NAME = "S3AStorageStatistics";
-  private final Map<Statistic, AtomicLong> opsCount =
-      new EnumMap<>(Statistic.class);
 
-  public S3AStorageStatistics() {
-    super(NAME);
-    for (Statistic opType : Statistic.values()) {
-      opsCount.put(opType, new AtomicLong(0));
-    }
-  }
-
-  /**
-   * Increment a specific counter.
-   * @param op operation
-   * @param count increment value
-   * @return the new value
-   */
-  public long incrementCounter(Statistic op, long count) {
-    long updated = opsCount.get(op).addAndGet(count);
-    LOG.debug("{} += {}  ->  {}", op, count, updated);
-    return updated;
-  }
-
-  private class LongIterator implements Iterator<LongStatistic> {
-    private Iterator<Map.Entry<Statistic, AtomicLong>> iterator =
-        Collections.unmodifiableSet(opsCount.entrySet()).iterator();
-
-    @Override
-    public boolean hasNext() {
-      return iterator.hasNext();
-    }
-
-    @Override
-    public LongStatistic next() {
-      if (!iterator.hasNext()) {
-        throw new NoSuchElementException();
-      }
-      final Map.Entry<Statistic, AtomicLong> entry = iterator.next();
-      return new LongStatistic(entry.getKey().getSymbol(),
-          entry.getValue().get());
-    }
-
-    @Override
-    public void remove() {
-      throw new UnsupportedOperationException();
-    }
-  }
-
-  @Override
-  public String getScheme() {
-    return "s3a";
+  public S3AStorageStatistics(final IOStatistics ioStatistics) {
+    super(NAME, "s3a", ioStatistics);
   }
 
-  @Override
-  public Iterator<LongStatistic> getLongStatistics() {
-    return new LongIterator();
-  }
-
-  @Override
-  public Iterator<LongStatistic> iterator() {
-    return getLongStatistics();
-  }
-
-  @Override
-  public Long getLong(String key) {
-    final Statistic type = Statistic.fromSymbol(key);
-    return type == null ? null : opsCount.get(type).get();
-  }
-
-  @Override
-  public boolean isTracked(String key) {
-    return Statistic.fromSymbol(key) != null;
-  }
-
-  @Override
-  public void reset() {
-    for (AtomicLong value : opsCount.values()) {
-      value.set(0);
-    }
+  public S3AStorageStatistics() {
+    super(NAME, "s3a", emptyStatistics());
   }
 
 }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
index 65c0c23..314f13f 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.util.functional.RemoteIterators;
 import org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider;
 import org.apache.hadoop.fs.s3a.impl.NetworkBinding;
 import org.apache.hadoop.fs.s3native.S3xLoginHelper;
@@ -1470,12 +1471,7 @@ public final class S3AUtils {
   public static long applyLocatedFiles(
       RemoteIterator<? extends LocatedFileStatus> iterator,
       CallOnLocatedFileStatus eval) throws IOException {
-    long count = 0;
-    while (iterator.hasNext()) {
-      count++;
-      eval.call(iterator.next());
-    }
-    return count;
+    return RemoteIterators.foreach(iterator, eval::call);
   }
 
   /**
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java
index e0a1d78..e04d3b5 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java
@@ -26,6 +26,7 @@ import com.amazonaws.services.s3.AmazonS3;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.s3a.statistics.StatisticsFromAwsSdk;
 
 /**
  * Factory for creation of {@link AmazonS3} client instances.
@@ -41,12 +42,14 @@ public interface S3ClientFactory {
    * @param bucket Optional bucket to use to look up per-bucket proxy secrets
    * @param credentialSet credentials to use
    * @param userAgentSuffix optional suffix for the UA field.
+   * @param statisticsFromAwsSdk binding for AWS stats - may be null
    * @return S3 client
    * @throws IOException IO problem
    */
   AmazonS3 createS3Client(URI name,
       String bucket,
       AWSCredentialsProvider credentialSet,
-      String userAgentSuffix) throws IOException;
+      String userAgentSuffix,
+      StatisticsFromAwsSdk statisticsFromAwsSdk) throws IOException;
 
 }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
index 1addfbe..6709382 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
@@ -18,248 +18,504 @@
 
 package org.apache.hadoop.fs.s3a;
 
-import org.apache.hadoop.fs.StorageStatistics.CommonStatisticNames;
-
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.s3a.statistics.StatisticTypeEnum;
+import org.apache.hadoop.fs.statistics.StoreStatisticNames;
+import org.apache.hadoop.fs.statistics.StreamStatisticNames;
+
+import static org.apache.hadoop.fs.s3a.statistics.StatisticTypeEnum.TYPE_COUNTER;
+import static org.apache.hadoop.fs.s3a.statistics.StatisticTypeEnum.TYPE_DURATION;
+import static org.apache.hadoop.fs.s3a.statistics.StatisticTypeEnum.TYPE_GAUGE;
+import static org.apache.hadoop.fs.s3a.statistics.StatisticTypeEnum.TYPE_QUANTILE;
+
 /**
  * Statistic which are collected in S3A.
- * These statistics are available at a low level in {@link S3AStorageStatistics}
- * and as metrics in {@link S3AInstrumentation}
+ * Counter and duration statistics are published in
+ * {@link S3AFileSystem#getStorageStatistics()}.
+ * and as metrics in {@link S3AInstrumentation}.
+ * <p>
+ * Where possible, stream names come from {@link StreamStatisticNames}
+ * and {@link StoreStatisticNames}
+ * </p>
  */
+@InterfaceStability.Unstable
 public enum Statistic {
 
+  /* Low-level duration counters */
+  ACTION_EXECUTOR_ACQUIRED(
+      StoreStatisticNames.ACTION_EXECUTOR_ACQUIRED,
+      "Executor acquired.",
+      TYPE_DURATION),
+  ACTION_HTTP_HEAD_REQUEST(
+      StoreStatisticNames.ACTION_HTTP_HEAD_REQUEST,
+      "HEAD request.",
+      TYPE_DURATION),
+  ACTION_HTTP_GET_REQUEST(
+      StoreStatisticNames.ACTION_HTTP_GET_REQUEST,
+      "GET request.",
+      TYPE_DURATION),
+
+  /* FileSystem Level statistics */
   DIRECTORIES_CREATED("directories_created",
-      "Total number of directories created through the object store."),
+      "Total number of directories created through the object store.",
+      TYPE_COUNTER),
   DIRECTORIES_DELETED("directories_deleted",
-      "Total number of directories deleted through the object store."),
+      "Total number of directories deleted through the object store.",
+      TYPE_COUNTER),
   FILES_COPIED("files_copied",
-      "Total number of files copied within the object store."),
+      "Total number of files copied within the object store.",
+      TYPE_COUNTER),
   FILES_COPIED_BYTES("files_copied_bytes",
-      "Total number of bytes copied within the object store."),
+      "Total number of bytes copied within the object store.",
+      TYPE_COUNTER),
   FILES_CREATED("files_created",
-      "Total number of files created through the object store."),
+      "Total number of files created through the object store.",
+      TYPE_COUNTER),
   FILES_DELETED("files_deleted",
-      "Total number of files deleted from the object store."),
+      "Total number of files deleted from the object store.",
+      TYPE_COUNTER),
   FILES_DELETE_REJECTED("files_delete_rejected",
-      "Total number of files whose delete request was rejected"),
+      "Total number of files whose delete request was rejected",
+      TYPE_COUNTER),
   FAKE_DIRECTORIES_CREATED("fake_directories_created",
-      "Total number of fake directory entries created in the object store."),
+      "Total number of fake directory entries created in the object store.",
+      TYPE_COUNTER),
   FAKE_DIRECTORIES_DELETED("fake_directories_deleted",
-      "Total number of fake directory deletes submitted to object store."),
-  IGNORED_ERRORS("ignored_errors", "Errors caught and ignored"),
-  INVOCATION_COPY_FROM_LOCAL_FILE(CommonStatisticNames.OP_COPY_FROM_LOCAL_FILE,
-      "Calls of copyFromLocalFile()"),
-  INVOCATION_CREATE(CommonStatisticNames.OP_CREATE,
-      "Calls of create()"),
-  INVOCATION_CREATE_NON_RECURSIVE(CommonStatisticNames.OP_CREATE_NON_RECURSIVE,
-      "Calls of createNonRecursive()"),
-  INVOCATION_DELETE(CommonStatisticNames.OP_DELETE,
-      "Calls of delete()"),
-  INVOCATION_EXISTS(CommonStatisticNames.OP_EXISTS,
-      "Calls of exists()"),
-  INVOCATION_GET_DELEGATION_TOKEN(CommonStatisticNames.OP_GET_DELEGATION_TOKEN,
-      "Calls of getDelegationToken()"),
-  INVOCATION_GET_FILE_CHECKSUM(CommonStatisticNames.OP_GET_FILE_CHECKSUM,
-      "Calls of getFileChecksum()"),
-  INVOCATION_GET_FILE_STATUS(CommonStatisticNames.OP_GET_FILE_STATUS,
-      "Calls of getFileStatus()"),
-  INVOCATION_GLOB_STATUS(CommonStatisticNames.OP_GLOB_STATUS,
-      "Calls of globStatus()"),
-  INVOCATION_IS_DIRECTORY(CommonStatisticNames.OP_IS_DIRECTORY,
-      "Calls of isDirectory()"),
-  INVOCATION_IS_FILE(CommonStatisticNames.OP_IS_FILE,
-      "Calls of isFile()"),
-  INVOCATION_LIST_FILES(CommonStatisticNames.OP_LIST_FILES,
-      "Calls of listFiles()"),
-  INVOCATION_LIST_LOCATED_STATUS(CommonStatisticNames.OP_LIST_LOCATED_STATUS,
-      "Calls of listLocatedStatus()"),
-  INVOCATION_LIST_STATUS(CommonStatisticNames.OP_LIST_STATUS,
-      "Calls of listStatus()"),
-  INVOCATION_MKDIRS(CommonStatisticNames.OP_MKDIRS,
-      "Calls of mkdirs()"),
-  INVOCATION_OPEN(CommonStatisticNames.OP_OPEN,
-      "Calls of open()"),
-  INVOCATION_RENAME(CommonStatisticNames.OP_RENAME,
-      "Calls of rename()"),
-  OBJECT_COPY_REQUESTS("object_copy_requests", "Object copy requests"),
-  OBJECT_DELETE_REQUESTS("object_delete_requests", "Object delete requests"),
-  OBJECT_DELETE_OBJECTS("object_delete_objects",
-      "Objects deleted in delete requests"),
-  OBJECT_LIST_REQUESTS("object_list_requests",
-      "Number of object listings made"),
-  OBJECT_CONTINUE_LIST_REQUESTS("object_continue_list_requests",
-      "Number of continued object listings made"),
-  OBJECT_METADATA_REQUESTS("object_metadata_requests",
-      "Number of requests for object metadata"),
-  OBJECT_MULTIPART_UPLOAD_INITIATED("object_multipart_initiated",
-      "Object multipart upload initiated"),
-  OBJECT_MULTIPART_UPLOAD_ABORTED("object_multipart_aborted",
-      "Object multipart upload aborted"),
-  OBJECT_PUT_REQUESTS("object_put_requests",
-      "Object put/multipart upload count"),
-  OBJECT_PUT_REQUESTS_COMPLETED("object_put_requests_completed",
-      "Object put/multipart upload completed count"),
-  OBJECT_PUT_REQUESTS_ACTIVE("object_put_requests_active",
-      "Current number of active put requests"),
-  OBJECT_PUT_BYTES("object_put_bytes", "number of bytes uploaded"),
-  OBJECT_PUT_BYTES_PENDING("object_put_bytes_pending",
-      "number of bytes queued for upload/being actively uploaded"),
-  OBJECT_SELECT_REQUESTS("object_select_requests",
-      "Count of S3 Select requests issued"),
-  STREAM_ABORTED("stream_aborted",
-      "Count of times the TCP stream was aborted"),
-  STREAM_BACKWARD_SEEK_OPERATIONS("stream_backward_seek_operations",
-      "Number of executed seek operations which went backwards in a stream"),
-  STREAM_CLOSED("stream_closed", "Count of times the TCP stream was closed"),
-  STREAM_CLOSE_OPERATIONS("stream_close_operations",
-      "Total count of times an attempt to close a data stream was made"),
-  STREAM_FORWARD_SEEK_OPERATIONS("stream_forward_seek_operations",
-      "Number of executed seek operations which went forward in a stream"),
-  STREAM_OPENED("stream_opened",
-      "Total count of times an input stream to object store was opened"),
-  STREAM_READ_EXCEPTIONS("stream_read_exceptions",
-      "Number of exceptions invoked on input streams"),
-  STREAM_READ_FULLY_OPERATIONS("stream_read_fully_operations",
-      "Count of readFully() operations in streams"),
-  STREAM_READ_OPERATIONS("stream_read_operations",
-      "Count of read() operations in streams"),
-  STREAM_READ_OPERATIONS_INCOMPLETE("stream_read_operations_incomplete",
-      "Count of incomplete read() operations in streams"),
-  STREAM_READ_VERSION_MISMATCHES("stream_read_version_mismatches",
-      "Count of version mismatches encountered while reading streams"),
-  STREAM_SEEK_BYTES_BACKWARDS("stream_bytes_backwards_on_seek",
-      "Count of bytes moved backwards during seek operations"),
-  STREAM_SEEK_BYTES_READ("stream_bytes_read",
-      "Count of bytes read during seek() in stream operations"),
-  STREAM_SEEK_BYTES_SKIPPED("stream_bytes_skipped_on_seek",
-      "Count of bytes skipped during forward seek operation"),
-  STREAM_SEEK_OPERATIONS("stream_seek_operations",
-      "Number of seek operations during stream IO."),
-  STREAM_CLOSE_BYTES_READ("stream_bytes_read_in_close",
-      "Count of bytes read when closing streams during seek operations."),
-  STREAM_ABORT_BYTES_DISCARDED("stream_bytes_discarded_in_abort",
-      "Count of bytes discarded by aborting the stream"),
-  STREAM_WRITE_FAILURES("stream_write_failures",
-      "Count of stream write failures reported"),
-  STREAM_WRITE_BLOCK_UPLOADS("stream_write_block_uploads",
-      "Count of block/partition uploads completed"),
-  STREAM_WRITE_BLOCK_UPLOADS_ACTIVE("stream_write_block_uploads_active",
-      "Count of block/partition uploads completed"),
-  STREAM_WRITE_BLOCK_UPLOADS_COMMITTED("stream_write_block_uploads_committed",
-      "Count of number of block uploads committed"),
-  STREAM_WRITE_BLOCK_UPLOADS_ABORTED("stream_write_block_uploads_aborted",
-      "Count of number of block uploads aborted"),
-
-  STREAM_WRITE_BLOCK_UPLOADS_PENDING("stream_write_block_uploads_pending",
-      "Gauge of block/partitions uploads queued to be written"),
-  STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING(
-      "stream_write_block_uploads_data_pending",
-      "Gauge of block/partitions data uploads queued to be written"),
-  STREAM_WRITE_TOTAL_TIME("stream_write_total_time",
-      "Count of total time taken for uploads to complete"),
-  STREAM_WRITE_TOTAL_DATA("stream_write_total_data",
-      "Count of total data uploaded in block output"),
-  STREAM_WRITE_QUEUE_DURATION("stream_write_queue_duration",
-      "Total queue duration of all block uploads"),
-
-  // S3guard committer stats
+      "Total number of fake directory deletes submitted to object store.",
+      TYPE_COUNTER),
+  IGNORED_ERRORS("ignored_errors", "Errors caught and ignored",
+      TYPE_COUNTER),
+  INVOCATION_COPY_FROM_LOCAL_FILE(
+      StoreStatisticNames.OP_COPY_FROM_LOCAL_FILE,
+      "Calls of copyFromLocalFile()",
+      TYPE_COUNTER),
+  INVOCATION_CREATE(
+      StoreStatisticNames.OP_CREATE,
+      "Calls of create()",
+      TYPE_COUNTER),
+  INVOCATION_CREATE_NON_RECURSIVE(
+      StoreStatisticNames.OP_CREATE_NON_RECURSIVE,
+      "Calls of createNonRecursive()",
+      TYPE_COUNTER),
+  INVOCATION_DELETE(
+      StoreStatisticNames.OP_DELETE,
+      "Calls of delete()",
+      TYPE_COUNTER),
+  INVOCATION_EXISTS(
+      StoreStatisticNames.OP_EXISTS,
+      "Calls of exists()",
+      TYPE_COUNTER),
+  INVOCATION_GET_DELEGATION_TOKEN(
+      StoreStatisticNames.OP_GET_DELEGATION_TOKEN,
+      "Calls of getDelegationToken()",
+      TYPE_COUNTER),
+  INVOCATION_GET_FILE_CHECKSUM(
+      StoreStatisticNames.OP_GET_FILE_CHECKSUM,
+      "Calls of getFileChecksum()",
+      TYPE_COUNTER),
+  INVOCATION_GET_FILE_STATUS(
+      StoreStatisticNames.OP_GET_FILE_STATUS,
+      "Calls of getFileStatus()",
+      TYPE_COUNTER),
+  INVOCATION_GLOB_STATUS(
+      StoreStatisticNames.OP_GLOB_STATUS,
+      "Calls of globStatus()",
+      TYPE_COUNTER),
+  INVOCATION_IS_DIRECTORY(
+      StoreStatisticNames.OP_IS_DIRECTORY,
+      "Calls of isDirectory()",
+      TYPE_COUNTER),
+  INVOCATION_IS_FILE(
+      StoreStatisticNames.OP_IS_FILE,
+      "Calls of isFile()",
+      TYPE_COUNTER),
+  INVOCATION_LIST_FILES(
+      StoreStatisticNames.OP_LIST_FILES,
+      "Calls of listFiles()",
+      TYPE_COUNTER),
+  INVOCATION_LIST_LOCATED_STATUS(
+      StoreStatisticNames.OP_LIST_LOCATED_STATUS,
+      "Calls of listLocatedStatus()",
+      TYPE_COUNTER),
+  INVOCATION_LIST_STATUS(
+      StoreStatisticNames.OP_LIST_STATUS,
+      "Calls of listStatus()",
+      TYPE_COUNTER),
+  INVOCATION_MKDIRS(
+      StoreStatisticNames.OP_MKDIRS,
+      "Calls of mkdirs()",
+      TYPE_COUNTER),
+  INVOCATION_OPEN(
+      StoreStatisticNames.OP_OPEN,
+      "Calls of open()",
+      TYPE_COUNTER),
+  INVOCATION_RENAME(
+      StoreStatisticNames.OP_RENAME,
+      "Calls of rename()",
+      TYPE_COUNTER),
+
+  /* Object IO */
+  OBJECT_COPY_REQUESTS(StoreStatisticNames.OBJECT_COPY_REQUESTS,
+      "Object copy requests",
+      TYPE_COUNTER),
+  OBJECT_DELETE_REQUEST(StoreStatisticNames.OBJECT_DELETE_REQUEST,
+      "Object delete requests",
+      TYPE_DURATION),
+  OBJECT_BULK_DELETE_REQUEST(StoreStatisticNames.OBJECT_BULK_DELETE_REQUEST,
+      "Object bulk delete requests",
+      TYPE_DURATION),
+  OBJECT_DELETE_OBJECTS(StoreStatisticNames.OBJECT_DELETE_OBJECTS,
+      "Objects deleted in delete requests",
+      TYPE_COUNTER),
+  OBJECT_LIST_REQUEST(StoreStatisticNames.OBJECT_LIST_REQUEST,
+      "Count of object listings made",
+      TYPE_DURATION),
+  OBJECT_CONTINUE_LIST_REQUESTS(
+      StoreStatisticNames.OBJECT_CONTINUE_LIST_REQUEST,
+      "Count of continued object listings made",
+      TYPE_DURATION),
+  OBJECT_METADATA_REQUESTS(
+      StoreStatisticNames.OBJECT_METADATA_REQUESTS,
+      "Count of requests for object metadata",
+      TYPE_COUNTER),
+  OBJECT_MULTIPART_UPLOAD_INITIATED(
+      StoreStatisticNames.OBJECT_MULTIPART_UPLOAD_INITIATED,
+      "Object multipart upload initiated",
+      TYPE_COUNTER),
+  OBJECT_MULTIPART_UPLOAD_ABORTED(
+      StoreStatisticNames.OBJECT_MULTIPART_UPLOAD_ABORTED,
+      "Object multipart upload aborted",
+      TYPE_COUNTER),
+  OBJECT_PUT_REQUESTS(
+      StoreStatisticNames.OBJECT_PUT_REQUEST,
+      "Object put/multipart upload count",
+      TYPE_COUNTER),
+  OBJECT_PUT_REQUESTS_COMPLETED(
+      StoreStatisticNames.OBJECT_PUT_REQUEST_COMPLETED,
+      "Object put/multipart upload completed count",
+      TYPE_COUNTER),
+  OBJECT_PUT_REQUESTS_ACTIVE(
+      StoreStatisticNames.OBJECT_PUT_REQUEST_ACTIVE,
+      "Current number of active put requests",
+      TYPE_GAUGE),
+  OBJECT_PUT_BYTES(
+      StoreStatisticNames.OBJECT_PUT_BYTES,
+      "number of bytes uploaded",
+      TYPE_COUNTER),
+  OBJECT_PUT_BYTES_PENDING(
+      StoreStatisticNames.OBJECT_PUT_BYTES_PENDING,
+      "number of bytes queued for upload/being actively uploaded",
+      TYPE_GAUGE),
+  OBJECT_SELECT_REQUESTS(
+      StoreStatisticNames.OBJECT_SELECT_REQUESTS,
+      "Count of S3 Select requests issued",
+      TYPE_COUNTER),
+  STREAM_READ_ABORTED(
+      StreamStatisticNames.STREAM_READ_ABORTED,
+      "Count of times the TCP stream was aborted",
+      TYPE_COUNTER),
+
+  /* Stream Reads */
+  STREAM_READ_BYTES(
+      StreamStatisticNames.STREAM_READ_BYTES,
+      "Bytes read from an input stream in read() calls",
+      TYPE_COUNTER),
+  STREAM_READ_BYTES_DISCARDED_ABORT(
+      StreamStatisticNames.STREAM_READ_BYTES_DISCARDED_ABORT,
+      "Count of bytes discarded by aborting an input stream",
+      TYPE_COUNTER),
+  STREAM_READ_BYTES_READ_CLOSE(
+      StreamStatisticNames.STREAM_READ_BYTES_DISCARDED_CLOSE,
+      "Count of bytes read and discarded when closing an input stream",
+      TYPE_COUNTER),
+  STREAM_READ_CLOSED(
+      StreamStatisticNames.STREAM_READ_CLOSED,
+      "Count of times the TCP stream was closed",
+      TYPE_COUNTER),
+  STREAM_READ_CLOSE_OPERATIONS(
+      StreamStatisticNames.STREAM_READ_CLOSE_OPERATIONS,
+      "Total count of times an attempt to close an input stream was made",
+      TYPE_COUNTER),
+  STREAM_READ_EXCEPTIONS(
+      StreamStatisticNames.STREAM_READ_EXCEPTIONS,
+      "Count of exceptions raised during input stream reads",
+      TYPE_COUNTER),
+  STREAM_READ_FULLY_OPERATIONS(
+      StreamStatisticNames.STREAM_READ_FULLY_OPERATIONS,
+      "Count of readFully() operations in an input stream",
+      TYPE_COUNTER),
+  STREAM_READ_OPENED(
+      StreamStatisticNames.STREAM_READ_OPENED,
+      "Total count of times an input stream to object store data was opened",
+      TYPE_COUNTER),
+  STREAM_READ_OPERATIONS(
+      StreamStatisticNames.STREAM_READ_OPERATIONS,
+      "Count of read() operations in an input stream",
+      TYPE_COUNTER),
+  STREAM_READ_OPERATIONS_INCOMPLETE(
+      StreamStatisticNames.STREAM_READ_OPERATIONS_INCOMPLETE,
+      "Count of incomplete read() operations in an input stream",
+      TYPE_COUNTER),
+  STREAM_READ_VERSION_MISMATCHES(
+      StreamStatisticNames.STREAM_READ_VERSION_MISMATCHES,
+      "Count of version mismatches encountered while reading an input stream",
+      TYPE_COUNTER),
+  STREAM_READ_SEEK_BACKWARD_OPERATIONS(
+      StreamStatisticNames.STREAM_READ_SEEK_BACKWARD_OPERATIONS,
+      "Count of executed seek operations which went backwards in a stream",
+      TYPE_COUNTER),
+  STREAM_READ_SEEK_BYTES_BACKWARDS(
+      StreamStatisticNames.STREAM_READ_SEEK_BYTES_BACKWARDS,
+      "Count of bytes moved backwards during seek operations"
+          + " in an input stream",
+      TYPE_COUNTER),
+  STREAM_READ_SEEK_BYTES_DISCARDED(
+      StreamStatisticNames.STREAM_READ_SEEK_BYTES_DISCARDED,
+      "Count of bytes read and discarded during seek() in an input stream",
+      TYPE_COUNTER),
+  STREAM_READ_SEEK_BYTES_SKIPPED(
+      StreamStatisticNames.STREAM_READ_SEEK_BYTES_SKIPPED,
+      "Count of bytes skipped during forward seek operations"
+          + " an input stream",
+      TYPE_COUNTER),
+  STREAM_READ_SEEK_FORWARD_OPERATIONS(
+      StreamStatisticNames.STREAM_READ_SEEK_FORWARD_OPERATIONS,
+      "Count of executed seek operations which went forward in"
+          + " an input stream",
+      TYPE_COUNTER),
+  STREAM_READ_SEEK_OPERATIONS(
+      StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS,
+      "Count of seek operations in an input stream",
+      TYPE_COUNTER),
+  STREAM_READ_SEEK_POLICY_CHANGED(
+      StreamStatisticNames.STREAM_READ_SEEK_POLICY_CHANGED,
+      "Count of times the seek policy was dynamically changed"
+          + " in an input stream",
+      TYPE_COUNTER),
+  STREAM_READ_TOTAL_BYTES(
+      StreamStatisticNames.STREAM_READ_TOTAL_BYTES,
+      "Total count of bytes read from an input stream",
+      TYPE_COUNTER),
+
+  /* Stream Write statistics */
+
+  STREAM_WRITE_EXCEPTIONS(
+      StreamStatisticNames.STREAM_WRITE_EXCEPTIONS,
+      "Count of stream write failures reported",
+      TYPE_COUNTER),
+  STREAM_WRITE_EXCEPTIONS_COMPLETING_UPLOADS(
+      StreamStatisticNames.STREAM_WRITE_EXCEPTIONS_COMPLETING_UPLOADS,
+      "Count of failures when finalizing a multipart upload",
+      TYPE_COUNTER),
+  STREAM_WRITE_BLOCK_UPLOADS(
+      StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS,
+      "Count of block/partition uploads completed",
+      TYPE_COUNTER),
+  STREAM_WRITE_BLOCK_UPLOADS_ACTIVE(
+      StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS_ACTIVE,
+      "Count of block/partition uploads active",
+      TYPE_GAUGE),
+  STREAM_WRITE_BLOCK_UPLOADS_COMMITTED(
+      StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS_COMMITTED,
+      "Count of number of block uploads committed",
+      TYPE_COUNTER),
+  STREAM_WRITE_BLOCK_UPLOADS_ABORTED(
+      StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS_ABORTED,
+      "Count of number of block uploads aborted",
+      TYPE_COUNTER),
+
+  STREAM_WRITE_BLOCK_UPLOADS_PENDING(
+      StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS_PENDING,
+      "Gauge of block/partitions uploads queued to be written",
+      TYPE_GAUGE),
+  STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING(
+      StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING,
+      "Gauge of data queued to be written",
+      TYPE_GAUGE),
+  STREAM_WRITE_TOTAL_TIME(
+      StreamStatisticNames.STREAM_WRITE_TOTAL_TIME,
+      "Count of total time taken for uploads to complete",
+      TYPE_COUNTER),
+  STREAM_WRITE_TOTAL_DATA(StreamStatisticNames.STREAM_WRITE_TOTAL_DATA,
+      "Count of total data uploaded",
+      TYPE_COUNTER),
+  STREAM_WRITE_BYTES(
+      StreamStatisticNames.STREAM_WRITE_BYTES,
+      "Count of bytes written to output stream"
+          + " (including all not yet uploaded)",
+      TYPE_COUNTER),
+  STREAM_WRITE_QUEUE_DURATION(
+      StreamStatisticNames.STREAM_WRITE_QUEUE_DURATION,
+      "Total queue duration of all block uploads",
+      TYPE_DURATION),
+
+  /* committer stats */
   COMMITTER_COMMITS_CREATED(
       "committer_commits_created",
-      "Number of files to commit created"),
+      "Count of files to commit created",
+      TYPE_COUNTER),
   COMMITTER_COMMITS_COMPLETED(
       "committer_commits_completed",
-      "Number of files committed"),
+      "Count of files committed",
+      TYPE_COUNTER),
+  COMMITTER_COMMIT_JOB(
+      "committer_commit_job",
+      "Duration Tracking of time to commit an entire job",
+      TYPE_DURATION),
   COMMITTER_JOBS_SUCCEEDED(
       "committer_jobs_completed",
-      "Number of successful jobs"),
+      "Count of successful jobs",
+      TYPE_COUNTER),
   COMMITTER_JOBS_FAILED(
       "committer_jobs_failed",
-      "Number of failed jobs"),
+      "Count of failed jobs",
+      TYPE_COUNTER),
   COMMITTER_TASKS_SUCCEEDED(
       "committer_tasks_completed",
-      "Number of successful tasks"),
+      "Count of successful tasks",
+      TYPE_COUNTER),
   COMMITTER_TASKS_FAILED(
       "committer_tasks_failed",
-      "Number of failed tasks"),
+      "Count of failed tasks",
+      TYPE_COUNTER),
   COMMITTER_BYTES_COMMITTED(
       "committer_bytes_committed",
-      "Amount of data committed"),
+      "Amount of data committed",
+      TYPE_COUNTER),
   COMMITTER_BYTES_UPLOADED(
       "committer_bytes_uploaded",
-      "Number of bytes uploaded duing commit operations"),
+      "Count of bytes uploaded duing commit operations",
+      TYPE_COUNTER),
   COMMITTER_COMMITS_FAILED(
-      "committer_commits_failed",
-      "Number of commits failed"),
+      "committer_commits"+ StoreStatisticNames.SUFFIX_FAILURES,
+      "Count of commits failed",
+      TYPE_COUNTER),
   COMMITTER_COMMITS_ABORTED(
       "committer_commits_aborted",
-      "Number of commits aborted"),
+      "Count of commits aborted",
+      TYPE_COUNTER),
   COMMITTER_COMMITS_REVERTED(
       "committer_commits_reverted",
-      "Number of commits reverted"),
+      "Count of commits reverted",
+      TYPE_COUNTER),
   COMMITTER_MAGIC_FILES_CREATED(
       "committer_magic_files_created",
-      "Number of files created under 'magic' paths"),
+      "Count of files created under 'magic' paths",
+      TYPE_COUNTER),
+  COMMITTER_MATERIALIZE_FILE(
+      "committer_materialize_file",
+      "Duration Tracking of time to materialize a file in job commit",
+      TYPE_DURATION),
+  COMMITTER_STAGE_FILE_UPLOAD(
+      "committer_stage_file_upload",
+      "Duration Tracking of files uploaded from a local staging path",
+      TYPE_DURATION),
 
-  // S3guard stats
+  /* S3guard stats */
   S3GUARD_METADATASTORE_PUT_PATH_REQUEST(
       "s3guard_metadatastore_put_path_request",
-      "S3Guard metadata store put one metadata path request"),
+      "S3Guard metadata store put one metadata path request",
+      TYPE_COUNTER),
   S3GUARD_METADATASTORE_PUT_PATH_LATENCY(
       "s3guard_metadatastore_put_path_latency",
-      "S3Guard metadata store put one metadata path latency"),
-  S3GUARD_METADATASTORE_INITIALIZATION("s3guard_metadatastore_initialization",
-      "S3Guard metadata store initialization times"),
+      "S3Guard metadata store put one metadata path latency",
+      TYPE_QUANTILE),
+  S3GUARD_METADATASTORE_INITIALIZATION(
+      "s3guard_metadatastore_initialization",
+      "S3Guard metadata store initialization times",
+      TYPE_COUNTER),
   S3GUARD_METADATASTORE_RECORD_DELETES(
       "s3guard_metadatastore_record_deletes",
-      "S3Guard metadata store records deleted"),
+      "S3Guard metadata store records deleted",
+      TYPE_COUNTER),
   S3GUARD_METADATASTORE_RECORD_READS(
       "s3guard_metadatastore_record_reads",
-      "S3Guard metadata store records read"),
+      "S3Guard metadata store records read",
+      TYPE_COUNTER),
   S3GUARD_METADATASTORE_RECORD_WRITES(
       "s3guard_metadatastore_record_writes",
-      "S3Guard metadata store records written"),
+      "S3Guard metadata store records written",
+      TYPE_COUNTER),
   S3GUARD_METADATASTORE_RETRY("s3guard_metadatastore_retry",
-      "S3Guard metadata store retry events"),
+      "S3Guard metadata store retry events",
+      TYPE_COUNTER),
   S3GUARD_METADATASTORE_THROTTLED("s3guard_metadatastore_throttled",
-      "S3Guard metadata store throttled events"),
+      "S3Guard metadata store throttled events",
+      TYPE_COUNTER),
   S3GUARD_METADATASTORE_THROTTLE_RATE(
       "s3guard_metadatastore_throttle_rate",
-      "S3Guard metadata store throttle rate"),
+      "S3Guard metadata store throttle rate",
+      TYPE_QUANTILE),
   S3GUARD_METADATASTORE_AUTHORITATIVE_DIRECTORIES_UPDATED(
       "s3guard_metadatastore_authoritative_directories_updated",
-      "S3Guard metadata store authoritative directories updated from S3"),
-
-  STORE_IO_THROTTLED("store_io_throttled", "Requests throttled and retried"),
-  STORE_IO_THROTTLE_RATE("store_io_throttle_rate",
-      "Rate of S3 request throttling"),
-
-  DELEGATION_TOKENS_ISSUED("delegation_tokens_issued",
-      "Number of delegation tokens issued"),
-
-  MULTIPART_INSTANTIATED(
-      "multipart_instantiated",
-      "Multipart Uploader Instantiated"),
-  MULTIPART_PART_PUT(
-      "multipart_part_put",
-      "Multipart Part Put Operation"),
-  MULTIPART_PART_PUT_BYTES(
-      "multipart_part_put_bytes",
-      "Multipart Part Put Bytes"),
+      "S3Guard metadata store authoritative directories updated from S3",
+      TYPE_COUNTER),
+
+
+  /* General Store operations */
+  STORE_IO_REQUEST(StoreStatisticNames.STORE_IO_REQUEST,
+      "requests made of the remote store",
+      TYPE_COUNTER),
+
+  STORE_IO_RETRY(StoreStatisticNames.STORE_IO_RETRY,
+      "retried requests made of the remote store",
+      TYPE_COUNTER),
+
+  STORE_IO_THROTTLED(
+      StoreStatisticNames.STORE_IO_THROTTLED,
+      "Requests throttled and retried",
+      TYPE_COUNTER),
+  STORE_IO_THROTTLE_RATE(
+      StoreStatisticNames.STORE_IO_THROTTLE_RATE,
+      "Rate of S3 request throttling",
+      TYPE_QUANTILE),
+
+  /*
+   * Delegation Token Operations.
+   */
+  DELEGATION_TOKEN_ISSUED(
+      StoreStatisticNames.DELEGATION_TOKEN_ISSUED,
+      "Count of delegation tokens issued",
+      TYPE_DURATION),
+
+  /* Multipart Upload API */
+
+  MULTIPART_UPLOAD_INSTANTIATED(
+      StoreStatisticNames.MULTIPART_UPLOAD_INSTANTIATED,
+      "Multipart Uploader Instantiated",
+      TYPE_COUNTER),
+  MULTIPART_UPLOAD_PART_PUT(
+      StoreStatisticNames.MULTIPART_UPLOAD_PART_PUT,
+      "Multipart Part Put Operation",
+      TYPE_COUNTER),
+  MULTIPART_UPLOAD_PART_PUT_BYTES(
+      StoreStatisticNames.MULTIPART_UPLOAD_PART_PUT_BYTES,
+      "Multipart Part Put Bytes",
+      TYPE_COUNTER),
   MULTIPART_UPLOAD_ABORTED(
-      "multipart_upload_aborted",
-      "Multipart Upload Aborted"),
+      StoreStatisticNames.MULTIPART_UPLOAD_ABORTED,
+      "Multipart Upload Aborted",
+      TYPE_COUNTER),
   MULTIPART_UPLOAD_ABORT_UNDER_PATH_INVOKED(
-      "multipart_upload_abort_under_path_invoked",
-      "Multipart Upload Abort Udner Path Invoked"),
+      StoreStatisticNames.MULTIPART_UPLOAD_ABORT_UNDER_PATH_INVOKED,
+      "Multipart Upload Abort Unner Path Invoked",
+      TYPE_COUNTER),
   MULTIPART_UPLOAD_COMPLETED(
-      "multipart_upload_completed",
-      "Multipart Upload Completed"),
+      StoreStatisticNames.MULTIPART_UPLOAD_COMPLETED,
+      "Multipart Upload Completed",
+      TYPE_COUNTER),
   MULTIPART_UPLOAD_STARTED(
-      "multipart_upload_started",
-      "Multipart Upload Started");
+      StoreStatisticNames.MULTIPART_UPLOAD_STARTED,
+      "Multipart Upload Started",
+      TYPE_COUNTER);
+
 
+  /**
+   * A map used to support the {@link #fromSymbol(String)} call.
+   */
   private static final Map<String, Statistic> SYMBOL_MAP =
       new HashMap<>(Statistic.values().length);
   static {
@@ -268,14 +524,28 @@ public enum Statistic {
     }
   }
 
-  Statistic(String symbol, String description) {
+
+  /**
+   * Statistic definition.
+   * @param symbol name
+   * @param description description.
+   * @param type type
+   */
+  Statistic(String symbol, String description, StatisticTypeEnum type) {
     this.symbol = symbol;
     this.description = description;
+    this.type = type;
   }
 
+  /** Statistic name. */
   private final String symbol;
+
+  /** Statistic description. */
   private final String description;
 
+  /** Statistic type. */
+  private final StatisticTypeEnum type;
+
   public String getSymbol() {
     return symbol;
   }
@@ -302,4 +572,12 @@ public enum Statistic {
   public String toString() {
     return symbol;
   }
+
+  /**
+   * What type is this statistic?
+   * @return the type.
+   */
+  public StatisticTypeEnum getType() {
+    return type;
+  }
 }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java
index 6b7c569..e75c09c 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java
@@ -50,10 +50,12 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathIOException;
+import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
 import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
 import org.apache.hadoop.fs.s3a.s3guard.S3Guard;
 import org.apache.hadoop.fs.s3a.select.SelectBinding;
 import org.apache.hadoop.util.DurationInfo;
+import org.apache.hadoop.util.functional.CallableRaisingIOE;
 
 import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkArgument;
 import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull;
@@ -109,16 +111,25 @@ public class WriteOperationHelper implements WriteOperations {
   private final String bucket;
 
   /**
+   * statistics context.
+   */
+  private final S3AStatisticsContext statisticsContext;
+
+  /**
    * Constructor.
    * @param owner owner FS creating the helper
    * @param conf Configuration object
+   * @param statisticsContext statistics context
    *
    */
-  protected WriteOperationHelper(S3AFileSystem owner, Configuration conf) {
+  protected WriteOperationHelper(S3AFileSystem owner,
+      Configuration conf,
+      S3AStatisticsContext statisticsContext) {
     this.owner = owner;
     this.invoker = new Invoker(new S3ARetryPolicy(conf),
         this::operationRetried);
     this.conf = conf;
+    this.statisticsContext = statisticsContext;
     bucket = owner.getBucket();
   }
 
@@ -138,19 +149,19 @@ public class WriteOperationHelper implements WriteOperations {
 
   /**
    * Execute a function with retry processing.
+   * @param <T> type of return value
    * @param action action to execute (used in error messages)
    * @param path path of work (used in error messages)
    * @param idempotent does the operation have semantics
    * which mean that it can be retried even if was already executed?
    * @param operation operation to execute
-   * @param <T> type of return value
    * @return the result of the call
    * @throws IOException any IOE raised, or translated exception
    */
   public <T> T retry(String action,
       String path,
       boolean idempotent,
-      Invoker.Operation<T> operation)
+      CallableRaisingIOE<T> operation)
       throws IOException {
 
     return invoker.retry(action, path, idempotent, operation);
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java
index 95cbd7e..0b33614 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathIOException;
 import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
+import org.apache.hadoop.util.functional.CallableRaisingIOE;
 
 /**
  * Operations to update the store.
@@ -56,19 +57,19 @@ public interface WriteOperations {
 
   /**
    * Execute a function with retry processing.
+   * @param <T> type of return value
    * @param action action to execute (used in error messages)
    * @param path path of work (used in error messages)
    * @param idempotent does the operation have semantics
    * which mean that it can be retried even if was already executed?
    * @param operation operation to execute
-   * @param <T> type of return value
    * @return the result of the call
    * @throws IOException any IOE raised, or translated exception
    */
   <T> T retry(String action,
       String path,
       boolean idempotent,
-      Invoker.Operation<T> operation)
+      CallableRaisingIOE<T> operation)
       throws IOException;
 
   /**
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/S3ADelegationTokens.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/S3ADelegationTokens.java
index db149de..d8fc8b8 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.commons.lang3.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.s3a.AWSCredentialProviderList;
-import org.apache.hadoop.fs.s3a.S3AInstrumentation;
 import org.apache.hadoop.fs.s3a.auth.RoleModel;
 import org.apache.hadoop.fs.s3a.impl.StoreContext;
+import org.apache.hadoop.fs.s3a.statistics.DelegationTokenStatistics;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -45,6 +45,8 @@ import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.service.ServiceOperations;
 import org.apache.hadoop.util.DurationInfo;
 
+import static org.apache.hadoop.fs.s3a.Statistic.DELEGATION_TOKEN_ISSUED;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration;
 import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkArgument;
 import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkState;
 import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.DEFAULT_DELEGATION_TOKEN_BINDING;
@@ -135,9 +137,9 @@ public class S3ADelegationTokens extends AbstractDTService {
           AWSPolicyProvider.AccessLevel.WRITE);
 
   /**
-   * Statistics for the owner FS.
+   * Statistics for the operations.
    */
-  private S3AInstrumentation.DelegationTokenStatistics stats;
+  private DelegationTokenStatistics stats;
 
   /**
    * Name of the token binding as extracted from token kind; used for
@@ -428,8 +430,10 @@ public class S3ADelegationTokens extends AbstractDTService {
 
     try(DurationInfo ignored = new DurationInfo(LOG, DURATION_LOG_AT_INFO,
         "Creating New Delegation Token", tokenBinding.getKind())) {
-      Token<AbstractS3ATokenIdentifier> token
-          = tokenBinding.createDelegationToken(rolePolicy, encryptionSecrets, renewer);
+      Token<AbstractS3ATokenIdentifier> token = trackDuration(stats,
+          DELEGATION_TOKEN_ISSUED.getSymbol(), () ->
+              tokenBinding.createDelegationToken(rolePolicy,
+                  encryptionSecrets, renewer));
       if (token != null) {
         token.setService(service);
         noteTokenCreated(token);
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java
index 771d606..f736899 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java
@@ -47,6 +47,10 @@ import org.apache.hadoop.fs.s3a.S3AFileSystem;
 import org.apache.hadoop.fs.s3a.commit.files.PendingSet;
 import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit;
 import org.apache.hadoop.fs.s3a.commit.files.SuccessData;
+import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.JobStatus;
@@ -60,6 +64,7 @@ import org.apache.hadoop.util.concurrent.HadoopExecutors;
 import static org.apache.hadoop.fs.s3a.Constants.THREAD_POOL_SHUTDOWN_DELAY_SECONDS;
 import static org.apache.hadoop.fs.s3a.Invoker.ignoreIOExceptions;
 import static org.apache.hadoop.fs.s3a.S3AUtils.*;
+import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_COMMIT_JOB;
 import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
 import static org.apache.hadoop.fs.s3a.commit.CommitUtils.*;
 import static org.apache.hadoop.fs.s3a.commit.CommitUtilsWithMR.*;
@@ -67,6 +72,7 @@ import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.E_NO_SP
 import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.FS_S3A_COMMITTER_UUID;
 import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.FS_S3A_COMMITTER_UUID_SOURCE;
 import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.SPARK_WRITE_UUID;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfInvocation;
 
 /**
  * Abstract base class for S3A committers; allows for any commonality
@@ -94,7 +100,8 @@ import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.SPARK_W
  * created by a few tasks, <i>and</i> the HTTP connection pool in the S3A
  * committer was large enough for more all the parallel POST requests.
  */
-public abstract class AbstractS3ACommitter extends PathOutputCommitter {
+public abstract class AbstractS3ACommitter extends PathOutputCommitter
+    implements IOStatisticsSource {
 
   private static final Logger LOG =
       LoggerFactory.getLogger(AbstractS3ACommitter.class);
@@ -166,6 +173,8 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
   /** Should a job marker be created? */
   private final boolean createJobMarker;
 
+  private final CommitterStatistics committerStatistics;
+
   /**
    * Create a committer.
    * This constructor binds the destination directory and configuration, but
@@ -197,7 +206,9 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
     this.createJobMarker = context.getConfiguration().getBoolean(
         CREATE_SUCCESSFUL_JOB_OUTPUT_DIR_MARKER,
         DEFAULT_CREATE_SUCCESSFUL_JOB_DIR_MARKER);
-    this.commitOperations = new CommitOperations(fs);
+    // the statistics are shared between this committer and its operations.
+    this.committerStatistics = fs.newCommitterStatistics();
+    this.commitOperations = new CommitOperations(fs, committerStatistics);
   }
 
   /**
@@ -437,7 +448,12 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
     // The list of committed objects in pending is size limited in
     // ActiveCommit.uploadCommitted.
     filenames.addAll(pending.committedObjects);
-    maybeCreateSuccessMarker(context, filenames);
+    // load in all the pending statistics
+    IOStatisticsSnapshot snapshot = new IOStatisticsSnapshot(
+        pending.getIOStatistics());
+    snapshot.aggregate(getIOStatistics());
+
+    maybeCreateSuccessMarker(context, filenames, snapshot);
   }
 
   /**
@@ -448,10 +464,12 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
    * PUT up a the contents of a {@link SuccessData} file.
    * @param context job context
    * @param filenames list of filenames.
+   * @param ioStatistics any IO Statistics to include
    * @throws IOException IO failure
    */
   protected void maybeCreateSuccessMarker(JobContext context,
-      List<String> filenames)
+      List<String> filenames,
+      final IOStatisticsSnapshot ioStatistics)
       throws IOException {
     if (createJobMarker) {
       // create a success data structure and then save it
@@ -465,6 +483,7 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
       successData.setTimestamp(now.getTime());
       successData.setDate(now.toString());
       successData.setFilenames(filenames);
+      successData.getIOStatistics().aggregate(ioStatistics);
       commitOperations.createSuccessMarker(getOutputPath(), successData, true);
     }
   }
@@ -644,6 +663,7 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
             activeCommit.uploadCommitted(
                 commit.getDestinationKey(), commit.getLength());
           });
+      activeCommit.pendingsetCommitted(pendingSet.getIOStatistics());
     }
   }
 
@@ -728,8 +748,9 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
   protected void commitJobInternal(JobContext context,
       ActiveCommit pending)
       throws IOException {
-
-    commitPendingUploads(context, pending);
+    trackDurationOfInvocation(committerStatistics,
+        COMMITTER_COMMIT_JOB.getSymbol(),
+        () -> commitPendingUploads(context, pending));
   }
 
   @Override
@@ -1175,6 +1196,11 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
     }
   }
 
+  @Override
+  public IOStatistics getIOStatistics() {
+    return committerStatistics.getIOStatistics();
+  }
+
   /**
    * Scan for active uploads and list them along with a warning message.
    * Errors are ignored.
@@ -1386,6 +1412,13 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
     private long committedBytes;
 
     /**
+     * Aggregate statistics of all supplied by
+     * committed uploads.
+     */
+    private final IOStatisticsSnapshot ioStatistics =
+        new IOStatisticsSnapshot();
+
+    /**
      * Construct from a source FS and list of files.
      * @param sourceFS filesystem containing the list of pending files
      * @param sourceFiles .pendingset files to load and commit.
@@ -1433,7 +1466,8 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
      * @param key key of the committed object.
      * @param size size in bytes.
      */
-    public synchronized void uploadCommitted(String key, long size) {
+    public synchronized void uploadCommitted(String key,
+        long size) {
       if (committedObjects.size() < SUCCESS_MARKER_FILE_LIMIT) {
         committedObjects.add(
             key.startsWith("/") ? key : ("/" + key));
@@ -1442,6 +1476,19 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
       committedBytes += size;
     }
 
+    /**
+     * Callback when a pendingset has been committed,
+     * including any source statistics.
+     * @param sourceStatistics any source statistics
+     */
+    public void pendingsetCommitted(final IOStatistics sourceStatistics) {
+      ioStatistics.aggregate(sourceStatistics);
+    }
+
+    public IOStatisticsSnapshot getIOStatistics() {
+      return ioStatistics;
+    }
+
     public synchronized List<String> getCommittedObjects() {
       return committedObjects;
     }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java
index 11eeee9..c9fb380 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java
@@ -45,7 +45,6 @@ import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.PathIOException;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
-import org.apache.hadoop.fs.s3a.S3AInstrumentation;
 import org.apache.hadoop.fs.s3a.S3AUtils;
 import org.apache.hadoop.fs.s3a.WriteOperationHelper;
 import org.apache.hadoop.fs.s3a.commit.files.PendingSet;
@@ -53,13 +52,21 @@ import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit;
 import org.apache.hadoop.fs.s3a.commit.files.SuccessData;
 import org.apache.hadoop.fs.s3a.impl.InternalConstants;
 import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
+import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics;
+import org.apache.hadoop.fs.statistics.DurationTracker;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.DurationInfo;
 import org.apache.hadoop.util.Progressable;
 
+import static java.util.Objects.requireNonNull;
 import static org.apache.hadoop.fs.s3a.S3AUtils.*;
+import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_MATERIALIZE_FILE;
+import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_STAGE_FILE_UPLOAD;
 import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
 import static org.apache.hadoop.fs.s3a.Constants.*;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration;
 
 /**
  * The implementation of the various actions a committer needs.
@@ -71,7 +78,7 @@ import static org.apache.hadoop.fs.s3a.Constants.*;
  * duplicate that work.
  *
  */
-public class CommitOperations {
+public class CommitOperations implements IOStatisticsSource {
   private static final Logger LOG = LoggerFactory.getLogger(
       CommitOperations.class);
 
@@ -81,7 +88,7 @@ public class CommitOperations {
   private final S3AFileSystem fs;
 
   /** Statistics. */
-  private final S3AInstrumentation.CommitterStatistics statistics;
+  private final CommitterStatistics statistics;
 
   /**
    * Write operations for the destination fs.
@@ -105,9 +112,18 @@ public class CommitOperations {
    * @param fs FS to bind to
    */
   public CommitOperations(S3AFileSystem fs) {
-    Preconditions.checkArgument(fs != null, "null fs");
-    this.fs = fs;
-    statistics = fs.newCommitterStatistics();
+    this(requireNonNull(fs), fs.newCommitterStatistics());
+  }
+
+  /**
+   * Instantiate.
+   * @param fs FS to bind to
+   * @param committerStatistics committer statistics
+   */
+  public CommitOperations(S3AFileSystem fs,
+      CommitterStatistics committerStatistics) {
+    this.fs = requireNonNull(fs);
+    statistics = requireNonNull(committerStatistics);
     writeOperations = fs.getWriteOperationHelper();
   }
 
@@ -128,10 +144,15 @@ public class CommitOperations {
   }
 
   /** @return statistics. */
-  protected S3AInstrumentation.CommitterStatistics getStatistics() {
+  protected CommitterStatistics getStatistics() {
     return statistics;
   }
 
+  @Override
+  public IOStatistics getIOStatistics() {
+    return statistics.getIOStatistics();
+  }
+
   /**
    * Commit the operation, throwing an exception on any failure.
    * @param commit commit to execute
@@ -166,7 +187,8 @@ public class CommitOperations {
 
       commit.validate();
       destKey = commit.getDestinationKey();
-      long l = innerCommit(commit, operationState);
+      long l = trackDuration(statistics, COMMITTER_MATERIALIZE_FILE.getSymbol(),
+          () -> innerCommit(commit, operationState));
       LOG.debug("Successful commit of file length {}", l);
       outcome = MaybeIOE.NONE;
       statistics.commitCompleted(commit.getLength());
@@ -449,7 +471,7 @@ public class CommitOperations {
    * @return a pending upload entry
    * @throws IOException failure
    */
-  public SinglePendingCommit uploadFileToPendingCommit(File localFile,
+  public SinglePendingCommit  uploadFileToPendingCommit(File localFile,
       Path destPath,
       String partition,
       long uploadPartSize,
@@ -466,7 +488,11 @@ public class CommitOperations {
     String destKey = fs.pathToKey(destPath);
     String uploadId = null;
 
+    // flag to indicate to the finally clause that the operation
+    // failed. it is cleared as the last action in the try block.
     boolean threw = true;
+    final DurationTracker tracker = statistics.trackDuration(
+        COMMITTER_STAGE_FILE_UPLOAD.getSymbol());
     try (DurationInfo d = new DurationInfo(LOG,
         "Upload staged file from %s to %s",
         localFile.getAbsolutePath(),
@@ -507,6 +533,7 @@ public class CommitOperations {
       LOG.debug("File size is {}, number of parts to upload = {}",
           length, numParts);
       for (int partNumber = 1; partNumber <= numParts; partNumber += 1) {
+        progress.progress();
         long size = Math.min(length - offset, uploadPartSize);
         UploadPartRequest part;
         part = writeOperations.newUploadPartRequest(
@@ -525,7 +552,7 @@ public class CommitOperations {
 
       commitData.bindCommitData(parts);
       statistics.commitUploaded(length);
-      progress.progress();
+      // clear the threw flag.
       threw = false;
       return commitData;
     } finally {
@@ -536,6 +563,11 @@ public class CommitOperations {
           LOG.error("Failed to abort upload {} to {}", uploadId, destKey, e);
         }
       }
+      if (threw) {
+        tracker.failed();
+      }
+      // close tracker and so report statistics of success/failure
+      tracker.close();
     }
   }
 
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java
index 7f9dadf..41f95c8 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.Path;
 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 class MagicCommitIntegration {
       boolean magicCommitEnabled) {
     this.owner = owner;
     this.magicCommitEnabled = magicCommitEnabled;
+    this.storeContext = owner.createStoreContext();
   }
 
   /**
@@ -94,10 +98,10 @@ public class MagicCommitIntegration {
       if (isMagicCommitPath(elements)) {
         final String destKey = keyOfFinalDestination(elements, key);
         String pendingsetPath = key + CommitConstants.PENDING_SUFFIX;
-        owner.getInstrumentation()
-            .incrementCounter(Statistic.COMMITTER_MAGIC_FILES_CREATED, 1);
+        storeContext.incrementStatistic(
+            Statistic.COMMITTER_MAGIC_FILES_CREATED);
         tracker = new MagicCommitTracker(path,
-            owner.getBucket(),
+            storeContext.getBucket(),
             key,
             destKey,
             pendingsetPath,
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/PutTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/PutTracker.java
index bbffef3..10440f7 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/PutTracker.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/PutTracker.java
@@ -24,6 +24,7 @@ import java.util.List;
 import com.amazonaws.services.s3.model.PartETag;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.statistics.IOStatistics;
 
 /**
  * Multipart put tracker.
@@ -68,6 +69,7 @@ public class PutTracker {
    * @param uploadId Upload ID
    * @param parts list of parts
    * @param bytesWritten bytes written
+   * @param iostatistics nullable IO statistics
    * @return true if the commit is to be initiated immediately.
    * False implies the output stream does not need to worry about
    * what happens.
@@ -75,7 +77,8 @@ public class PutTracker {
    */
   public boolean aboutToComplete(String uploadId,
       List<PartETag> parts,
-      long bytesWritten)
+      long bytesWritten,
+      final IOStatistics iostatistics)
       throws IOException {
     return true;
   }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PendingSet.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PendingSet.java
index 8ad0342..fd73410 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PendingSet.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PendingSet.java
@@ -27,6 +27,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import com.fasterxml.jackson.annotation.JsonProperty;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -36,6 +37,8 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.s3a.commit.ValidationFailure;
+import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
 import org.apache.hadoop.util.JsonSerialization;
 
 import static org.apache.hadoop.fs.s3a.commit.CommitUtils.validateCollectionClass;
@@ -45,11 +48,22 @@ import static org.apache.hadoop.fs.s3a.commit.ValidationFailure.verify;
  * Persistent format for multiple pending commits.
  * Contains 0 or more {@link SinglePendingCommit} entries; validation logic
  * checks those values on load.
+ * <p>
+ * The statistics published through the {@link IOStatisticsSource}
+ * interface are the static ones marshalled with the commit data;
+ * they may be empty.
+ * </p>
+ * <p>
+ * As single commits are added via {@link #add(SinglePendingCommit)},
+ * any statistics from those commits are merged into the aggregate
+ * statistics, <i>and those of the single commit cleared.</i>
+ * </p>
  */
 @SuppressWarnings("unused")
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
-public class PendingSet extends PersistentCommitData {
+public class PendingSet extends PersistentCommitData
+    implements IOStatisticsSource {
   private static final Logger LOG = LoggerFactory.getLogger(PendingSet.class);
 
   /**
@@ -57,7 +71,7 @@ public class PendingSet extends PersistentCommitData {
    * If this is changed the value of {@link #serialVersionUID} will change,
    * to avoid deserialization problems.
    */
-  public static final int VERSION = 2;
+  public static final int VERSION = 3;
 
   /**
    * Serialization ID: {@value}.
@@ -81,6 +95,12 @@ public class PendingSet extends PersistentCommitData {
    */
   private final Map<String, String> extraData = new HashMap<>(0);
 
+  /**
+   * IOStatistics.
+   */
+  @JsonProperty("iostatistics")
+  private IOStatisticsSnapshot iostats = new IOStatisticsSnapshot();
+
   public PendingSet() {
     this(0);
   }
@@ -133,6 +153,12 @@ public class PendingSet extends PersistentCommitData {
    */
   public void add(SinglePendingCommit commit) {
     commits.add(commit);
+    // add any statistics.
+    IOStatisticsSnapshot st = commit.getIOStatistics();
+    if (st != null) {
+      iostats.aggregate(st);
+      st.clear();
+    }
   }
 
   /**
@@ -225,4 +251,12 @@ public class PendingSet extends PersistentCommitData {
     this.jobId = jobId;
   }
 
+  @Override
+  public IOStatisticsSnapshot getIOStatistics() {
+    return iostats;
+  }
+
+  public void setIOStatistics(final IOStatisticsSnapshot ioStatistics) {
+    this.iostats = ioStatistics;
+  }
 }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java
index ee7c7b2..021cc7b 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java
@@ -31,6 +31,8 @@ import java.util.List;
 import java.util.Map;
 
 import com.amazonaws.services.s3.model.PartETag;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
 import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
 
 import org.apache.commons.lang3.StringUtils;
@@ -39,6 +41,8 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.s3a.commit.ValidationFailure;
+import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
 import org.apache.hadoop.util.JsonSerialization;
 
 import static org.apache.hadoop.fs.s3a.commit.CommitUtils.validateCollectionClass;
@@ -46,20 +50,25 @@ import static org.apache.hadoop.fs.s3a.commit.ValidationFailure.verify;
 import static org.apache.hadoop.util.StringUtils.join;
 
 /**
- * This is the serialization format for uploads yet to be committerd.
- *
+ * This is the serialization format for uploads yet to be committed.
+ * <p>
  * It's marked as {@link Serializable} so that it can be passed in RPC
  * calls; for this to work it relies on the fact that java.io ArrayList
  * and LinkedList are serializable. If any other list type is used for etags,
  * it must also be serialized. Jackson expects lists, and it is used
  * to persist to disk.
- *
+ * </p>
+ * <p>
+ * The statistics published through the {@link IOStatisticsSource}
+ * interface are the static ones marshalled with the commit data;
+ * they may be empty.
+ * </p>
  */
 @SuppressWarnings("unused")
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 public class SinglePendingCommit extends PersistentCommitData
-    implements Iterable<String> {
+    implements Iterable<String>, IOStatisticsSource {
 
   /**
    * Serialization ID: {@value}.
@@ -113,6 +122,12 @@ public class SinglePendingCommit extends PersistentCommitData
    */
   private Map<String, String> extraData = new HashMap<>(0);
 
+  /**
+   * IOStatistics.
+   */
+  @JsonProperty("iostatistics")
+  private IOStatisticsSnapshot iostats = new IOStatisticsSnapshot();
+
   /** Destination file size. */
   private long length;
 
@@ -439,4 +454,12 @@ public class SinglePendingCommit extends PersistentCommitData
     this.length = length;
   }
 
+  @Override
+  public IOStatisticsSnapshot getIOStatistics() {
+    return iostats;
+  }
+
+  public void setIOStatistics(final IOStatisticsSnapshot ioStatistics) {
+    this.iostats = ioStatistics;
+  }
 }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SuccessData.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SuccessData.java
index b7509d6..4500e14 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SuccessData.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SuccessData.java
@@ -25,6 +25,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import com.fasterxml.jackson.annotation.JsonProperty;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -34,6 +35,8 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.s3a.commit.ValidationFailure;
+import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
 import org.apache.hadoop.util.JsonSerialization;
 
 /**
@@ -62,10 +65,19 @@ import org.apache.hadoop.util.JsonSerialization;
 @SuppressWarnings("unused")
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
-public class SuccessData extends PersistentCommitData {
+public class SuccessData extends PersistentCommitData
+    implements IOStatisticsSource {
+
   private static final Logger LOG = LoggerFactory.getLogger(SuccessData.class);
 
   /**
+   * Supported version value: {@value}.
+   * If this is changed the value of {@link #serialVersionUID} will change,
+   * to avoid deserialization problems.
+   */
+  public static final int VERSION = 1;
+
+  /**
    * Serialization ID: {@value}.
    */
   private static final long serialVersionUID = 507133045258460083L + VERSION;
@@ -75,7 +87,7 @@ public class SuccessData extends PersistentCommitData {
    * any other manifests: {@value}.
    */
   public static final String NAME
-      = "org.apache.hadoop.fs.s3a.commit.files.SuccessData/1";
+      = "org.apache.hadoop.fs.s3a.commit.files.SuccessData/" + VERSION;
 
   /**
    * Name of file; includes version marker.
@@ -126,6 +138,12 @@ public class SuccessData extends PersistentCommitData {
    */
   private List<String> filenames = new ArrayList<>(0);
 
+  /**
+   * IOStatistics.
+   */
+  @JsonProperty("iostatistics")
+  private IOStatisticsSnapshot iostats = new IOStatisticsSnapshot();
+
   @Override
   public void validate() throws ValidationFailure {
     ValidationFailure.verify(name != null,
@@ -350,4 +368,13 @@ public class SuccessData extends PersistentCommitData {
   public void setJobIdSource(final String jobIdSource) {
     this.jobIdSource = jobIdSource;
   }
+
+  @Override
+  public IOStatisticsSnapshot getIOStatistics() {
+    return iostats;
+  }
+
+  public void setIOStatistics(final IOStatisticsSnapshot ioStatistics) {
+    this.iostats = ioStatistics;
+  }
 }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java
index 1396764..0f1a0a6 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java
@@ -34,6 +34,8 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.s3a.WriteOperationHelper;
 import org.apache.hadoop.fs.s3a.commit.PutTracker;
 import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
 
 /**
  * Put tracker for Magic commits.
@@ -102,6 +104,7 @@ public class MagicCommitTracker extends PutTracker {
    * @param uploadId Upload ID
    * @param parts list of parts
    * @param bytesWritten bytes written
+   * @param iostatistics nullable IO statistics
    * @return false, indicating that the commit must fail.
    * @throws IOException any IO problem.
    * @throws IllegalArgumentException bad argument
@@ -109,7 +112,8 @@ public class MagicCommitTracker extends PutTracker {
   @Override
   public boolean aboutToComplete(String uploadId,
       List<PartETag> parts,
-      long bytesWritten)
+      long bytesWritten,
+      final IOStatistics iostatistics)
       throws IOException {
     Preconditions.checkArgument(StringUtils.isNotEmpty(uploadId),
         "empty/null upload ID: "+ uploadId);
@@ -117,6 +121,15 @@ public class MagicCommitTracker extends PutTracker {
         "No uploaded parts list");
     Preconditions.checkArgument(!parts.isEmpty(),
         "No uploaded parts to save");
+
+    // put a 0-byte file with the name of the original under-magic path
+    PutObjectRequest originalDestPut = writer.createPutObjectRequest(
+        originalDestKey,
+        new ByteArrayInputStream(EMPTY),
+        0);
+    writer.uploadObject(originalDestPut);
+
+    // build the commit summary
     SinglePendingCommit commitData = new SinglePendingCommit();
     commitData.touch(System.currentTimeMillis());
     commitData.setDestinationKey(getDestKey());
@@ -126,6 +139,8 @@ public class MagicCommitTracker extends PutTracker {
     commitData.setText("");
     commitData.setLength(bytesWritten);
     commitData.bindCommitData(parts);
+    commitData.setIOStatistics(
+        new IOStatisticsSnapshot(iostatistics));
     byte[] bytes = commitData.toBytes();
     LOG.info("Uncommitted data pending to file {};"
             + " commit metadata for {} parts in {}. sixe: {} byte(s)",
@@ -138,12 +153,6 @@ public class MagicCommitTracker extends PutTracker {
         bytes.length);
     writer.uploadObject(put);
 
-    // now put a 0-byte file with the name of the original under-magic path
-    PutObjectRequest originalDestPut = writer.createPutObjectRequest(
-        originalDestKey,
-        new ByteArrayInputStream(EMPTY),
-        0);
-    writer.uploadObject(originalDestPut);
     return false;
   }
 
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java
index b330cee..34bbfd4 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.fs.s3a.commit.CommitConstants;
 import org.apache.hadoop.fs.s3a.commit.CommitUtilsWithMR;
 import org.apache.hadoop.fs.s3a.commit.files.PendingSet;
 import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit;
+import org.apache.hadoop.fs.statistics.IOStatisticsLogging;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
@@ -47,6 +48,7 @@ import static org.apache.hadoop.fs.s3a.commit.CommitConstants.TASK_ATTEMPT_ID;
 import static org.apache.hadoop.fs.s3a.commit.CommitUtils.*;
 import static org.apache.hadoop.fs.s3a.commit.MagicCommitPaths.*;
 import static org.apache.hadoop.fs.s3a.commit.CommitUtilsWithMR.*;
+import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.demandStringifyIOStatistics;
 
 /**
  * This is a dedicated committer which requires the "magic" directory feature
@@ -169,6 +171,8 @@ public class MagicS3GuardCommitter extends AbstractS3ACommitter {
       destroyThreadPool();
     }
     getCommitOperations().taskCompleted(true);
+    LOG.debug("aggregate statistics\n{}",
+        demandStringifyIOStatistics(getIOStatistics()));
   }
 
   /**
@@ -213,6 +217,8 @@ public class MagicS3GuardCommitter extends AbstractS3ACommitter {
         taskAttemptID.getTaskID().toString() +
         CommitConstants.PENDINGSET_SUFFIX);
     LOG.info("Saving work of {} to {}", taskAttemptID, taskOutcomePath);
+    LOG.debug("task statistics\n{}",
+        IOStatisticsLogging.demandStringifyIOStatisticsSource(pendingSet));
     try {
       // We will overwrite if there exists a pendingSet file already
       pendingSet.save(getDestFS(), taskOutcomePath, true);
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ActiveOperationContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ActiveOperationContext.java
new file mode 100644
index 0000000..3ad2bbf
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ActiveOperationContext.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.impl;
+
+import javax.annotation.Nullable;
+
+import java.util.Objects;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
+import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
+
+/**
+ * Context for any active operation.
+ */
+public class ActiveOperationContext {
+
+  /**
+   * An operation ID; currently just for logging...proper tracing needs more.
+   */
+  private final long operationId;
+
+  /**
+   * Statistics context.
+   */
+  private final S3AStatisticsContext statisticsContext;
+
+  /**
+   * S3Guard bulk operation state, if (currently) set.
+   */
+  @Nullable private BulkOperationState bulkOperationState;
+
+  public ActiveOperationContext(
+      final long operationId,
+      final S3AStatisticsContext statisticsContext,
+      @Nullable final BulkOperationState bulkOperationState) {
+    this.operationId = operationId;
+    this.statisticsContext = Objects.requireNonNull(statisticsContext,
+        "null statistics context");
+    this.bulkOperationState = bulkOperationState;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "ActiveOperation{");
+    sb.append("operationId=").append(operationId);
+    sb.append(", bulkOperationState=").append(bulkOperationState);
+    sb.append('}');
+    return sb.toString();
+  }
+
+  @Nullable
+  public BulkOperationState getBulkOperationState() {
+    return bulkOperationState;
+  }
+
+  public long getOperationId() {
+    return operationId;
+  }
+
+  public S3AStatisticsContext getS3AStatisticsContext() {
+    return statisticsContext;
+  }
+
+  private static final AtomicLong NEXT_OPERATION_ID = new AtomicLong(0);
+
+  /**
+   * Create an operation ID. The nature of it should be opaque.
+   * @return an ID for the constructor.
+   */
+  protected static long newOperationId() {
+    return NEXT_OPERATION_ID.incrementAndGet();
+  }
+
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteRetryHandler.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteRetryHandler.java
index b2c1cc6..4169a98 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.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.fs.s3a.AWSClientIOException;
-import org.apache.hadoop.fs.s3a.S3AInstrumentation;
 import org.apache.hadoop.fs.s3a.S3AStorageStatistics;
 import org.apache.hadoop.fs.s3a.Statistic;
+import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
 
 import static org.apache.hadoop.fs.s3a.S3AUtils.isThrottleException;
 import static org.apache.hadoop.fs.s3a.Statistic.IGNORED_ERRORS;
@@ -53,7 +53,7 @@ public class BulkDeleteRetryHandler extends AbstractStoreOperation {
    */
   public static final String XML_PARSE_BROKEN = "Failed to parse XML document";
 
-  private final S3AInstrumentation instrumentation;
+  private final S3AStatisticsContext instrumentation;
 
   private final S3AStorageStatistics storageStatistics;
 
@@ -84,7 +84,6 @@ public class BulkDeleteRetryHandler extends AbstractStoreOperation {
    */
   protected void incrementStatistic(Statistic statistic, long count) {
     instrumentation.incrementCounter(statistic, count);
-    storageStatistics.incrementCounter(statistic, count);
   }
 
   /**
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CallableSupplier.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CallableSupplier.java
index 1c61d30..b788f50 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CallableSupplier.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CallableSupplier.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.fs.s3a.impl;
 
 import javax.annotation.Nullable;
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CancellationException;
@@ -31,7 +32,6 @@ import java.util.function.Supplier;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hadoop.fs.impl.WrappedIOException;
 import org.apache.hadoop.util.DurationInfo;
 
 import static org.apache.hadoop.fs.impl.FutureIOSupport.raiseInnerCause;
@@ -63,9 +63,9 @@ public final class CallableSupplier<T> implements Supplier {
     } catch (RuntimeException e) {
       throw e;
     } catch (IOException e) {
-      throw new WrappedIOException(e);
+      throw new UncheckedIOException(e);
     } catch (Exception e) {
-      throw new WrappedIOException(new IOException(e));
+      throw new UncheckedIOException(new IOException(e));
     }
   }
 
@@ -73,7 +73,7 @@ public final class CallableSupplier<T> implements Supplier {
    * Submit a callable into a completable future.
    * RTEs are rethrown.
    * Non RTEs are caught and wrapped; IOExceptions to
-   * {@link WrappedIOException} instances.
+   * {@code RuntimeIOException} instances.
    * @param executor executor.
    * @param call call to invoke
    * @param <T> type
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java
index bb025c4..4805056 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.PathIOException;
 import org.apache.hadoop.fs.s3a.NoVersionAttributeException;
 import org.apache.hadoop.fs.s3a.RemoteFileChangedException;
 import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
+import org.apache.hadoop.fs.s3a.statistics.ChangeTrackerStatistics;
 
 import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull;
 
@@ -72,7 +71,7 @@ public class ChangeTracker {
    * Mismatch counter; expected to be wired up to StreamStatistics except
    * during testing.
    */
-  private final AtomicLong versionMismatches;
+  private final ChangeTrackerStatistics versionMismatches;
 
   /**
    * Revision identifier (e.g. eTag or versionId, depending on change
@@ -90,7 +89,7 @@ public class ChangeTracker {
    */
   public ChangeTracker(final String uri,
       final ChangeDetectionPolicy policy,
-      final AtomicLong versionMismatches,
+      final ChangeTrackerStatistics versionMismatches,
       final S3ObjectAttributes s3ObjectAttributes) {
     this.policy = checkNotNull(policy);
     this.uri = uri;
@@ -111,8 +110,8 @@ public class ChangeTracker {
   }
 
   @VisibleForTesting
-  public AtomicLong getVersionMismatches() {
-    return versionMismatches;
+  public long getVersionMismatches() {
+    return versionMismatches.getVersionMismatches();
   }
 
   /**
@@ -177,7 +176,7 @@ public class ChangeTracker {
       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 class ChangeTracker {
       // 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 @@ public class ChangeTracker {
               uri,
               pos,
               operation,
-              versionMismatches.get());
+              versionMismatches.getVersionMismatches());
       if (pair.left) {
         // an mismatch has occurred: note it.
-        versionMismatches.incrementAndGet();
+        versionMismatches.versionMismatchError();
       }
       if (pair.right != null) {
         // there's an exception to raise: do it
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ErrorTranslation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ErrorTranslation.java
index d1baf3c..f7e0641 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ErrorTranslation.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ErrorTranslation.java
@@ -55,6 +55,18 @@ public class ErrorTranslation {
   }
 
   /**
+   * Does this exception indicate that a reference to an object
+   * returned a 404. Unknown bucket errors do not match this
+   * predicate.
+   * @param e exception.
+   * @return true if the status code and error code mean that the
+   * HEAD request returned 404 but the bucket was there.
+   */
+  public static boolean isObjectNotFound(AmazonServiceException e) {
+    return e.getStatusCode() == SC_404 && !isUnknownBucket(e);
+  }
+
+  /**
    * AWS error codes explicitly recognized and processes specially;
    * kept in their own class for isolation.
    */
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java
index 9ecc1a9..0e8d864 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java
@@ -110,4 +110,11 @@ public final class InternalConstants {
    * Value: {@value}.
    */
   public static final int DEFAULT_UPLOAD_PART_COUNT_LIMIT = 10000;
+
+  /**
+   * Flag to enable AWS Statistics binding. As this is triggering
+   * problems related to region/endpoint setup, it is currently
+   * disabled.
+   */
+  public static final boolean AWS_SDK_METRICS_ENABLED = false;
 }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ListingOperationCallbacks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ListingOperationCallbacks.java
index e5f9f7d..b0ebf1c 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ListingOperationCallbacks.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ListingOperationCallbacks.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus;
 import org.apache.hadoop.fs.s3a.S3ListRequest;
 import org.apache.hadoop.fs.s3a.S3ListResult;
 import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider;
+import org.apache.hadoop.fs.statistics.DurationTrackerFactory;
 
 /**
  * These are all the callbacks which
@@ -44,12 +45,14 @@ public interface ListingOperationCallbacks {
    *
    * Retry policy: retry untranslated.
    * @param request request to initiate
+   * @param trackerFactory tracker with statistics to update
    * @return the results
    * @throws IOException if the retry invocation raises one (it shouldn't).
    */
   @Retries.RetryRaw
   CompletableFuture<S3ListResult> listObjectsAsync(
-          S3ListRequest request)
+          S3ListRequest request,
+          DurationTrackerFactory trackerFactory)
           throws IOException;
 
   /**
@@ -57,13 +60,15 @@ public interface ListingOperationCallbacks {
    * Retry policy: retry untranslated.
    * @param request last list objects request to continue
    * @param prevResult last paged result to continue from
+   * @param trackerFactory tracker with statistics to update
    * @return the next result object
    * @throws IOException none, just there for retryUntranslated.
    */
   @Retries.RetryRaw
   CompletableFuture<S3ListResult> continueListObjectsAsync(
           S3ListRequest request,
-          S3ListResult prevResult)
+          S3ListResult prevResult,
+          DurationTrackerFactory trackerFactory)
           throws IOException;
 
   /**
@@ -117,4 +122,5 @@ public interface ListingOperationCallbacks {
    * @return true iff the path is authoritative on the client.
    */
   boolean allowAuthoritative(Path p);
+
 }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/NetworkBinding.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/NetworkBinding.java
index ca1b09e..3eeb01f 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/NetworkBinding.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/NetworkBinding.java
@@ -51,22 +51,18 @@ public class NetworkBinding {
           ".thirdparty.apache.http.conn.ssl.SSLConnectionSocketFactory";
 
   /**
-   * Configures the {@link com.amazonaws.thirdparty.apache.http.conn.ssl
-   * .SSLConnectionSocketFactory} used by the AWS SDK. A custom
-   * SSLConnectionSocketFactory can be set using the method
-   * {@link com.amazonaws.ApacheHttpClientConfig#setSslSocketFactory(
-   * com.amazonaws.thirdparty.apache.http.conn.socket.ConnectionSocketFactory)}.
-   * If {@link com.amazonaws.thirdparty.apache.http.conn.ssl
-   * .SSLConnectionSocketFactory} cannot be found on the classpath, the value
+   * Configures the {@code SSLConnectionSocketFactory} used by the AWS SDK.
+   * A custom Socket Factory can be set using the method
+   * {@code setSslSocketFactory()}.
+   * If {@code SSLConnectionSocketFactory} cannot be found on the classpath, the value
    * of {@link org.apache.hadoop.fs.s3a.Constants#SSL_CHANNEL_MODE} is ignored.
    *
    * @param conf the {@link Configuration} used to get the client specified
-   *             value of {@link org.apache.hadoop.fs.s3a.Constants
-   *             #SSL_CHANNEL_MODE}
-   * @param awsConf the {@link ClientConfiguration} to set the
+   *             value of {@code SSL_CHANNEL_MODE}
+   * @param awsConf the {@code ClientConfiguration} to set the
    *                SSLConnectionSocketFactory for.
    * @throws IOException if there is an error while initializing the
-   * {@link SSLSocketFactory} other than classloader problems.
+   * {@code SSLSocketFactory} other than classloader problems.
    */
   public static void bindSSLChannelMode(Configuration conf,
       ClientConfiguration awsConf) throws IOException {
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploader.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploader.java
index e811af9..5ff116f 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploader.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploader.java
@@ -30,6 +30,7 @@ import java.util.Comparator;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 import java.util.Set;
 import java.util.concurrent.CompletableFuture;
 
@@ -53,8 +54,11 @@ import org.apache.hadoop.fs.PathIOException;
 import org.apache.hadoop.fs.UploadHandle;
 import org.apache.hadoop.fs.impl.AbstractMultipartUploader;
 import org.apache.hadoop.fs.s3a.WriteOperations;
-import org.apache.hadoop.fs.s3a.impl.statistics.S3AMultipartUploaderStatistics;
+import org.apache.hadoop.fs.s3a.statistics.S3AMultipartUploaderStatistics;
 import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+
+import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToString;
 
 /**
  * MultipartUploader for S3AFileSystem. This uses the S3 multipart
@@ -102,7 +106,7 @@ class S3AMultipartUploader extends AbstractMultipartUploader {
     this.builder = builder;
     this.writeOperations = writeOperations;
     this.context = context;
-    this.statistics = statistics;
+    this.statistics = Objects.requireNonNull(statistics);
   }
 
   @Override
@@ -113,6 +117,22 @@ class S3AMultipartUploader extends AbstractMultipartUploader {
     super.close();
   }
 
+  @Override
+  public IOStatistics getIOStatistics() {
+    return statistics.getIOStatistics();
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "S3AMultipartUploader{");
+    sb.append("base=").append(getBasePath());
+    sb.append("; statistics=").append(
+        ioStatisticsToString(statistics.getIOStatistics()));
+    sb.append('}');
+    return sb.toString();
+  }
+
   /**
    * Retrieve the operation state; create one on demand if needed
    * <i>and there has been no unsuccessful attempt to create one.</i>
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploaderBuilder.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploaderBuilder.java
index 3bf1a7d..cb38b82 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploaderBuilder.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploaderBuilder.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.impl.MultipartUploaderBuilderImpl;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
 import org.apache.hadoop.fs.s3a.WriteOperations;
-import org.apache.hadoop.fs.s3a.impl.statistics.S3AMultipartUploaderStatistics;
+import org.apache.hadoop.fs.s3a.statistics.S3AMultipartUploaderStatistics;
 
 /**
  * Builder for S3A multipart uploaders.
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java
index b375c31..28be486 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java
@@ -33,9 +33,9 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.s3a.Invoker;
 import org.apache.hadoop.fs.s3a.S3AFileStatus;
 import org.apache.hadoop.fs.s3a.S3AInputPolicy;
-import org.apache.hadoop.fs.s3a.S3AInstrumentation;
 import org.apache.hadoop.fs.s3a.S3AStorageStatistics;
 import org.apache.hadoop.fs.s3a.Statistic;
+import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
 import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider;
 import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -88,7 +88,8 @@ public class StoreContext {
   private final Invoker invoker;
 
   /** Instrumentation and statistics. */
-  private final S3AInstrumentation instrumentation;
+  private final S3AStatisticsContext instrumentation;
+
   private final S3AStorageStatistics storageStatistics;
 
   /** Seek policy. */
@@ -129,7 +130,7 @@ public class 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,
@@ -186,7 +187,12 @@ public class StoreContext {
     return invoker;
   }
 
-  public S3AInstrumentation getInstrumentation() {
+  /**
+   * Get the statistics context for this StoreContext.
+   * @return the statistics context this store context was created
+   * with.
+   */
+  public S3AStatisticsContext getInstrumentation() {
     return instrumentation;
   }
 
@@ -268,7 +274,6 @@ public class StoreContext {
    */
   public void incrementStatistic(Statistic statistic, long count) {
     instrumentation.incrementCounter(statistic, count);
-    storageStatistics.incrementCounter(statistic, count);
   }
 
   /**
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContextBuilder.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContextBuilder.java
index 44353b9..e1f2cb1 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContextBuilder.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContextBuilder.java
@@ -25,8 +25,8 @@ import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListeningE
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.s3a.Invoker;
 import org.apache.hadoop.fs.s3a.S3AInputPolicy;
-import org.apache.hadoop.fs.s3a.S3AInstrumentation;
 import org.apache.hadoop.fs.s3a.S3AStorageStatistics;
+import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
 import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider;
 import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -52,7 +52,7 @@ public class StoreContextBuilder {
 
   private Invoker invoker;
 
-  private S3AInstrumentation instrumentation;
+  private S3AStatisticsContext instrumentation;
 
   private S3AStorageStatistics storageStatistics;
 
@@ -113,7 +113,7 @@ public class StoreContextBuilder {
   }
 
   public StoreContextBuilder setInstrumentation(
-      final S3AInstrumentation instr) {
+      final S3AStatisticsContext instr) {
     this.instrumentation = instr;
     return this;
   }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java
index 8ec8488..eaf9ee2 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.fs.s3a.s3guard;
 import javax.annotation.Nullable;
 import java.io.IOException;
 import java.io.InterruptedIOException;
+import java.io.UncheckedIOException;
 import java.net.URI;
 import java.nio.file.AccessDeniedException;
 import java.util.ArrayList;
@@ -79,8 +80,8 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathIOException;
 import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.fs.impl.FunctionsRaisingIOE;
-import org.apache.hadoop.fs.impl.WrappedIOException;
+import org.apache.hadoop.util.functional.CallableRaisingIOE;
+import org.apache.hadoop.util.functional.RemoteIterators;
 import org.apache.hadoop.fs.s3a.AWSCredentialProviderList;
 import org.apache.hadoop.fs.s3a.AWSServiceThrottledException;
 import org.apache.hadoop.fs.s3a.Constants;
@@ -450,7 +451,8 @@ public class DynamoDBMetadataStore implements MetadataStore,
     owner = fs;
     conf = owner.getConf();
     StoreContext context = owner.createStoreContext();
-    instrumentation = context.getInstrumentation().getS3GuardInstrumentation();
+    instrumentation = context.getInstrumentation()
+        .getS3GuardInstrumentation();
     username = context.getUsername();
     executor = context.createThrottledExecutor();
     ttlTimeProvider = Preconditions.checkNotNull(
@@ -638,8 +640,9 @@ public class DynamoDBMetadataStore implements MetadataStore,
       LOG.debug("Subtree path {} is deleted; this will be a no-op", path);
       return;
     }
-    deleteEntries(new InternalIterators.PathFromRemoteStatusIterator(
-        new DescendantsIterator(this, meta)),
+    deleteEntries(RemoteIterators.mappingRemoteIterator(
+        new DescendantsIterator(this, meta),
+        FileStatus::getPath),
         operationState);
   }
 
@@ -648,8 +651,7 @@ public class DynamoDBMetadataStore implements MetadataStore,
   public void deletePaths(Collection<Path> paths,
       final BulkOperationState operationState)
       throws IOException {
-    deleteEntries(
-        new InternalIterators.RemoteIteratorFromIterator<>(paths.iterator()),
+    deleteEntries(RemoteIterators.remoteIteratorFromIterable(paths),
         operationState);
   }
 
@@ -826,7 +828,7 @@ public class DynamoDBMetadataStore implements MetadataStore,
       for (Item item : wrapWithRetries(items)) {
         metas.add(itemToPathMetadata(item, username));
       }
-    } catch (WrappedIOException e) {
+    } catch (UncheckedIOException e) {
       // failure in the iterators; unwrap.
       throw e.getCause();
     }
@@ -1634,7 +1636,7 @@ public class DynamoDBMetadataStore implements MetadataStore,
       Set<Path> clearedParentPathSet = new HashSet<>();
       // declare the operation to delete a batch as a function so
       // as to keep the code consistent across multiple uses.
-      FunctionsRaisingIOE.CallableRaisingIOE<Void> deleteBatchOperation =
+      CallableRaisingIOE<Void> deleteBatchOperation =
           () -> {
             // lowest path entries get deleted first.
             deletionBatch.sort(PathOrderComparators.TOPMOST_PATH_LAST);
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/InternalIterators.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/InternalIterators.java
deleted file mode 100644
index dd6fb5f..0000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/InternalIterators.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.s3a.s3guard;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.fs.s3a.S3AFileStatus;
-
-/**
- * Internal iterators.
- */
-final class InternalIterators {
-
-  private InternalIterators() {
-  }
-
-  /**
-   * From a remote status iterator, build a path iterator.
-   */
-  static final class PathFromRemoteStatusIterator implements
-      RemoteIterator<Path> {
-
-    private final RemoteIterator<S3AFileStatus> source;
-
-    /**
-     * Construct.
-     * @param source source iterator.
-     */
-    PathFromRemoteStatusIterator(final RemoteIterator<S3AFileStatus> source) {
-      this.source = source;
-    }
-
-    @Override
-    public boolean hasNext() throws IOException {
-      return source.hasNext();
-    }
-
-    @Override
-    public Path next() throws IOException {
-      return source.next().getPath();
-    }
-  }
-
-  /**
-   * From a classic java.util.Iterator, build a Hadoop remote iterator.
-   * @param <T> type of iterated value.
-   */
-  static final class RemoteIteratorFromIterator<T> implements
-      RemoteIterator<T> {
-
-    private final Iterator<T> source;
-
-    /**
-     * Construct.
-     * @param source source iterator.
-     */
-    RemoteIteratorFromIterator(final Iterator<T> source) {
-      this.source = source;
-    }
-
-    @Override
-    public boolean hasNext() {
-      return source.hasNext();
-    }
-
-    @Override
-    public T next() {
-      return source.next();
-    }
-  }
-
-}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RetryingCollection.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RetryingCollection.java
index 394f393..108d205 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RetryingCollection.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RetryingCollection.java
@@ -19,9 +19,9 @@
 package org.apache.hadoop.fs.s3a.s3guard;
 
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.util.Iterator;
 
-import org.apache.hadoop.fs.impl.WrappedIOException;
 import org.apache.hadoop.fs.s3a.Invoker;
 import org.apache.hadoop.fs.s3a.Retries;
 
@@ -88,7 +88,7 @@ class RetryingCollection<T> implements Iterable<T> {
 
     /**
      * {@inheritDoc}.
-     * @throws WrappedIOException for IO failure, including throttling.
+     * @throws UncheckedIOException for IO failure, including throttling.
      */
     @Override
     @Retries.RetryTranslated
@@ -100,13 +100,13 @@ class RetryingCollection<T> implements Iterable<T> {
             true,
             iterator::hasNext);
       } catch (IOException e) {
-        throw new WrappedIOException(e);
+        throw new UncheckedIOException(e);
       }
     }
 
     /**
      * {@inheritDoc}.
-     * @throws WrappedIOException for IO failure, including throttling.
+     * @throws UncheckedIOException for IO failure, including throttling.
      */
     @Override
     @Retries.RetryTranslated
@@ -118,7 +118,7 @@ class RetryingCollection<T> implements Iterable<T> {
             true,
             iterator::next);
       } catch (IOException e) {
-        throw new WrappedIOException(e);
+        throw new UncheckedIOException(e);
       }
     }
   }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectInputStream.java
index 69b9aaf..02d1e53 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.FSExceptionMessages;
 import org.apache.hadoop.fs.FSInputStream;
 import org.apache.hadoop.fs.PathIOException;
 import org.apache.hadoop.fs.s3a.Retries;
-import org.apache.hadoop.fs.s3a.S3AInstrumentation;
 import org.apache.hadoop.fs.s3a.S3AReadOpContext;
 import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
+import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
 import org.apache.hadoop.io.IOUtils;
 
 import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull;
@@ -103,7 +103,7 @@ public class SelectInputStream extends FSInputStream implements
 
   private final S3AReadOpContext readContext;
 
-  private final S3AInstrumentation.InputStreamStatistics streamStatistics;
+  private final S3AInputStreamStatistics streamStatistics;
 
   private long readahead;
 
@@ -130,7 +130,7 @@ public class SelectInputStream extends FSInputStream implements
     this.uri = "s3a://" + this.bucket + "/" + this.key;
     this.readContext = readContext;
     this.readahead = readContext.getReadahead();
-    this.streamStatistics = readContext.getInstrumentation()
+    this.streamStatistics = readContext.getS3AStatisticsContext()
         .newInputStreamStatistics();
     SelectRecordsInputStream stream = once(
         "S3 Select",
@@ -204,7 +204,7 @@ public class SelectInputStream extends FSInputStream implements
     long skipped = once("skip", uri, () -> wrappedStream.skip(n));
     pos.addAndGet(skipped);
     // treat as a forward skip for stats
-    streamStatistics.seekForwards(skipped);
+    streamStatistics.seekForwards(skipped, skipped);
     return skipped;
   }
 
@@ -331,7 +331,7 @@ public class SelectInputStream extends FSInputStream implements
         bytesSkipped++;
       }
       // read has finished.
-      streamStatistics.seekForwards(bytesSkipped);
+      streamStatistics.seekForwards(bytesSkipped, bytesSkipped);
     }
   }
 
@@ -428,7 +428,7 @@ public class SelectInputStream extends FSInputStream implements
    */
   @InterfaceAudience.Private
   @InterfaceStability.Unstable
-  public S3AInstrumentation.InputStreamStatistics getS3AStreamStatistics() {
+  public S3AInputStreamStatistics getS3AStreamStatistics() {
     return streamStatistics;
   }
 
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/BlockOutputStreamStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/BlockOutputStreamStatistics.java
new file mode 100644
index 0000000..b1cee71
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/BlockOutputStreamStatistics.java
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.statistics;
+
+import java.io.Closeable;
+import java.time.Duration;
+
+/**
+ * Block output stream statistics.
+ */
+public interface BlockOutputStreamStatistics extends Closeable,
+    S3AStatisticInterface {
+
+  /**
+   * Block is queued for upload.
+   * @param blockSize block size.
+   */
+  void blockUploadQueued(int blockSize);
+
+  /**
+   * Queued block has been scheduled for upload.
+   * @param timeInQueue time in the queue.
+   * @param blockSize block size.
+   */
+  void blockUploadStarted(Duration timeInQueue, int blockSize);
+
+  /**
+   * A block upload has completed. Duration excludes time in the queue.
+   * @param timeSinceUploadStarted time in since the transfer began.
+   * @param blockSize block size
+   */
+  void blockUploadCompleted(Duration timeSinceUploadStarted, int blockSize);
+
+  /**
+   *  A block upload has failed. Duration excludes time in the queue.
+   * <p>
+   *  A final transfer completed event is still expected, so this
+   *  does not decrement the active block counter.
+   * </p>
+   * @param timeSinceUploadStarted time in since the transfer began.
+   * @param blockSize block size
+   */
+  void blockUploadFailed(Duration timeSinceUploadStarted, int blockSize);
+
+  /**
+   * Intermediate report of bytes uploaded.
+   * @param byteCount bytes uploaded
+   */
+  void bytesTransferred(long byteCount);
+
+  /**
+   * Note exception in a multipart complete.
+   * @param count count of exceptions
+   */
+  void exceptionInMultipartComplete(int count);
+
+  /**
+   * Note an exception in a multipart abort.
+   */
+  void exceptionInMultipartAbort();
+
+  /**
+   * Get the number of bytes pending upload.
+   * @return the number of bytes in the pending upload state.
+   */
+  long getBytesPendingUpload();
+
+  /**
+   * Data has been uploaded to be committed in a subsequent operation;
+   * to be called at the end of the write.
+   * @param size size in bytes
+   */
+  void commitUploaded(long size);
+
+  int getBlocksAllocated();
+
+  int getBlocksReleased();
+
+  /**
+   * Get counters of blocks actively allocated; may be inaccurate
+   * if the numbers change during the (non-synchronized) calculation.
+   * @return the number of actively allocated blocks.
+   */
+  int getBlocksActivelyAllocated();
+
+  /**
+   * Record bytes written.
+   * @param count number of bytes
+   */
+  void writeBytes(long count);
+
+  /**
+   * Get the current count of bytes written.
+   * @return the counter value.
+   */
+  long getBytesWritten();
+
+  /**
+   * A block has been allocated.
+   */
+  void blockAllocated();
+
+  /**
+   * A block has been released.
+   */
+  void blockReleased();
+
+  /**
+   * Get the value of a counter.
+   * @param name counter name
+   * @return the value or null if no matching counter was found.
+   */
+  Long lookupCounterValue(String name);
+
+  /**
+   * Get the value of a gauge.
+   * @param name gauge name
+   * @return the value or null if no matching gauge was found.
+   */
+  Long lookupGaugeValue(String name);
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/ChangeTrackerStatistics.java
similarity index 69%
copy from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatistics.java
copy to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/ChangeTrackerStatistics.java
index 2cd74ff..572581e 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatistics.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/ChangeTrackerStatistics.java
@@ -16,24 +16,21 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.fs.s3a.impl.statistics;
-
-import java.io.Closeable;
+package org.apache.hadoop.fs.s3a.statistics;
 
 /**
- * Statistics for the S3A multipart uploader.
+ * Interface for change tracking statistics.
  */
-public interface S3AMultipartUploaderStatistics extends Closeable {
-
-  void instantiated();
-
-  void uploadStarted();
-
-  void partPut(long lengthInBytes);
-
-  void uploadCompleted();
-
-  void uploadAborted();
-
-  void abortUploadsUnderPathInvoked();
+public interface ChangeTrackerStatistics {
+
+  /**
+   * A version mismatch was detected.
+   */
+  void versionMismatchError();
+
+  /**
+   * How many version mismatches have occurred.
+   * @return counter of mismatches.
+   */
+  long getVersionMismatches();
 }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/CommitterStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/CommitterStatistics.java
new file mode 100644
index 0000000..fd232a0
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/CommitterStatistics.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.statistics;
+
+/**
+ * Statistics for S3A committers.
+ */
+public interface CommitterStatistics
+    extends S3AStatisticInterface {
+
+  /** A commit has been created. */
+  void commitCreated();
+
+  /**
+   * Data has been uploaded to be committed in a subsequent operation.
+   * @param size size in bytes
+   */
+  void commitUploaded(long size);
+
+  /**
+   * A commit has been completed.
+   * @param size size in bytes
+   */
+  void commitCompleted(long size);
+
+  /** A commit has been aborted. */
+  void commitAborted();
+
+  /**
+   * A commit was reverted.
+   */
+  void commitReverted();
+
+  /**
+   * A commit failed.
+   */
+  void commitFailed();
+
+  /**
+   * Note that a task has completed.
+   * @param success success flag
+   */
+  void taskCompleted(boolean success);
+
+  /**
+   * Note that a job has completed.
+   * @param success success flag
+   */
+  void jobCompleted(boolean success);
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/CountersAndGauges.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/CountersAndGauges.java
new file mode 100644
index 0000000..61cc033
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/CountersAndGauges.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.statistics;
+
+import java.time.Duration;
+
+import org.apache.hadoop.fs.s3a.Statistic;
+
+/**
+ * This is the foundational API for collecting S3A statistics.
+ */
+public interface CountersAndGauges {
+
+  /**
+   * Increment a specific counter.
+   * No-op if not defined.
+   * @param op operation
+   * @param count increment value
+   */
+  void incrementCounter(Statistic op, long count);
+
+  /**
+   * Increment a specific gauge.
+   * No-op if not defined.
+   * @param op operation
+   * @param count increment value
+   * @throws ClassCastException if the metric is of the wrong type
+   */
+  void incrementGauge(Statistic op, long count);
+
+  /**
+   * Decrement a specific gauge.
+   * No-op if not defined.
+   * @param op operation
+   * @param count increment value
+   * @throws ClassCastException if the metric is of the wrong type
+   */
+  void decrementGauge(Statistic op, long count);
+
+  /**
+   * Add a value to a quantiles statistic. No-op if the quantile
+   * isn't found.
+   * @param op operation to look up.
+   * @param value value to add.
+   * @throws ClassCastException if the metric is not a Quantiles.
+   */
+  void addValueToQuantiles(Statistic op, long value);
+
+  /**
+   * Record a duration.
+   * @param op operation
+   * @param success was the operation a success?
+   * @param duration how long did it take
+   */
+  void recordDuration(Statistic op, boolean success, Duration duration);
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/DelegationTokenStatistics.java
similarity index 69%
copy from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatistics.java
copy to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/DelegationTokenStatistics.java
index 2cd74ff..3ab9224 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatistics.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/DelegationTokenStatistics.java
@@ -16,24 +16,13 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.fs.s3a.impl.statistics;
-
-import java.io.Closeable;
+package org.apache.hadoop.fs.s3a.statistics;
 
 /**
- * Statistics for the S3A multipart uploader.
+ * Instrumentation exported to for S3A Delegation Token support.
  */
-public interface S3AMultipartUploaderStatistics extends Closeable {
-
-  void instantiated();
-
-  void uploadStarted();
-
-  void partPut(long lengthInBytes);
-
-  void uploadCompleted();
-
-  void uploadAborted();
+public interface DelegationTokenStatistics extends S3AStatisticInterface {
 
-  void abortUploadsUnderPathInvoked();
+  /** A token has been issued. */
+  void tokenIssued();
 }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AInputStreamStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AInputStreamStatistics.java
new file mode 100644
index 0000000..328d9f7
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AInputStreamStatistics.java
@@ -0,0 +1,191 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.statistics;
+
+import org.apache.hadoop.fs.statistics.DurationTracker;
+
+/**
+ * Statistics updated by a
+ * {@link org.apache.hadoop.fs.s3a.S3AInputStream} during its use.
+ * It also contains getters for tests.
+ */
+public interface S3AInputStreamStatistics extends AutoCloseable,
+    S3AStatisticInterface {
+
+  /**
+   * Seek backwards, incrementing the seek and backward seek counters.
+   * @param negativeOffset how far was the seek?
+   * This is expected to be negative.
+   */
+  void seekBackwards(long negativeOffset);
+
+  /**
+   * Record a forward seek, adding a seek operation, a forward
+   * seek operation, and any bytes skipped.
+   * @param skipped bytes moved forward in stream
+   * @param bytesReadInSeek number of bytes skipped by reading from the stream.
+   * If the seek was implemented by a close + reopen, set this to zero.
+   */
+  void seekForwards(long skipped, long bytesReadInSeek);
+
+  /**
+   * The inner stream was opened.
+   * The return value is used in the input stream to decide whether it is
+   * the initial vs later count.
+   * @return the previous count or zero if this is the first opening.
+   */
+  long streamOpened();
+
+  /**
+   * The inner stream was closed.
+   * @param abortedConnection flag to indicate the stream was aborted,
+   * rather than closed cleanly
+   * @param remainingInCurrentRequest the number of bytes remaining in
+   * the current request.
+   */
+  void streamClose(boolean abortedConnection,
+      long remainingInCurrentRequest);
+
+  /**
+   * An ignored stream read exception was received.
+   */
+  void readException();
+
+  /**
+   * Increment the bytes read counter by the number of bytes;
+   * no-op if the argument is negative.
+   * @param bytes number of bytes read
+   */
+  void bytesRead(long bytes);
+
+  /**
+   * A {@code read(byte[] buf, int off, int len)} operation has started.
+   * @param pos starting position of the read
+   * @param len length of bytes to read
+   */
+  void readOperationStarted(long pos, long len);
+
+  /**
+   * A {@code PositionedRead.read(position, buffer, offset, length)}
+   * operation has just started.
+   * @param pos starting position of the read
+   * @param len length of bytes to read
+   */
+  void readFullyOperationStarted(long pos, long len);
+
+  /**
+   * A read operation has completed.
+   * @param requested number of requested bytes
+   * @param actual the actual number of bytes
+   */
+  void readOperationCompleted(int requested, int actual);
+
+  @Override
+  void close();
+
+  /**
+   * The input policy has been switched.
+   * @param updatedPolicy enum value of new policy.
+   */
+  void inputPolicySet(int updatedPolicy);
+
+  /**
+   * Get a reference to the change tracker statistics for this
+   * stream.
+   * @return a reference to the change tracker statistics
+   */
+  ChangeTrackerStatistics getChangeTrackerStatistics();
+
+  /**
+   * A stream {@code unbuffer()} call has been made.
+   */
+  void unbuffered();
+
+  long getCloseOperations();
+
+  long getClosed();
+
+  long getAborted();
+
+  long getForwardSeekOperations();
+
+  long getBackwardSeekOperations();
+
+  /**
+   * The bytes read in read() operations.
+   * @return the number of bytes returned to the caller.
+   */
+  long getBytesRead();
+
+  /**
+   * The total number of bytes read, including
+   * all read and discarded when closing streams
+   * or skipped during seek calls.
+   * @return the total number of bytes read from
+   * S3.
+   */
+  long getTotalBytesRead();
+
+  long getBytesSkippedOnSeek();
+
+  long getBytesBackwardsOnSeek();
+
+  long getBytesReadInClose();
+
+  long getBytesDiscardedInAbort();
+
+  long getOpenOperations();
+
+  long getSeekOperations();
+
+  long getReadExceptions();
+
+  long getReadOperations();
+
+  long getReadFullyOperations();
+
+  long getReadsIncomplete();
+
+  long getPolicySetCount();
+
+  long getVersionMismatches();
+
+  long getInputPolicy();
+
+  /**
+   * Get the value of a counter.
+   * @param name counter name
+   * @return the value or null if no matching counter was found.
+   */
+  Long lookupCounterValue(String name);
+
+  /**
+   * Get the value of a gauge.
+   * @param name gauge name
+   * @return the value or null if no matching gauge was found.
+   */
+  Long lookupGaugeValue(String name);
+
+  /**
+   * Initiate a GET request.
+   * @return duration tracker;
+   */
+  DurationTracker initiateGetRequest();
+
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AMultipartUploaderStatistics.java
similarity index 88%
copy from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatistics.java
copy to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AMultipartUploaderStatistics.java
index 2cd74ff..2aa7b34 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatistics.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AMultipartUploaderStatistics.java
@@ -16,14 +16,17 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.fs.s3a.impl.statistics;
+package org.apache.hadoop.fs.s3a.statistics;
 
 import java.io.Closeable;
 
 /**
  * Statistics for the S3A multipart uploader.
+ * It is expected to act as the statistics source for
+ * the uploader.
  */
-public interface S3AMultipartUploaderStatistics extends Closeable {
+public interface S3AMultipartUploaderStatistics extends Closeable,
+    S3AStatisticInterface {
 
   void instantiated();
 
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AStatisticInterface.java
similarity index 69%
copy from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatistics.java
copy to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AStatisticInterface.java
index 2cd74ff..836f203 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatistics.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AStatisticInterface.java
@@ -16,24 +16,17 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.fs.s3a.impl.statistics;
+package org.apache.hadoop.fs.s3a.statistics;
 
-import java.io.Closeable;
+import org.apache.hadoop.fs.statistics.DurationTrackerFactory;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
 
 /**
- * Statistics for the S3A multipart uploader.
+ * Base Interface for S3A Statistics, as a way of managing
+ * them.
  */
-public interface S3AMultipartUploaderStatistics extends Closeable {
+public interface S3AStatisticInterface extends IOStatisticsSource,
+    DurationTrackerFactory {
 
-  void instantiated();
 
-  void uploadStarted();
-
-  void partPut(long lengthInBytes);
-
-  void uploadCompleted();
-
-  void uploadAborted();
-
-  void abortUploadsUnderPathInvoked();
 }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AStatisticsContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AStatisticsContext.java
new file mode 100644
index 0000000..27f1398
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AStatisticsContext.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.statistics;
+
+import org.apache.hadoop.fs.s3a.s3guard.MetastoreInstrumentation;
+
+/**
+ * This is the statistics context for ongoing operations in S3A.
+ */
+public interface S3AStatisticsContext extends CountersAndGauges {
+
+  /**
+   * Get the metastore instrumentation.
+   * @return an instance of the metastore statistics tracking.
+   */
+  MetastoreInstrumentation getS3GuardInstrumentation();
+
+  /**
+   * Create a stream input statistics instance.
+   * @return the new instance
+   */
+  S3AInputStreamStatistics newInputStreamStatistics();
+
+  /**
+   * Create a new instance of the committer statistics.
+   * @return a new committer statistics instance
+   */
+  CommitterStatistics newCommitterStatistics();
+
+  /**
+   * Create a stream output statistics instance.
+   * @return the new instance
+   */
+  BlockOutputStreamStatistics newOutputStreamStatistics();
+
+  /**
+   * Create a delegation token statistics instance.
+   * @return an instance of delegation token statistics
+   */
+  DelegationTokenStatistics newDelegationTokenStatistics();
+
+  /**
+   * Create a StatisticsFromAwsSdk instance.
+   * @return an instance of StatisticsFromAwsSdk
+   */
+  StatisticsFromAwsSdk newStatisticsFromAwsSdk();
+
+  /**
+   * Creaet a multipart statistics collector.
+   * @return an instance
+   */
+  S3AMultipartUploaderStatistics createMultipartUploaderStatistics();
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/StatisticTypeEnum.java
similarity index 69%
copy from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatistics.java
copy to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/StatisticTypeEnum.java
index 2cd74ff..ac7bd9b 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatistics.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/StatisticTypeEnum.java
@@ -16,24 +16,31 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.fs.s3a.impl.statistics;
-
-import java.io.Closeable;
+package org.apache.hadoop.fs.s3a.statistics;
 
 /**
- * Statistics for the S3A multipart uploader.
+ * Enum of statistic types.
  */
-public interface S3AMultipartUploaderStatistics extends Closeable {
-
-  void instantiated();
+public enum StatisticTypeEnum {
 
-  void uploadStarted();
+  /**
+   * Counter.
+   */
+  TYPE_COUNTER,
 
-  void partPut(long lengthInBytes);
+  /**
+   * Duration.
+   */
+  TYPE_DURATION,
 
-  void uploadCompleted();
+  /**
+   * Gauge.
+   */
+  TYPE_GAUGE,
 
-  void uploadAborted();
+  /**
+   * Quantile.
+   */
+  TYPE_QUANTILE,
 
-  void abortUploadsUnderPathInvoked();
 }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/StatisticsFromAwsSdk.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/StatisticsFromAwsSdk.java
new file mode 100644
index 0000000..2fd7c68
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/StatisticsFromAwsSdk.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.statistics;
+
+import java.time.Duration;
+
+import org.apache.hadoop.fs.s3a.statistics.impl.AwsStatisticsCollector;
+
+/**
+ * interface to receive statistics events from the AWS SDK
+ * by way of {@link AwsStatisticsCollector}.
+ */
+public interface StatisticsFromAwsSdk {
+
+  /**
+   * Record a number of AWS requests.
+   * @param count number of events.
+   */
+  void updateAwsRequestCount(long count);
+
+  /**
+   * Record a number of AWS request retries.
+   * @param count number of events.
+   */
+  void updateAwsRetryCount(long count);
+
+  /**
+   * Record a number of throttle exceptions received.
+   * @param count number of events.
+   */
+  void updateAwsThrottleExceptionsCount(long count);
+
+  /**
+   * Record how long a request took overall.
+   * @param duration duration of operation.
+   */
+  void noteAwsRequestTime(Duration duration);
+
+  /**
+   * Record how long a request took to execute on the
+   * client.
+   * @param duration duration of operation.
+   */
+  void noteAwsClientExecuteTime(Duration duration);
+
+  /**
+   * Record how long a request took to marshall into
+   * XML.
+   * @param duration duration of operation.
+   */
+  void noteRequestMarshallTime(Duration duration);
+
+  /**
+   * Record how long a request took to sign, including
+   * any calls to EC2 credential endpoints.
+   * @param duration duration of operation.
+   */
+  void noteRequestSigningTime(Duration duration);
+
+  /**
+   * Record how long it took to process the response.
+   * @param duration duration of operation.
+   */
+  void noteResponseProcessingTime(Duration duration);
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/AbstractS3AStatisticsSource.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/AbstractS3AStatisticsSource.java
new file mode 100644
index 0000000..6b7eb24
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/AbstractS3AStatisticsSource.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.statistics.impl;
+
+import org.apache.hadoop.fs.statistics.DurationTracker;
+import org.apache.hadoop.fs.statistics.DurationTrackerFactory;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
+
+/**
+ * Base class for implementing IOStatistics sources in the S3 module.
+ * <p>
+ * A lot of the methods are very terse, because S3AInstrumentation has
+ * verbose methods of similar names; the short ones always
+ * refer to the inner class and not any superclass method.
+ * </p>
+ */
+public abstract class AbstractS3AStatisticsSource implements
+    IOStatisticsSource, DurationTrackerFactory {
+
+  private IOStatisticsStore ioStatistics;
+
+  protected AbstractS3AStatisticsSource() {
+  }
+
+  @Override
+  public IOStatisticsStore getIOStatistics() {
+    return ioStatistics;
+  }
+
+  /**
+   * Setter.
+   * this must be called in the subclass constructor with
+   * whatever
+   * @param statistics statistics to set
+   */
+  protected void setIOStatistics(final IOStatisticsStore statistics) {
+    this.ioStatistics = statistics;
+  }
+
+  /**
+   * Increment a named counter by 1.
+   * @param name counter name
+   * @return the updated value or, if the counter is unknown: 0
+   */
+  public long incCounter(String name) {
+    return incCounter(name, 1);
+  }
+
+  /**DefaultS3ClientFactoryDefaultS3ClientFactory
+   * Increment a named counter by 1.
+   * @param name counter name
+   * @param value value to increment by
+   * @return the updated value or, if the counter is unknown: 0
+   */
+  public long incCounter(String name, long value) {
+    return ioStatistics.incrementCounter(name, value);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  public Long lookupCounterValue(final String name) {
+    return ioStatistics.counters().get(name);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  public Long lookupGaugeValue(final String name) {
+    return ioStatistics.gauges().get(name);
+  }
+
+  public long incGauge(String name, long v) {
+    return ioStatistics.incrementGauge(name, v);
+  }
+
+  public long incGauge(String name) {
+    return incGauge(name, 1);
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "AbstractS3AStatisticsSource{");
+    sb.append(ioStatistics);
+    sb.append('}');
+    return sb.toString();
+  }
+
+  @Override
+  public DurationTracker trackDuration(final String key, final long count) {
+    return getIOStatistics().trackDuration(key, count);
+  }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/AwsStatisticsCollector.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/AwsStatisticsCollector.java
new file mode 100644
index 0000000..c002a4a
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/AwsStatisticsCollector.java
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.statistics.impl;
+
+import java.time.Duration;
+import java.util.function.Consumer;
+import java.util.function.LongConsumer;
+
+import com.amazonaws.Request;
+import com.amazonaws.Response;
+import com.amazonaws.metrics.RequestMetricCollector;
+import com.amazonaws.util.TimingInfo;
+
+import org.apache.hadoop.fs.s3a.statistics.StatisticsFromAwsSdk;
+
+import static com.amazonaws.util.AWSRequestMetrics.Field.ClientExecuteTime;
+import static com.amazonaws.util.AWSRequestMetrics.Field.HttpClientRetryCount;
+import static com.amazonaws.util.AWSRequestMetrics.Field.HttpRequestTime;
+import static com.amazonaws.util.AWSRequestMetrics.Field.RequestCount;
+import static com.amazonaws.util.AWSRequestMetrics.Field.RequestMarshallTime;
+import static com.amazonaws.util.AWSRequestMetrics.Field.RequestSigningTime;
+import static com.amazonaws.util.AWSRequestMetrics.Field.ResponseProcessingTime;
+import static com.amazonaws.util.AWSRequestMetrics.Field.ThrottleException;
+
+/**
+ * Collect statistics from the AWS SDK and forward to an instance of
+ * {@link StatisticsFromAwsSdk} and thence into the S3A statistics.
+ * <p>
+ * See {@code com.facebook.presto.hive.s3.PrestoS3FileSystemMetricCollector}
+ * for the inspiration for this.
+ * <p>
+ * See {@code com.amazonaws.util.AWSRequestMetrics} for metric names.
+ */
+public class AwsStatisticsCollector extends RequestMetricCollector {
+
+  /**
+   * final destination of updates.
+   */
+  private final StatisticsFromAwsSdk collector;
+
+  /**
+   * Instantiate.
+   * @param collector final destination of updates
+   */
+  public AwsStatisticsCollector(final StatisticsFromAwsSdk collector) {
+    this.collector = collector;
+  }
+
+  /**
+   * This is the callback from the AWS SDK where metrics
+   * can be collected.
+   * @param request AWS request
+   * @param response AWS response
+   */
+  @Override
+  public void collectMetrics(
+      final Request<?> request,
+      final Response<?> response) {
+
+    TimingInfo timingInfo = request.getAWSRequestMetrics().getTimingInfo();
+
+    counter(timingInfo, HttpClientRetryCount.name(),
+        collector::updateAwsRetryCount);
+    counter(timingInfo, RequestCount.name(),
+        collector::updateAwsRequestCount);
+    counter(timingInfo, ThrottleException.name(),
+        collector::updateAwsThrottleExceptionsCount);
+
+    timing(timingInfo, ClientExecuteTime.name(),
+        collector::noteAwsClientExecuteTime);
+    timing(timingInfo, HttpRequestTime.name(),
+        collector::noteAwsRequestTime);
+    timing(timingInfo, RequestMarshallTime.name(),
+        collector::noteRequestMarshallTime);
+    timing(timingInfo, RequestSigningTime.name(),
+        collector::noteRequestSigningTime);
+    timing(timingInfo, ResponseProcessingTime.name(),
+        collector::noteResponseProcessingTime);
+  }
+
+  /**
+   * Process a timing.
+   * @param timingInfo timing info
+   * @param subMeasurementName sub measurement
+   * @param durationConsumer consumer
+   */
+  private void timing(
+      TimingInfo timingInfo,
+      String subMeasurementName,
+      Consumer<Duration> durationConsumer) {
+    TimingInfo t1 = timingInfo.getSubMeasurement(subMeasurementName);
+    if (t1 != null && t1.getTimeTakenMillisIfKnown() != null) {
+      durationConsumer.accept(Duration.ofMillis(
+          t1.getTimeTakenMillisIfKnown().longValue()));
+    }
+  }
+
+  /**
+   * Process a counter.
+   * @param timingInfo timing info
+   * @param subMeasurementName sub measurement
+   * @param consumer consumer
+   */
+  private void counter(
+      TimingInfo timingInfo,
+      String subMeasurementName,
+      LongConsumer consumer) {
+    Number n = timingInfo.getCounter(subMeasurementName);
+    if (n != null) {
+      consumer.accept(n.longValue());
+    }
+  }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/BondedS3AStatisticsContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/BondedS3AStatisticsContext.java
new file mode 100644
index 0000000..006eb24
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/BondedS3AStatisticsContext.java
@@ -0,0 +1,234 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.statistics.impl;
+
+import javax.annotation.Nullable;
+
+import java.time.Duration;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.s3a.S3AInstrumentation;
+import org.apache.hadoop.fs.s3a.Statistic;
+import org.apache.hadoop.fs.s3a.s3guard.MetastoreInstrumentation;
+import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics;
+import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics;
+import org.apache.hadoop.fs.s3a.statistics.DelegationTokenStatistics;
+import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
+import org.apache.hadoop.fs.s3a.statistics.S3AMultipartUploaderStatistics;
+import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
+import org.apache.hadoop.fs.s3a.statistics.StatisticsFromAwsSdk;
+
+/**
+ * An S3A statistics context which is bonded to a
+ * S3AInstrumentation instance -inevitably that of an S3AFileSystem
+ * instance.
+ * <p>
+ * 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.
+ * </p>
+ * <p>
+ * All operations are passed through directly to that class.
+ * </p>
+ * <p>
+ * 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.
+ * </p>
+ * <p>
+ * As a result, cross-thread IO will under-report.
+ * </p>
+ *
+ * This is addressed through the stream statistics classes
+ * only updating the stats in the close() call. Provided
+ * they are closed in the worker thread, all stats collected in
+ * helper threads will be included.
+ */
+public class BondedS3AStatisticsContext implements S3AStatisticsContext {
+
+  /** Source of statistics services. */
+  private final S3AFSStatisticsSource statisticsSource;
+
+  /**
+   * Instantiate.
+   * @param statisticsSource integration binding
+   */
+  public BondedS3AStatisticsContext(
+      final S3AFSStatisticsSource statisticsSource) {
+    this.statisticsSource = statisticsSource;
+  }
+
+
+  /**
+   * Get the instrumentation from the FS integration.
+   * @return instrumentation instance.
+   */
+  private S3AInstrumentation getInstrumentation() {
+    return statisticsSource.getInstrumentation();
+  }
+
+  /**
+   * The filesystem statistics: know this is thread-local.
+   * @return FS statistics.
+   */
+  private FileSystem.Statistics getInstanceStatistics() {
+    return statisticsSource.getInstanceStatistics();
+  }
+
+  /**
+   * Get a MetastoreInstrumentation getInstrumentation() instance for this
+   * context.
+   * @return the S3Guard getInstrumentation() point.
+   */
+  @Override
+  public MetastoreInstrumentation getS3GuardInstrumentation() {
+    return getInstrumentation().getS3GuardInstrumentation();
+  }
+
+  /**
+   * Create a stream input statistics instance.
+   * The FileSystem.Statistics instance of the {@link #statisticsSource}
+   * is used as the reference to FileSystem statistics to update
+   * @return the new instance
+   */
+  @Override
+  public S3AInputStreamStatistics newInputStreamStatistics() {
+    return getInstrumentation().newInputStreamStatistics(
+        statisticsSource.getInstanceStatistics());
+  }
+
+  /**
+   * Create a new instance of the committer statistics.
+   * @return a new committer statistics instance
+   */
+  @Override
+  public CommitterStatistics newCommitterStatistics() {
+    return getInstrumentation().newCommitterStatistics();
+  }
+
+  /**
+   * Create a stream output statistics instance.
+   * @return the new instance
+   */
+  @Override
+  public BlockOutputStreamStatistics newOutputStreamStatistics() {
+    return getInstrumentation()
+        .newOutputStreamStatistics(getInstanceStatistics());
+  }
+
+  /**
+   * Increment a specific counter.
+   * <p>
+   * 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.
+   * <p>
+   * 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.
+   * <p>
+   * No-op if not defined.
+   * @param op operation
+   * @param count increment value
+   * @throws ClassCastException if the metric is of the wrong type
+   */
+  @Override
+  public void decrementGauge(Statistic op, long count) {
+    getInstrumentation().decrementGauge(op, count);
+  }
+
+  /**
+   * Add a value to a quantiles statistic. No-op if the quantile
+   * isn't found.
+   * @param op operation to look up.
+   * @param value value to add.
+   * @throws ClassCastException if the metric is not a Quantiles.
+   */
+  @Override
+  public void addValueToQuantiles(Statistic op, long value) {
+    getInstrumentation().addValueToQuantiles(op, value);
+  }
+
+  @Override
+  public void recordDuration(final Statistic op,
+      final boolean success,
+      final Duration duration) {
+    getInstrumentation().recordDuration(op, success, duration);
+  }
+
+  /**
+   * Create a delegation token statistics instance.
+   * @return an instance of delegation token statistics
+   */
+  @Override
+  public DelegationTokenStatistics newDelegationTokenStatistics() {
+    return getInstrumentation().newDelegationTokenStatistics();
+  }
+
+  @Override
+  public StatisticsFromAwsSdk newStatisticsFromAwsSdk() {
+    return new StatisticsFromAwsSdkImpl(getInstrumentation());
+  }
+
+  @Override
+  public S3AMultipartUploaderStatistics createMultipartUploaderStatistics() {
+    return new S3AMultipartUploaderStatisticsImpl(this::incrementCounter);
+  }
+
+  /**
+   * This is the interface which an integration source must implement
+   * for the integration.
+   * Note that the FileSystem.statistics field may be null for a class;
+   */
+  public interface S3AFSStatisticsSource {
+
+    /**
+     * Get the S3A Instrumentation.
+     * @return a non-null instrumentation instance
+     */
+    S3AInstrumentation getInstrumentation();
+
+    /**
+     * Get the statistics of the FS instance, shared across all threads.
+     * @return filesystem statistics
+     */
+    @Nullable
+    FileSystem.Statistics getInstanceStatistics();
+
+  }
+}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestLocatedFileStatusFetcher.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/CountingChangeTracker.java
similarity index 53%
copy from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestLocatedFileStatusFetcher.java
copy to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/CountingChangeTracker.java
index bd6bf2f..1815774 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestLocatedFileStatusFetcher.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/CountingChangeTracker.java
@@ -16,25 +16,38 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.fs.s3a;
+package org.apache.hadoop.fs.s3a.statistics.impl;
 
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.fs.s3a.statistics.ChangeTrackerStatistics;
 
 /**
- * Test the LocatedFileStatusFetcher can do.
- * This is related to HADOOP-16458.
- * There's basic tests in ITestS3AFSMainOperations; this
- * is see if we can create better corner cases.
+ * A change tracker which increments an atomic long.
  */
-public class ITestLocatedFileStatusFetcher extends AbstractS3ATestBase {
+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;
+  }
 
-  private static final Logger LOG =
-      LoggerFactory.getLogger(ITestLocatedFileStatusFetcher.class);
+  public CountingChangeTracker() {
+    this(new AtomicLong());
+  }
 
-  @Test
-  public void testGlobScan() throws Throwable {
+  @Override
+  public void versionMismatchError() {
+    counter.incrementAndGet();
+  }
 
+  @Override
+  public long getVersionMismatches() {
+    return counter.get();
   }
 }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/EmptyS3AStatisticsContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/EmptyS3AStatisticsContext.java
new file mode 100644
index 0000000..c8cd805
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/EmptyS3AStatisticsContext.java
@@ -0,0 +1,597 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.statistics.impl;
+
+import java.io.IOException;
+import java.time.Duration;
+
+import org.apache.hadoop.fs.s3a.Statistic;
+import org.apache.hadoop.fs.s3a.s3guard.MetastoreInstrumentation;
+import org.apache.hadoop.fs.s3a.s3guard.MetastoreInstrumentationImpl;
+import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics;
+import org.apache.hadoop.fs.s3a.statistics.ChangeTrackerStatistics;
+import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics;
+import org.apache.hadoop.fs.s3a.statistics.DelegationTokenStatistics;
+import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
+import org.apache.hadoop.fs.s3a.statistics.S3AMultipartUploaderStatistics;
+import org.apache.hadoop.fs.s3a.statistics.S3AStatisticInterface;
+import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
+import org.apache.hadoop.fs.s3a.statistics.StatisticsFromAwsSdk;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.DurationTracker;
+
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.emptyStatistics;
+import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.stubDurationTracker;
+
+/**
+ * Special statistics context, all of whose context operations are no-ops.
+ * All statistics instances it returns are also empty.
+ * <p>
+ * This class is here primarily to aid in testing, but it also allows for
+ * classes to require a non-empty statistics context in their constructor -yet
+ * still be instantiated without one bound to any filesystem.
+ */
+public final class EmptyS3AStatisticsContext implements S3AStatisticsContext {
+
+  public static final MetastoreInstrumentation
+      METASTORE_INSTRUMENTATION = new MetastoreInstrumentationImpl();
+
+  public static final S3AInputStreamStatistics
+      EMPTY_INPUT_STREAM_STATISTICS = new EmptyInputStreamStatistics();
+
+  public static final CommitterStatistics
+      EMPTY_COMMITTER_STATISTICS = new EmptyCommitterStatistics();
+
+  @SuppressWarnings("IOResourceOpenedButNotSafelyClosed")
+  public static final BlockOutputStreamStatistics
+      EMPTY_BLOCK_OUTPUT_STREAM_STATISTICS
+      = new EmptyBlockOutputStreamStatistics();
+
+  public static final DelegationTokenStatistics
+      EMPTY_DELEGATION_TOKEN_STATISTICS = new EmptyDelegationTokenStatistics();
+
+  public static final StatisticsFromAwsSdk
+      EMPTY_STATISTICS_FROM_AWS_SDK = new EmptyStatisticsFromAwsSdk();
+
+  @Override
+  public MetastoreInstrumentation getS3GuardInstrumentation() {
+    return METASTORE_INSTRUMENTATION;
+  }
+
+  @Override
+  public S3AInputStreamStatistics newInputStreamStatistics() {
+    return EMPTY_INPUT_STREAM_STATISTICS;
+  }
+
+  @Override
+  public CommitterStatistics newCommitterStatistics() {
+    return EMPTY_COMMITTER_STATISTICS;
+  }
+
+  @Override
+  public BlockOutputStreamStatistics newOutputStreamStatistics() {
+    return EMPTY_BLOCK_OUTPUT_STREAM_STATISTICS;
+  }
+
+  @Override
+  public DelegationTokenStatistics newDelegationTokenStatistics() {
+    return EMPTY_DELEGATION_TOKEN_STATISTICS;
+  }
+
+  @Override
+  public StatisticsFromAwsSdk newStatisticsFromAwsSdk() {
+    return EMPTY_STATISTICS_FROM_AWS_SDK;
+  }
+
+  @Override
+  public S3AMultipartUploaderStatistics createMultipartUploaderStatistics() {
+    return new EmptyMultipartUploaderStatistics();
+  }
+
+  @Override
+  public void incrementCounter(final Statistic op, final long count) {
+
+  }
+
+  @Override
+  public void incrementGauge(final Statistic op, final long count) {
+
+  }
+
+  @Override
+  public void decrementGauge(final Statistic op, final long count) {
+
+  }
+
+  @Override
+  public void addValueToQuantiles(final Statistic op, final long value) {
+
+  }
+
+  @Override
+  public void recordDuration(final Statistic op,
+      final boolean success,
+      final Duration duration) {
+
+  }
+
+  /**
+   * Base class for all the empty implementations.
+   */
+  private static class EmptyS3AStatisticImpl implements
+      S3AStatisticInterface {
+
+    /**
+     * Always return the stub duration tracker.
+     * @param key statistic key prefix
+     * @param count  #of times to increment the matching counter in this
+     * operation.
+     * @return stub tracker.
+     */
+    public DurationTracker trackDuration(String key, long count) {
+      return stubDurationTracker();
+    }
+  }
+
+  /**
+   * Input Stream statistics callbacks.
+   */
+  private static final class EmptyInputStreamStatistics
+      extends EmptyS3AStatisticImpl
+      implements S3AInputStreamStatistics {
+
+    @Override
+    public void seekBackwards(final long negativeOffset) {
+
+    }
+
+    @Override
+    public void seekForwards(final long skipped,
+        final long bytesReadInSeek) {
+
+    }
+
+    @Override
+    public long streamOpened() {
+      return 0;
+    }
+
+    @Override
+    public void streamClose(final boolean abortedConnection,
+        final long remainingInCurrentRequest) {
+
+    }
+
+    @Override
+    public void readException() {
+
+    }
+
+    @Override
+    public void bytesRead(final long bytes) {
+
+    }
+
+    @Override
+    public void readOperationStarted(final long pos, final long len) {
+
+    }
+
+    @Override
+    public void readFullyOperationStarted(final long pos, final long len) {
+
+    }
+
+    @Override
+    public void readOperationCompleted(final int requested, final int actual) {
+
+    }
+
+    @Override
+    public void close() {
+
+    }
+
+    @Override
+    public void inputPolicySet(final int updatedPolicy) {
+
+    }
+
+    @Override
+    public void unbuffered() {
+
+    }
+
+    /**
+     * Return an IO statistics instance.
+     * @return an empty IO statistics instance.
+     */
+    @Override
+    public IOStatistics getIOStatistics() {
+      return emptyStatistics();
+    }
+
+    @Override
+    public long getCloseOperations() {
+      return 0;
+    }
+
+    @Override
+    public long getClosed() {
+      return 0;
+    }
+
+    @Override
+    public long getAborted() {
+      return 0;
+    }
+
+    @Override
+    public long getForwardSeekOperations() {
+      return 0;
+    }
+
+    @Override
+    public long getBackwardSeekOperations() {
+      return 0;
+    }
+
+    @Override
+    public long getBytesRead() {
+      return 0;
+    }
+
+    @Override
+    public long getTotalBytesRead() {
+      return 0;
+    }
+
+    @Override
+    public long getBytesSkippedOnSeek() {
+      return 0;
+    }
+
+    @Override
+    public long getBytesBackwardsOnSeek() {
+      return 0;
+    }
+
+    @Override
+    public long getBytesReadInClose() {
+      return 0;
+    }
+
+    @Override
+    public long getBytesDiscardedInAbort() {
+      return 0;
+    }
+
+    @Override
+    public long getOpenOperations() {
+      return 0;
+    }
+
+    @Override
+    public long getSeekOperations() {
+      return 0;
+    }
+
+    @Override
+    public long getReadExceptions() {
+      return 0;
+    }
+
+    @Override
+    public long getReadOperations() {
+      return 0;
+    }
+
+    @Override
+    public long getReadFullyOperations() {
+      return 0;
+    }
+
+    @Override
+    public long getReadsIncomplete() {
+      return 0;
+    }
+
+    @Override
+    public long getPolicySetCount() {
+      return 0;
+    }
+
+    @Override
+    public long getVersionMismatches() {
+      return 0;
+    }
+
+    @Override
+    public long getInputPolicy() {
+      return 0;
+    }
+
+    @Override
+    public Long lookupCounterValue(final String name) {
+      return 0L;
+    }
+
+    @Override
+    public Long lookupGaugeValue(final String name) {
+      return 0L;
+    }
+
+    @Override
+    public ChangeTrackerStatistics getChangeTrackerStatistics() {
+      return new CountingChangeTracker();
+    }
+
+    @Override
+    public DurationTracker initiateGetRequest() {
+      return stubDurationTracker();
+    }
+
+  }
+
+  /**
+   * Committer statistics.
+   */
+  private static final class EmptyCommitterStatistics
+      extends EmptyS3AStatisticImpl
+      implements CommitterStatistics {
+
+    @Override
+    public void commitCreated() {
+    }
+
+    @Override
+    public void commitUploaded(final long size) {
+    }
+
+    @Override
+    public void commitCompleted(final long size) {
+    }
+
+    @Override
+    public void commitAborted() {
+    }
+
+    @Override
+    public void commitReverted() {
+    }
+
+    @Override
+    public void commitFailed() {
+    }
+
+    @Override
+    public void taskCompleted(final boolean success) {
+    }
+
+    @Override
+    public void jobCompleted(final boolean success) {
+    }
+  }
+
+  private static final class EmptyBlockOutputStreamStatistics
+      extends EmptyS3AStatisticImpl
+      implements BlockOutputStreamStatistics {
+
+    @Override
+    public void blockUploadQueued(final int blockSize) {
+    }
+
+    @Override
+    public void blockUploadStarted(final Duration timeInQueue,
+        final int blockSize) {
+    }
+
+    @Override
+    public void blockUploadCompleted(final Duration timeSinceUploadStarted,
+        final int blockSize) {
+    }
+
+    @Override
+    public void blockUploadFailed(final Duration timeSinceUploadStarted,
+        final int blockSize) {
+    }
+
+    @Override
+    public void bytesTransferred(final long byteCount) {
+    }
+
+    @Override
+    public void exceptionInMultipartComplete(final int count) {
+
+    }
+
+    @Override
+    public void exceptionInMultipartAbort() {
+    }
+
+    @Override
+    public long getBytesPendingUpload() {
+      return 0;
+    }
+
+    @Override
+    public void commitUploaded(final long size) {
+
+    }
+
+    @Override
+    public int getBlocksAllocated() {
+      return 0;
+    }
+
+    @Override
+    public int getBlocksReleased() {
+      return 0;
+    }
+
+    @Override
+    public int getBlocksActivelyAllocated() {
+      return 0;
+    }
+
+    @Override
+    public IOStatistics getIOStatistics() {
+      return emptyStatistics();
+    }
+
+    @Override
+    public void blockAllocated() {
+    }
+
+    @Override
+    public void blockReleased() {
+    }
+
+    @Override
+    public void writeBytes(final long count) {
+    }
+
+    @Override
+    public long getBytesWritten() {
+      return 0;
+    }
+
+    @Override
+    public Long lookupCounterValue(final String name) {
+      return 0L;
+    }
+
+    @Override
+    public Long lookupGaugeValue(final String name) {
+      return 0L;
+    }
+
+    @Override
+    public void close() throws IOException {
+    }
+
+  }
+
+  /**
+   * Delegation Token Statistics.
+   */
+  private static final class EmptyDelegationTokenStatistics
+      extends EmptyS3AStatisticImpl
+      implements DelegationTokenStatistics {
+
+    @Override
+    public void tokenIssued() {
+
+    }
+  }
+
+  /**
+   * AWS SDK Callbacks.
+   */
+  private static final class EmptyStatisticsFromAwsSdk
+      implements StatisticsFromAwsSdk {
+
+    @Override
+    public void updateAwsRequestCount(final long longValue) {
+
+    }
+
+    @Override
+    public void updateAwsRetryCount(final long longValue) {
+
+    }
+
+    @Override
+    public void updateAwsThrottleExceptionsCount(final long longValue) {
+
+    }
+
+    @Override
+    public void noteAwsRequestTime(final Duration ofMillis) {
+
+    }
+
+    @Override
+    public void noteAwsClientExecuteTime(final Duration ofMillis) {
+
+    }
+
+    @Override
+    public void noteRequestMarshallTime(final Duration duration) {
+
+    }
+
+    @Override
+    public void noteRequestSigningTime(final Duration duration) {
+
+    }
+
+    @Override
+    public void noteResponseProcessingTime(final Duration duration) {
+
+    }
+  }
+
+  /**
+   * Multipart Uploader.
+   */
+  public static final class EmptyMultipartUploaderStatistics
+      implements S3AMultipartUploaderStatistics {
+
+    @Override
+    public void instantiated() {
+
+    }
+
+    @Override
+    public void uploadStarted() {
+
+    }
+
+    @Override
+    public void partPut(final long lengthInBytes) {
+
+    }
+
+    @Override
+    public void uploadCompleted() {
+
+    }
+
+    @Override
+    public void uploadAborted() {
+
+    }
+
+    @Override
+    public void abortUploadsUnderPathInvoked() {
+
+    }
+
+    @Override
+    public void close() throws IOException {
+
+    }
+
+    @Override
+    public DurationTracker trackDuration(final String key, final long count) {
+      return stubDurationTracker();
+    }
+  }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatisticsImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/S3AMultipartUploaderStatisticsImpl.java
similarity index 60%
rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatisticsImpl.java
rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/S3AMultipartUploaderStatisticsImpl.java
index 70e4785..7b6d559 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatisticsImpl.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/S3AMultipartUploaderStatisticsImpl.java
@@ -16,30 +16,42 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.fs.s3a.impl.statistics;
+package org.apache.hadoop.fs.s3a.statistics.impl;
 
 
 import java.io.IOException;
+import java.util.Objects;
 import java.util.function.BiConsumer;
 
 import org.apache.hadoop.fs.s3a.Statistic;
+import org.apache.hadoop.fs.s3a.statistics.S3AMultipartUploaderStatistics;
+import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
 
-import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_ABORT_UNDER_PATH_INVOKED;
-import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_INSTANTIATED;
-import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_PART_PUT;
-import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_PART_PUT_BYTES;
 import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_ABORTED;
+import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_ABORT_UNDER_PATH_INVOKED;
 import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_COMPLETED;
+import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_INSTANTIATED;
+import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_PART_PUT;
+import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_PART_PUT_BYTES;
 import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_STARTED;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore;
 
 /**
  * Implementation of the uploader statistics.
+ * <p>
  * This takes a function to update some counter and will update
  * this value when things change, so it can be bonded to arbitrary
  * statistic collectors.
+ * </p>
+ * <p>
+ * Internally it builds a map of the relevant multipart statistics,
+ * increments as appropriate and serves this data back through
+ * the {@code IOStatisticsSource} API.
+ * </p>
  */
-public final class S3AMultipartUploaderStatisticsImpl implements
-    S3AMultipartUploaderStatistics {
+public final class S3AMultipartUploaderStatisticsImpl
+    extends AbstractS3AStatisticsSource
+    implements S3AMultipartUploaderStatistics {
 
   /**
    * The operation to increment a counter/statistic by a value.
@@ -53,16 +65,28 @@ public final class S3AMultipartUploaderStatisticsImpl implements
    */
   public S3AMultipartUploaderStatisticsImpl(
       final BiConsumer<Statistic, Long> incrementCallback) {
-    this.incrementCallback = incrementCallback;
+    this.incrementCallback = Objects.requireNonNull(incrementCallback);
+    IOStatisticsStore st = iostatisticsStore()
+        .withCounters(
+            MULTIPART_UPLOAD_INSTANTIATED.getSymbol(),
+            MULTIPART_UPLOAD_PART_PUT.getSymbol(),
+            MULTIPART_UPLOAD_PART_PUT_BYTES.getSymbol(),
+            MULTIPART_UPLOAD_ABORTED.getSymbol(),
+            MULTIPART_UPLOAD_ABORT_UNDER_PATH_INVOKED.getSymbol(),
+            MULTIPART_UPLOAD_COMPLETED.getSymbol(),
+            MULTIPART_UPLOAD_STARTED.getSymbol())
+        .build();
+    setIOStatistics(st);
   }
 
   private void inc(Statistic op, long count) {
     incrementCallback.accept(op, count);
+    incCounter(op.getSymbol(), count);
   }
 
   @Override
   public void instantiated() {
-    inc(MULTIPART_INSTANTIATED, 1);
+    inc(MULTIPART_UPLOAD_INSTANTIATED, 1);
   }
 
   @Override
@@ -72,8 +96,8 @@ public final class S3AMultipartUploaderStatisticsImpl implements
 
   @Override
   public void partPut(final long lengthInBytes) {
-    inc(MULTIPART_PART_PUT, 1);
-    inc(MULTIPART_PART_PUT_BYTES, lengthInBytes);
+    inc(MULTIPART_UPLOAD_PART_PUT, 1);
+    inc(MULTIPART_UPLOAD_PART_PUT_BYTES, lengthInBytes);
   }
 
   @Override
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/StatisticsFromAwsSdkImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/StatisticsFromAwsSdkImpl.java
new file mode 100644
index 0000000..48b0b2b
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/StatisticsFromAwsSdkImpl.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.statistics.impl;
+
+import java.time.Duration;
+
+import org.apache.hadoop.fs.s3a.statistics.CountersAndGauges;
+import org.apache.hadoop.fs.s3a.statistics.StatisticsFromAwsSdk;
+
+import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_REQUEST;
+import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_RETRY;
+import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_THROTTLED;
+import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_THROTTLE_RATE;
+
+/**
+ * Hook up AWS SDK Statistics to the S3 counters.
+ * <p>
+ * Durations are not currently being used; that could be
+ * changed in future once an effective strategy for reporting
+ * them is determined.
+ */
+public final class StatisticsFromAwsSdkImpl implements
+    StatisticsFromAwsSdk {
+
+  private final CountersAndGauges countersAndGauges;
+
+  public StatisticsFromAwsSdkImpl(
+      final CountersAndGauges countersAndGauges) {
+    this.countersAndGauges = countersAndGauges;
+  }
+
+  @Override
+  public void updateAwsRequestCount(final long count) {
+    countersAndGauges.incrementCounter(STORE_IO_REQUEST, count);
+  }
+
+  @Override
+  public void updateAwsRetryCount(final long count) {
+    countersAndGauges.incrementCounter(STORE_IO_RETRY, count);
+  }
+
+  @Override
+  public void updateAwsThrottleExceptionsCount(final long count) {
+    countersAndGauges.incrementCounter(STORE_IO_THROTTLED, count);
+    countersAndGauges.addValueToQuantiles(STORE_IO_THROTTLE_RATE, count);
+  }
+
+  @Override
+  public void noteAwsRequestTime(final Duration duration) {
+
+  }
+
+  @Override
+  public void noteAwsClientExecuteTime(final Duration duration) {
+
+  }
+
+  @Override
+  public void noteRequestMarshallTime(final Duration duration) {
+
+  }
+
+  @Override
+  public void noteRequestSigningTime(final Duration duration) {
+
+  }
+
+  @Override
+  public void noteResponseProcessingTime(final Duration duration) {
+
+  }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/package-info.java
similarity index 68%
copy from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatistics.java
copy to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/package-info.java
index 2cd74ff..6b94ac7 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatistics.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/package-info.java
@@ -16,24 +16,14 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.fs.s3a.impl.statistics;
-
-import java.io.Closeable;
-
 /**
- * Statistics for the S3A multipart uploader.
+ * Statistics collection for the S3A connector: implementation.
+ * Not for use by anything outside the hadoop-aws source tree.
  */
-public interface S3AMultipartUploaderStatistics extends Closeable {
-
-  void instantiated();
-
-  void uploadStarted();
-
-  void partPut(long lengthInBytes);
-
-  void uploadCompleted();
 
-  void uploadAborted();
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.fs.s3a.statistics.impl;
 
-  void abortUploadsUnderPathInvoked();
-}
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/package-info.java
similarity index 61%
rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatistics.java
rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/package-info.java
index 2cd74ff..b74bc3e 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AMultipartUploaderStatistics.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/package-info.java
@@ -16,24 +16,16 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.fs.s3a.impl.statistics;
-
-import java.io.Closeable;
-
 /**
- * Statistics for the S3A multipart uploader.
+ * Statistics collection for the S3A connector: interfaces.
+ * This is private, though there's a risk that some extension
+ * points (delegation tokens?) may need access to the internal
+ * API. Hence the split packaging...with a java 9 module, the
+ * implementation classes would be declared internal.
  */
-public interface S3AMultipartUploaderStatistics extends Closeable {
-
-  void instantiated();
-
-  void uploadStarted();
-
-  void partPut(long lengthInBytes);
-
-  void uploadCompleted();
-
-  void uploadAborted();
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.fs.s3a.statistics;
 
-  void abortUploadsUnderPathInvoked();
-}
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java
index c7d7fda3..6d386f2 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java
@@ -65,6 +65,7 @@ import static org.apache.hadoop.fs.s3a.Constants.AUTHORITATIVE_PATH;
 import static org.apache.hadoop.fs.s3a.Constants.BULK_DELETE_PAGE_SIZE;
 import static org.apache.hadoop.fs.s3a.Constants.BULK_DELETE_PAGE_SIZE_DEFAULT;
 import static org.apache.hadoop.fs.s3a.Invoker.once;
+import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsSourceToString;
 import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_INTERRUPTED;
 import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_NOT_ACCEPTABLE;
 import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_NOT_FOUND;
@@ -672,6 +673,7 @@ public final class MarkerTool extends S3GuardTool {
       final int limit) throws IOException {
 
     int count = 0;
+    boolean result = true;
     RemoteIterator<S3AFileStatus> listing = operations
         .listObjects(path, storeContext.pathToKey(path));
     while (listing.hasNext()) {
@@ -700,10 +702,16 @@ public final class MarkerTool extends S3GuardTool {
       if (limit > 0 && count >= limit) {
         println(out, "Limit of scan reached - %,d object%s",
             limit, suffix(limit));
-        return false;
+        result = false;
+        break;
       }
     }
-    return true;
+    LOG.debug("Listing summary {}", listing);
+    if (verbose) {
+      println(out, "%nListing statistics:%n  %s%n",
+          ioStatisticsSourceToString(listing));
+    }
+    return result;
   }
 
   /**
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java
index 740f256..ae54dfe 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java
@@ -79,15 +79,15 @@ public class ITestS3AContractDistCp extends AbstractContractDistCpTest {
 
   @Override
   public void testDirectWrite() throws Exception {
-    resetStorageStatistics();
+    final long renames = getRenameOperationCount();
     super.testDirectWrite();
     assertEquals("Expected no renames for a direct write distcp", 0L,
-        getRenameOperationCount());
+        getRenameOperationCount() - renames);
   }
 
   @Override
   public void testNonDirectWrite() throws Exception {
-    resetStorageStatistics();
+    final long renames = getRenameOperationCount();
     try {
       super.testNonDirectWrite();
     } catch (FileNotFoundException e) {
@@ -96,11 +96,7 @@ public class ITestS3AContractDistCp extends AbstractContractDistCpTest {
       // S3Guard is not enabled
     }
     assertEquals("Expected 2 renames for a non-direct write distcp", 2L,
-        getRenameOperationCount());
-  }
-
-  private void resetStorageStatistics() {
-    getFileSystem().getStorageStatistics().reset();
+        getRenameOperationCount() - renames);
   }
 
   private long getRenameOperationCount() {
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java
index 73a503a..b1ea45f 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java
@@ -26,7 +26,10 @@ import org.apache.hadoop.fs.contract.AbstractFSContractTestBase;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.contract.s3a.S3AContract;
 import org.apache.hadoop.fs.s3a.tools.MarkerTool;
+import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
 import org.apache.hadoop.io.IOUtils;
+
+import org.junit.AfterClass;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -39,6 +42,8 @@ import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestDynamoTablePrefix;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestPropertyBool;
 import static org.apache.hadoop.fs.s3a.S3AUtils.E_FS_CLOSED;
 import static org.apache.hadoop.fs.s3a.tools.MarkerTool.UNLIMITED_LISTING;
+import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString;
+import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.snapshotIOStatistics;
 
 /**
  * An extension of the contract test base set up for S3A tests.
@@ -48,6 +53,12 @@ public abstract class AbstractS3ATestBase extends AbstractFSContractTestBase
   protected static final Logger LOG =
       LoggerFactory.getLogger(AbstractS3ATestBase.class);
 
+  /**
+   * FileSystem statistics are collected across every test case.
+   */
+  protected static final IOStatisticsSnapshot FILESYSTEM_IOSTATS =
+      snapshotIOStatistics();
+
   @Override
   protected AbstractFSContract createContract(Configuration conf) {
     return new S3AContract(conf, false);
@@ -73,11 +84,23 @@ public abstract class AbstractS3ATestBase extends AbstractFSContractTestBase
     maybeAuditTestPath();
 
     super.teardown();
+    if (getFileSystem() != null) {
+      FILESYSTEM_IOSTATS.aggregate(getFileSystem().getIOStatistics());
+    }
     describe("closing file system");
     IOUtils.closeStream(getFileSystem());
   }
 
   /**
+   * Dump the filesystem statistics after the class.
+   */
+  @AfterClass
+  public static void dumpFileSystemIOStatistics() {
+    LOG.info("Aggregate FileSystem Statistics {}",
+        ioStatisticsToPrettyString(FILESYSTEM_IOSTATS));
+  }
+
+  /**
    * Audit the FS under {@link #methodPath()} if
    * the test option {@link #DIRECTORY_MARKER_AUDIT} is
    * true.
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestLocatedFileStatusFetcher.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestLocatedFileStatusFetcher.java
index bd6bf2f..5b6e634 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestLocatedFileStatusFetcher.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestLocatedFileStatusFetcher.java
@@ -18,23 +18,287 @@
 
 package org.apache.hadoop.fs.s3a;
 
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collection;
+
+import org.assertj.core.api.Assertions;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.fs.s3a.auth.ITestRestrictedReadAccess;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.mapred.LocatedFileStatusFetcher;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
+import static org.apache.hadoop.fs.s3a.Constants.AUTHORITATIVE_PATH;
+import static org.apache.hadoop.fs.s3a.Constants.METADATASTORE_AUTHORITATIVE;
+import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBucketOverrides;
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticCounter;
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.extractStatistics;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_LIST_REQUEST;
+import static org.apache.hadoop.mapreduce.lib.input.FileInputFormat.LIST_STATUS_NUM_THREADS;
+
 /**
  * Test the LocatedFileStatusFetcher can do.
  * This is related to HADOOP-16458.
  * There's basic tests in ITestS3AFSMainOperations; this
  * is see if we can create better corner cases.
+ * <p></p>
+ * Much of the class is based on tests in {@link ITestRestrictedReadAccess},
+ * but whereas that tests failure paths, this looks at the performance
+ * of successful invocations.
  */
+@RunWith(Parameterized.class)
 public class ITestLocatedFileStatusFetcher extends AbstractS3ATestBase {
 
   private static final Logger LOG =
       LoggerFactory.getLogger(ITestLocatedFileStatusFetcher.class);
 
+
+  /**
+   * Parameterization.
+   */
+  @Parameterized.Parameters(name = "{0}")
+  public static Collection<Object[]> params() {
+    return Arrays.asList(new Object[][]{
+        {"raw", false},
+        {"nonauth", true}
+    });
+  }
+
+  /** Filter to select everything. */
+  private static final PathFilter EVERYTHING = t -> true;
+
+  /** Filter to select .txt files. */
+  private static final PathFilter TEXT_FILE =
+      path -> path.toUri().toString().endsWith(".txt");
+
+  /** The same path filter used in FileInputFormat. */
+  private static final PathFilter HIDDEN_FILE_FILTER =
+      (p) -> {
+        String n = p.getName();
+        return !n.startsWith("_") && !n.startsWith(".");
+      };
+
+  /**
+   * Text found in LocatedFileStatusFetcher exception when the glob
+   * returned "null".
+   */
+  private static final String DOES_NOT_EXIST = "does not exist";
+
+  /**
+   * Text found in LocatedFileStatusFetcher exception when
+   * the glob returned an empty list.
+   */
+  private static final String MATCHES_0_FILES = "matches 0 files";
+
+  /**
+   * Text used in files.
+   */
+  public static final byte[] HELLO = "hello".getBytes(StandardCharsets.UTF_8);
+
+  /**
+   * How many list calls are expected in a run which collects them: {@value}.
+   */
+  private static final int EXPECTED_LIST_COUNT = 4;
+
+  private final String name;
+
+  private final boolean s3guard;
+
+  private Path basePath;
+
+  private Path emptyDir;
+
+  private Path emptyFile;
+
+  private Path subDir;
+
+  private Path subdirFile;
+
+  private Path subDir2;
+
+  private Path subdir2File1;
+
+  private Path subdir2File2;
+
+  private Configuration listConfig;
+
+  public ITestLocatedFileStatusFetcher(final String name,
+      final boolean s3guard) {
+    this.name = name;
+    this.s3guard = s3guard;
+  }
+
+  @Override
+  public Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    String bucketName = getTestBucketName(conf);
+
+    removeBaseAndBucketOverrides(bucketName, conf,
+        METADATASTORE_AUTHORITATIVE,
+        AUTHORITATIVE_PATH);
+    removeBucketOverrides(bucketName, conf,
+        S3_METADATA_STORE_IMPL);
+    if (!s3guard) {
+      removeBaseAndBucketOverrides(bucketName, conf,
+          S3_METADATA_STORE_IMPL);
+    }
+    conf.setBoolean(METADATASTORE_AUTHORITATIVE, false);
+    disableFilesystemCaching(conf);
+    return conf;
+  }
+  @Override
+  public void setup() throws Exception {
+    super.setup();
+    S3AFileSystem fs
+        = getFileSystem();
+    // avoiding the parameterization to steer clear of accidentally creating
+    // patterns; a timestamp is used to ensure tombstones from previous runs
+    // do not interfere
+    basePath = path("ITestLocatedFileStatusFetcher-" + name
+        + "-" + System.currentTimeMillis() / 1000);
+
+    // define the paths and create them.
+    describe("Creating test directories and files");
+
+    // an empty directory directory under the noReadDir
+    emptyDir = new Path(basePath, "emptyDir");
+    fs.mkdirs(emptyDir);
+
+    // an empty file directory under the noReadDir
+    emptyFile = new Path(basePath, "emptyFile.txt");
+    touch(fs, emptyFile);
+
+    // a subdirectory
+    subDir = new Path(basePath, "subDir");
+
+    // and a file in that subdirectory
+    subdirFile = new Path(subDir, "subdirFile.txt");
+    createFile(fs, subdirFile, true, HELLO);
+    subDir2 = new Path(subDir, "subDir2");
+    subdir2File1 = new Path(subDir2, "subdir2File1.txt");
+    subdir2File2 = new Path(subDir2, "subdir2File2.txt");
+    createFile(fs, subdir2File1, true, HELLO);
+    createFile(fs, subdir2File2, true, HELLO);
+    listConfig = new Configuration(getConfiguration());
+  }
+
+
+  /**
+   * Assert that the fetcher stats logs the expected number of calls.
+   * @param fetcher fetcher
+   * @param expectedListCount expected number of list calls
+   */
+  private void assertListCount(final LocatedFileStatusFetcher fetcher,
+      final int expectedListCount) {
+    IOStatistics iostats = extractStatistics(fetcher);
+    LOG.info("Statistics of fetcher: {}", iostats);
+    assertThatStatisticCounter(iostats,
+        OBJECT_LIST_REQUEST)
+        .describedAs("stats of %s", iostats)
+        .isEqualTo(expectedListCount);
+  }
+
+  /**
+   * Run a located file status fetcher against the directory tree.
+   */
   @Test
-  public void testGlobScan() throws Throwable {
+  public void testSingleThreadedLocatedFileStatus() throws Throwable {
+
+    describe("LocatedFileStatusFetcher operations");
+    // use the same filter as FileInputFormat; single thread.
 
+    listConfig.setInt(LIST_STATUS_NUM_THREADS, 1);
+    LocatedFileStatusFetcher fetcher =
+        new LocatedFileStatusFetcher(
+            listConfig,
+            new Path[]{basePath},
+            true,
+            HIDDEN_FILE_FILTER,
+            true);
+    Iterable<FileStatus> stats = fetcher.getFileStatuses();
+    Assertions.assertThat(stats)
+        .describedAs("result of located scan")
+        .flatExtracting(FileStatus::getPath)
+        .containsExactlyInAnyOrder(
+            emptyFile,
+            subdirFile,
+            subdir2File1,
+            subdir2File2);
+    assertListCount(fetcher, EXPECTED_LIST_COUNT);
   }
+
+  /**
+   * Run a located file status fetcher against the directory tree.
+   */
+  @Test
+  public void testLocatedFileStatusFourThreads() throws Throwable {
+
+    // four threads and the text filter.
+    int threads = 4;
+    describe("LocatedFileStatusFetcher with %d", threads);
+    listConfig.setInt(LIST_STATUS_NUM_THREADS, threads);
+    LocatedFileStatusFetcher fetcher =
+        new LocatedFileStatusFetcher(
+            listConfig,
+            new Path[]{basePath},
+            true,
+            EVERYTHING,
+            true);
+    Iterable<FileStatus> stats = fetcher.getFileStatuses();
+    IOStatistics iostats = extractStatistics(fetcher);
+    LOG.info("Statistics of fetcher: {}", iostats);
+    Assertions.assertThat(stats)
+        .describedAs("result of located scan")
+        .isNotNull()
+        .flatExtracting(FileStatus::getPath)
+        .containsExactlyInAnyOrder(
+            emptyFile,
+            subdirFile,
+            subdir2File1,
+            subdir2File2);
+    assertListCount(fetcher, EXPECTED_LIST_COUNT);
+  }
+
+  /**
+   * Run a located file status fetcher against a file.
+   */
+  @Test
+  public void testLocatedFileStatusScanFile() throws Throwable {
+    // pass in a file as the base of the scan.
+    describe("LocatedFileStatusFetcher with file %s", subdirFile);
+    listConfig.setInt(LIST_STATUS_NUM_THREADS, 16);
+    LocatedFileStatusFetcher fetcher
+        = new LocatedFileStatusFetcher(
+        listConfig,
+        new Path[]{subdirFile},
+        true,
+        TEXT_FILE,
+        true);
+    Iterable<FileStatus> stats = fetcher.getFileStatuses();
+    Assertions.assertThat(stats)
+        .describedAs("result of located scan")
+        .isNotNull()
+        .flatExtracting(FileStatus::getPath)
+        .containsExactly(subdirFile);
+    IOStatistics ioStatistics = fetcher.getIOStatistics();
+    Assertions.assertThat(ioStatistics)
+        .describedAs("IO statistics of %s", fetcher)
+        .isNull();
+  }
+
 }
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java
index afd3ec2..88e0cef 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.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics;
 import org.apache.hadoop.io.IOUtils;
 
 import org.junit.BeforeClass;
@@ -94,16 +95,16 @@ public class ITestS3ABlockOutputArray extends AbstractS3ATestBase {
     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 @@ public class ITestS3ABlockOutputArray extends AbstractS3ATestBase {
       throws Exception {
     S3AInstrumentation instrumentation =
         new S3AInstrumentation(new URI("s3a://example"));
-    S3AInstrumentation.OutputStreamStatistics outstats
+    BlockOutputStreamStatistics outstats
         = instrumentation.newOutputStreamStatistics(null);
     S3ADataBlocks.DataBlock block = factory.create(1, BLOCK_SIZE, outstats);
     block.write(dataset, 0, dataset.length);
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetrics.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetrics.java
index 972c665..3bfe69c 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetrics.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetrics.java
@@ -21,11 +21,15 @@ package org.apache.hadoop.fs.s3a;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+
+import org.assertj.core.api.Assertions;
 import org.junit.Test;
 
 import java.io.IOException;
 import java.io.InputStream;
 
+import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsSourceToString;
+
 /**
  * Test s3a performance metrics register and output.
  */
@@ -51,17 +55,34 @@ public class ITestS3AMetrics extends AbstractS3ATestBase {
     Path file = path("testStreamStatistics");
     byte[] data = "abcdefghijklmnopqrstuvwxyz".getBytes();
     ContractTestUtils.createFile(fs, file, false, data);
-
-    try (InputStream inputStream = fs.open(file)) {
+    InputStream inputStream = fs.open(file);
+    try {
       while (inputStream.read(data) != -1) {
         LOG.debug("Read batch of data from input stream...");
       }
+      LOG.info("Final stream statistics: {}",
+          ioStatisticsSourceToString(inputStream));
+    } finally {
+      // this is not try-with-resources only to aid debugging
+      inputStream.close();
     }
 
+    final String statName = Statistic.STREAM_READ_BYTES.getSymbol();
+
+    final S3AInstrumentation instrumentation = fs.getInstrumentation();
+
+    final long counterValue = instrumentation.getCounterValue(statName);
+
+    final int expectedBytesRead = 26;
+    Assertions.assertThat(counterValue)
+        .describedAs("Counter %s from instrumentation %s",
+            statName, instrumentation)
+        .isEqualTo(expectedBytesRead);
     MutableCounterLong read = (MutableCounterLong)
-        fs.getInstrumentation().getRegistry()
-        .get(Statistic.STREAM_SEEK_BYTES_READ.getSymbol());
-    assertEquals("Stream statistics were not merged", 26, read.value());
+        instrumentation.getRegistry()
+        .get(statName);
+    assertEquals("Stream statistics were not merged", expectedBytesRead,
+        read.value());
   }
 
 
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java
index d0d42b8..2f0599d 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java
@@ -276,8 +276,14 @@ public class ITestS3AMiscOperations extends AbstractS3ATestBase {
     }
   }
 
+  @Test
+  public void testS3AIOStatisticsUninitialized() throws Throwable {
+    try (S3AFileSystem fs = new S3AFileSystem()) {
+      fs.getIOStatistics();
+    }
+
+  }
   /**
-<<<<<<< ours
    * Verify that paths with a trailing "/" are fixed up.
    */
   @Test
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AUnbuffer.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AUnbuffer.java
index 2ba3fd7..3d7ee08 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AUnbuffer.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AUnbuffer.java
@@ -21,13 +21,23 @@ package org.apache.hadoop.fs.s3a;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
+import org.apache.hadoop.fs.statistics.StoreStatisticNames;
+import org.apache.hadoop.fs.statistics.StreamStatisticNames;
 import org.apache.hadoop.io.IOUtils;
 
+import org.assertj.core.api.Assertions;
 import org.junit.Test;
 
 import java.io.IOException;
 
-import static org.apache.hadoop.fs.s3a.Statistic.STREAM_SEEK_BYTES_READ;
+import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_BYTES;
+import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_BYTES_READ_CLOSE;
+import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_TOTAL_BYTES;
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue;
+import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.demandStringifyIOStatisticsSource;
 
 /**
  * Integration test for calling
@@ -40,6 +50,8 @@ import static org.apache.hadoop.fs.s3a.Statistic.STREAM_SEEK_BYTES_READ;
  */
 public class ITestS3AUnbuffer extends AbstractS3ATestBase {
 
+  public static final int FILE_LENGTH = 16;
+
   private Path dest;
 
   @Override
@@ -48,7 +60,7 @@ public class ITestS3AUnbuffer extends AbstractS3ATestBase {
     dest = path("ITestS3AUnbuffer");
     describe("ITestS3AUnbuffer");
 
-    byte[] data = ContractTestUtils.dataset(16, 'a', 26);
+    byte[] data = ContractTestUtils.dataset(FILE_LENGTH, 'a', 26);
     ContractTestUtils.writeDataset(getFileSystem(), dest, data, data.length,
             16, true);
   }
@@ -57,13 +69,41 @@ public class ITestS3AUnbuffer extends AbstractS3ATestBase {
   public void testUnbuffer() throws IOException {
     describe("testUnbuffer");
 
+    IOStatisticsSnapshot iostats = new IOStatisticsSnapshot();
     // Open file, read half the data, and then call unbuffer
     try (FSDataInputStream inputStream = getFileSystem().open(dest)) {
       assertTrue(inputStream.getWrappedStream() instanceof S3AInputStream);
-      readAndAssertBytesRead(inputStream, 8);
+      int bytesToRead = 8;
+      readAndAssertBytesRead(inputStream, bytesToRead);
       assertTrue(isObjectStreamOpen(inputStream));
+      assertTrue("No IOstatistics from " + inputStream,
+          iostats.aggregate(inputStream.getIOStatistics()));
+      verifyStatisticCounterValue(iostats,
+          StreamStatisticNames.STREAM_READ_BYTES,
+          bytesToRead);
+      verifyStatisticCounterValue(iostats,
+          StoreStatisticNames.ACTION_HTTP_GET_REQUEST,
+          1);
+
+      // do the unbuffering
       inputStream.unbuffer();
 
+      // audit the updated statistics
+      IOStatistics st2 = inputStream.getIOStatistics();
+
+      // the unbuffered operation must be tracked
+      verifyStatisticCounterValue(st2,
+          StreamStatisticNames.STREAM_READ_UNBUFFERED,
+          1);
+
+      // all other counter values consistent.
+      verifyStatisticCounterValue(st2,
+          StreamStatisticNames.STREAM_READ_BYTES,
+          bytesToRead);
+      verifyStatisticCounterValue(st2,
+          StoreStatisticNames.ACTION_HTTP_GET_REQUEST,
+          1);
+
       // Check the the wrapped stream is closed
       assertFalse(isObjectStreamOpen(inputStream));
     }
@@ -71,7 +111,7 @@ public class ITestS3AUnbuffer extends AbstractS3ATestBase {
 
   /**
    * Test that calling {@link S3AInputStream#unbuffer()} merges a stream's
-   * {@link org.apache.hadoop.fs.s3a.S3AInstrumentation.InputStreamStatistics}
+   * {@code InputStreamStatistics}
    * into the {@link S3AFileSystem}'s {@link S3AInstrumentation} instance.
    */
   @Test
@@ -79,36 +119,73 @@ public class ITestS3AUnbuffer extends AbstractS3ATestBase {
     describe("testUnbufferStreamStatistics");
 
     // Validate bytesRead is updated correctly
+    S3AFileSystem fs = getFileSystem();
     S3ATestUtils.MetricDiff bytesRead = new S3ATestUtils.MetricDiff(
-            getFileSystem(), STREAM_SEEK_BYTES_READ);
+        fs, STREAM_READ_BYTES);
+    S3ATestUtils.MetricDiff totalBytesRead = new S3ATestUtils.MetricDiff(
+        fs, STREAM_READ_TOTAL_BYTES);
+    S3ATestUtils.MetricDiff bytesReadInClose = new S3ATestUtils.MetricDiff(
+        fs, STREAM_READ_BYTES_READ_CLOSE);
 
     // Open file, read half the data, and then call unbuffer
     FSDataInputStream inputStream = null;
+    int firstBytesToRead = 8;
+
+    int secondBytesToRead = 1;
+    long expectedFinalBytesRead;
+    long expectedTotalBytesRead;
+
+    Object streamStatsStr;
     try {
-      inputStream = getFileSystem().open(dest);
+      inputStream = fs.open(dest);
+      streamStatsStr = demandStringifyIOStatisticsSource(inputStream);
 
-      readAndAssertBytesRead(inputStream, 8);
+      LOG.info("initial stream statistics {}", streamStatsStr);
+      readAndAssertBytesRead(inputStream, firstBytesToRead);
+      LOG.info("stream statistics after read {}", streamStatsStr);
       inputStream.unbuffer();
 
       // Validate that calling unbuffer updates the input stream statistics
-      bytesRead.assertDiffEquals(8);
+      bytesRead.assertDiffEquals(firstBytesToRead);
+      final long bytesInUnbuffer = bytesReadInClose.diff();
+      totalBytesRead.assertDiffEquals(firstBytesToRead + bytesInUnbuffer);
 
       // Validate that calling unbuffer twice in a row updates the statistics
       // correctly
-      readAndAssertBytesRead(inputStream, 4);
... 2252 lines suppressed ...


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org