diff --git a/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml b/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml
index b51053603fa..1ebf8587e88 100644
--- a/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml
+++ b/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml
@@ -74,4 +74,14 @@
+
+
+
+
+
+
diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml
index af8983e2ebe..f6d70230944 100644
--- a/hadoop-tools/hadoop-aws/pom.xml
+++ b/hadoop-tools/hadoop-aws/pom.xml
@@ -201,6 +201,8 @@
**/ITestMarkerToolRootOperations.java**/ITestS3GuardDDBRootOperations.java
+
+ **/ITestAggregateIOStatistics.java
@@ -250,6 +252,8 @@
**/ITestS3AContractRootDir.java**/ITestS3GuardDDBRootOperations.java
+
+ **/ITestAggregateIOStatistics.java
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 a1c1d969a82..fcaec509290 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 276961bf8b7..96d16e8b1b8 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.
+ *
+ * If the AWS stats are not null then a {@link AwsStatisticsCollector}.
+ * is created to bind to the two.
+ * Important: until this binding works properly across regions,
+ * this should be null.
+ */
@Override
public AmazonS3 createS3Client(URI name,
final String bucket,
final AWSCredentialsProvider credentials,
- final String userAgentSuffix) throws IOException {
+ final String userAgentSuffix,
+ final StatisticsFromAwsSdk statisticsFromAwsSdk) throws IOException {
Configuration conf = getConf();
final ClientConfiguration awsConf = S3AUtils
.createAwsConf(conf, bucket, Constants.AWS_SERVICE_IDENTIFIER_S3);
@@ -72,36 +99,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));
+ }
+
+ /**
+ * Create an {@link AmazonS3} client.
+ * Override this to provide an extended version of the client
+ * @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
+ */
+ protected AmazonS3 newAmazonS3Client(
+ 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);
+ }
+ }
+
+ /**
+ * Use the (newer) Builder SDK to create a an AWS S3 client.
+ *
+ * This has a more complex endpoint configuration in a
+ * way which does not yet work in this code in a way
+ * which doesn't trigger regressions. So it is only used
+ * when SDK metrics are supplied.
+ * @param credentials credentials to use
+ * @param awsConf AWS configuration
+ * @param metrics metrics collector or null
+ * @param endpoint endpoint string; may be ""
+ * @param pathStyleAccess enable path style access?
+ * @return new AmazonS3 client
+ */
+ private AmazonS3 buildAmazonS3Client(
+ final AWSCredentialsProvider credentials,
+ final ClientConfiguration awsConf,
+ final RequestMetricCollector metrics,
+ final String endpoint,
+ final boolean pathStyleAccess) {
+ AmazonS3ClientBuilder b = AmazonS3Client.builder();
+ b.withCredentials(credentials);
+ b.withClientConfiguration(awsConf);
+ b.withPathStyleAccessEnabled(pathStyleAccess);
+ if (metrics != null) {
+ b.withMetricsCollector(metrics);
+ }
+
+ // endpoint set up is a PITA
+ // client.setEndpoint("") is no longer available
+ AwsClientBuilder.EndpointConfiguration epr
+ = createEndpointConfiguration(endpoint, awsConf);
+ if (epr != null) {
+ // an endpoint binding was constructed: use it.
+ b.withEndpointConfiguration(epr);
+ }
+ final AmazonS3 client = b.build();
+ return client;
}
/**
* Wrapper around constructor for {@link AmazonS3} client.
- * Override this to provide an extended version of the client
+ * Override this to provide an extended version of the client.
+ *
+ * This uses a deprecated constructor -it is currently
+ * the only one which works for us.
* @param credentials credentials to use
* @param awsConf AWS configuration
- * @return new AmazonS3 client
+ * @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);
+ @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 S3 client from the Hadoop configuration.
- *
+ * Configure classic S3 client.
+ *
* 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.
- *
+ *
* The {@link Constants#PATH_STYLE_ACCESS} option enables path-style access
* to S3 buckets if configured. By default, the
* behavior is to use virtual hosted-style access with URIs of the form
* {@code http://bucketname.s3.amazonaws.com}
+ *
* Enabling path-style access and a
* region-specific endpoint switches the behavior to use URIs of the form
* {@code http://s3-eu-west-1.amazonaws.com/bucketname}.
* It is common to use this when connecting to private S3 servers, as it
* avoids the need to play with DNS entries.
* @param s3 S3 client
- * @param conf Hadoop configuration
+ * @param pathStyleAccess enable path style access?
* @return the S3 client
*/
- private static AmazonS3 applyS3ClientOptions(AmazonS3 s3,
- Configuration conf) {
- final boolean pathStyleAccess = conf.getBoolean(PATH_STYLE_ACCESS, false);
+ protected static AmazonS3 applyS3ClientOptions(AmazonS3 s3,
+ final boolean pathStyleAccess) {
if (pathStyleAccess) {
LOG.debug("Enabling path style access!");
s3.setS3ClientOptions(S3ClientOptions.builder()
@@ -144,4 +259,54 @@ public class DefaultS3ClientFactory extends Configured
}
return s3;
}
+
+ /**
+ * Given an endpoint string, return an endpoint config, or null, if none
+ * is needed.
+ *
+ * This is a pretty painful piece of code. It is trying to replicate
+ * what AwsClient.setEndpoint() does, because you can't
+ * call that setter on an AwsClient constructed via
+ * the builder, and you can't pass a metrics collector
+ * down except through the builder.
+ *
+ * Note also that AWS signing is a mystery which nobody fully
+ * understands, especially given all problems surface in a
+ * "400 bad request" response, which, like all security systems,
+ * provides minimal diagnostics out of fear of leaking
+ * secrets.
+ *
+ * @param endpoint possibly null endpoint.
+ * @param awsConf config to build the URI from.
+ * @return a configuration for the S3 client builder.
+ */
+ @VisibleForTesting
+ public static AwsClientBuilder.EndpointConfiguration
+ createEndpointConfiguration(
+ final String endpoint, final ClientConfiguration awsConf) {
+ LOG.debug("Creating endpoint configuration for {}", endpoint);
+ if (endpoint == null || endpoint.isEmpty()) {
+ // the default endpoint...we should be using null at this point.
+ LOG.debug("Using default endpoint -no need to generate a configuration");
+ return null;
+ }
+
+ final URI epr = RuntimeHttpUtils.toUri(endpoint, awsConf);
+ LOG.debug("Endpoint URI = {}", epr);
+
+ String region;
+ if (!ServiceUtils.isS3USStandardEndpoint(endpoint)) {
+ LOG.debug("Endpoint {} is not the default; parsing", epr);
+ region = AwsHostNameUtils.parseRegion(
+ epr.getHost(),
+ S3_SERVICE_NAME);
+ } else {
+ // US-east, set region == null.
+ LOG.debug("Endpoint {} is the standard one; declare region as null", epr);
+ region = null;
+ }
+ LOG.debug("Region for endpoint {}, URI {} is determined as {}",
+ endpoint, epr, region);
+ return new AwsClientBuilder.EndpointConfiguration(endpoint, region);
+ }
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java
index 932c472f5be..ddc492235db 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 a79737df376..19cd6c985b5 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 once(String action, String path, Operation operation)
+ public static T once(String action, String path,
+ CallableRaisingIOE 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 operation) {
+ CallableRaisingIOE operation) {
try {
once(action, path, operation);
} catch (IOException e) {
@@ -280,7 +285,7 @@ public class Invoker {
public T retry(String action,
@Nullable String path,
boolean idempotent,
- Operation operation)
+ CallableRaisingIOE 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 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 operation)
+ CallableRaisingIOE 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 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 operation)
+ CallableRaisingIOE operation)
throws IOException {
if (doRetry) {
return retryUntranslated(
@@ -366,7 +371,7 @@ public class Invoker {
public T retryUntranslated(
String text,
boolean idempotent,
- Operation operation) throws IOException {
+ CallableRaisingIOE operation) throws IOException {
return retryUntranslated(text, idempotent,
retryCallback, operation);
}
@@ -391,7 +396,7 @@ public class Invoker {
String text,
boolean idempotent,
Retried retrying,
- Operation operation) throws IOException {
+ CallableRaisingIOE 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 Optional quietlyEval(String action,
String path,
- Operation operation) {
+ CallableRaisingIOE operation) {
try {
return Optional.of(once(action, path, operation));
} catch (Exception e) {
@@ -510,15 +515,6 @@ public class Invoker {
(StringUtils.isNotEmpty(path) ? (" on " + path) : "");
}
- /**
- * Arbitrary operation throwing an IOException.
- * @param return type
- */
- @FunctionalInterface
- public interface Operation {
- T execute() throws IOException;
- }
-
/**
* Void operation which may raise an IOException.
*/
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 8b35edaf59b..3cb3d5d832d 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 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 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 createLocatedFileStatusIterator(
RemoteIterator 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 iterator, Set tombstones) {
- return new TombstoneReconcilingIterator(iterator, tombstones);
+ RemoteIterator createTombstoneReconcilingIterator(
+ RemoteIterator iterator,
+ @Nullable Set tombstones) {
+ if (tombstones == null || tombstones.isEmpty()) {
+ // no need to filter.
+ return iterator;
+ } else {
+ return filteringRemoteIterator(
+ iterator,
+ candidate -> !tombstones.contains(candidate.getPath()));
+ }
}
+ /**
+ * Create a remote iterator from a single status entry.
+ * @param status status
+ * @return iterator.
+ */
+ public RemoteIterator createSingleStatusIterator(
+ S3ALocatedFileStatus status) {
+ return 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 mfsItr = createProvidedFileStatusIterator(
S3Guard.dirMetaToStatuses(dirMeta),
ACCEPT_ALL,
Listing.ACCEPT_ALL_BUT_S3N);
@@ -429,105 +460,6 @@ public class Listing extends AbstractStoreOperation {
boolean accept(FileStatus status);
}
- /**
- * A remote iterator which only iterates over a single `LocatedFileStatus`
- * value.
- *
- * If the status value is null, the iterator declares that it has no data.
- * This iterator is used to handle {@link S3AFileSystem#listStatus(Path)}
- * calls where the path handed in refers to a file, not a directory:
- * this is the iterator returned.
- */
- static final class SingleStatusRemoteIterator
- implements RemoteIterator {
-
- /**
- * The status to return; set to null after the first iteration.
- */
- private S3ALocatedFileStatus status;
-
- /**
- * Constructor.
- * @param status status value: may be null, in which case
- * the iterator is empty.
- */
- SingleStatusRemoteIterator(S3ALocatedFileStatus status) {
- this.status = status;
- }
-
- /**
- * {@inheritDoc}
- * @return true if there is a file status to return: this is always false
- * for the second iteration, and may be false for the first.
- * @throws IOException never
- */
- @Override
- public boolean hasNext() throws IOException {
- return status != null;
- }
-
- /**
- * {@inheritDoc}
- * @return the non-null status element passed in when the instance was
- * constructed, if it ha not already been retrieved.
- * @throws IOException never
- * @throws NoSuchElementException if this is the second call, or it is
- * the first call and a null {@link LocatedFileStatus} entry was passed
- * to the constructor.
- */
- @Override
- public S3ALocatedFileStatus next() throws IOException {
- if (hasNext()) {
- S3ALocatedFileStatus s = this.status;
- status = null;
- return s;
- } else {
- throw new NoSuchElementException();
- }
- }
- }
-
- /**
- * This wraps up a provided non-null list of file status as a remote iterator.
- *
- * 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 {
- private final ArrayList 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 {
+ implements RemoteIterator, 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 {
+ class ObjectListingIterator implements RemoteIterator,
+ 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;
}
/**
@@ -966,89 +928,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 {
- private final RemoteIterator statusIterator;
-
- /**
- * Constructor.
- * @param statusIterator an iterator over the remote status entries
- */
- LocatedFileStatusIterator(RemoteIterator statusIterator) {
- this.statusIterator = statusIterator;
- }
-
- @Override
- public boolean hasNext() throws IOException {
- return statusIterator.hasNext();
- }
-
- @Override
- public S3ALocatedFileStatus next() throws IOException {
- return listingOperationCallbacks
- .toLocatedFileStatus(statusIterator.next());
- }
- }
-
- /**
- * Wraps another iterator and filters out files that appear in the provided
- * set of tombstones. Will read ahead in the iterator when necessary to
- * ensure that emptiness is detected early enough if only deleted objects
- * remain in the source iterator.
- */
- static class TombstoneReconcilingIterator implements
- RemoteIterator {
- private S3ALocatedFileStatus next = null;
- private final RemoteIterator iterator;
- private final Set tombstones;
-
- /**
- * @param iterator Source iterator to filter
- * @param tombstones set of tombstone markers to filter out of results
- */
- TombstoneReconcilingIterator(RemoteIterator
- iterator, Set tombstones) {
- this.iterator = iterator;
- if (tombstones != null) {
- this.tombstones = tombstones;
- } else {
- this.tombstones = Collections.emptySet();
- }
- }
-
- private boolean fetch() throws IOException {
- while (next == null && iterator.hasNext()) {
- S3ALocatedFileStatus candidate = iterator.next();
- if (!tombstones.contains(candidate.getPath())) {
- next = candidate;
- return true;
- }
- }
- return false;
- }
-
- public boolean hasNext() throws IOException {
- if (next != null) {
- return true;
- }
- return fetch();
- }
-
- public S3ALocatedFileStatus next() throws IOException {
- if (hasNext()) {
- S3ALocatedFileStatus result = next;
- next = null;
- fetch();
- return result;
- }
- throw new NoSuchElementException();
- }
- }
-
/**
* Accept all entries except those which map to S3N pseudo directory markers.
*/
@@ -1117,4 +996,9 @@ public class Listing extends AbstractStoreOperation {
}
}
+ public static RemoteIterator 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 b17412f211d..0fdad2150b6 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 =
@@ -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 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 fa38f246dd4..25031770690 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 a02947b0152..f60ff75c7d6 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);
@@ -531,6 +555,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
@@ -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 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 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);
}
}
@@ -1917,6 +1990,27 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
return storageStatistics;
}
+ /**
+ * 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.
@@ -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);
+ 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();
}
- ObjectMetadata objectMetadata = s3.getObjectMetadata(request);
- if (changeTracker != null) {
- changeTracker.processMetadata(objectMetadata, operation);
- }
- 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 listStatusItr = once("listStatus",
p.toString(), () -> innerListStatus(p));
- return new RemoteIterator() {
- @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 toLocatedFileStatusIterator(
- RemoteIterator extends LocatedFileStatus> iterator) {
- return new RemoteIterator() {
- @Override
- public boolean hasNext() throws IOException {
- return iterator.hasNext();
- }
-
- @Override
- public LocatedFileStatus next() throws IOException {
- return iterator.next();
- }
- };
- }
-
/**
* Recursive List of files and empty directories.
* @param f path to list from
@@ -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 bd8adad035d..c725fdf37ff 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;
@@ -131,6 +135,11 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
/** change tracker. */
private final ChangeTracker changeTracker;
+ /**
+ * IOStatistics report.
+ */
+ private final IOStatistics ioStatistics;
+
/**
* Create the stream.
* This does not attempt to open it; that is only done on the first
@@ -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 19f42cf9e39..982611a0984 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.
+ *
+ * History
+ *
+ *
+ *
+ * HADOOP-13028. Initial implementation.
+ * Derived from the {@code AzureFileSystemInstrumentation}.
+ *
+ *
+ * Broadly (and directly) used in S3A.
+ * The use of direct references causes "problems" in mocking tests.
+ *
+ *
+ * HADOOP-16830. IOStatistics. Move to an interface and implementation
+ * design for the different inner classes.
+ *
+ *
+ *
* Counters and metrics are generally addressed in code by their name or
* {@link Statistic} key. There may be some Statistics which do
* not have an entry here. To avoid attempts to access such counters failing,
* the operations to increment/query metric values are designed to handle
* lookup failures.
+ *
+ *
+ * S3AFileSystem StorageStatistics are dynamically derived from
+ * the IOStatistics.
+ *
+ *
+ * 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.
+ *
*/
@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.
+ *
+ * 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 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++;
@@ -312,6 +301,15 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
return counter(op.getSymbol(), op.getDescription());
}
+ /**
+ * 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
@@ -322,6 +320,15 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
return registry.newGauge(name, desc, 0L);
}
+ /**
+ * 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
@@ -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;
}
@@ -442,11 +450,57 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
return metric;
}
+ /**
+ * 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
*/
@@ -566,12 +639,30 @@ 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 final class MetricUpdatingDurationTracker
+ implements DurationTracker {
- private InputStreamStatistics() {
+ 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.
+ *
+ * When {@code unbuffer()} is called, the changed numbers are propagated
+ * to the S3AFileSystem metrics.
+ *
+ *
+ * When {@code close()} is called, the final set of numbers are propagated
+ * to the S3AFileSystem metrics.
+ *
+ * The {@link FileSystem.Statistics} statistics passed in are also
+ * updated. This ensures that whichever thread calls close() gets the
+ * total count of bytes read, even if any work is done in other
+ * threads.
+ *
+ */
+ 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. Behavior is undefined if called on a
- * closed instance.
+ *
+ * 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.
+ *
+ *
+ * Whichever thread close()d the stream will have its counters
+ * updated.
+ *
+ *
+ * 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.
+ *
+ *
+ * @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
+ *
+ * Some of these stats are propagated to any passed in
+ * {@link FileSystem.Statistics} instance; this is done
+ * in close() for better cross-thread accounting.
+ *
+ *
+ * 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.
+ *
*/
- @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();
}
- /**
- * A block has been released.
- */
- void blockReleased() {
+ @Override
+ public void blockReleased() {
blocksReleased.incrementAndGet();
}
/**
- * Block is queued for upload.
+ * {@inheritDoc}
+ * Increments the counter of block uplaods, and the gauges
+ * of block uploads pending (1) and the bytes pending (blockSize).
*/
- 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);
+ @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);
}
- /** 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);
+ /**
+ * {@inheritDoc}
+ * Update {@link #queueDuration} with queue duration, decrement
+ * {@code STREAM_WRITE_BLOCK_UPLOADS_PENDING} gauge and increment
+ * {@code STREAM_WRITE_BLOCK_UPLOADS_ACTIVE}.
+ */
+ @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);
}
- /** A block upload has completed. */
- void blockUploadCompleted(long duration, int blockSize) {
- this.transferDuration.addAndGet(duration);
- incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_ACTIVE, -1);
- blocksActive.decrementAndGet();
+ /**
+ * 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();
- }
-
- /** Intermediate report of bytes uploaded. */
- void bytesTransferred(long byteCount) {
- bytesUploaded.addAndGet(byteCount);
- statistics.incrementBytesWritten(byteCount);
- bytesPendingUpload.addAndGet(-byteCount);
- incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING, -byteCount);
+ @Override
+ public void blockUploadFailed(
+ Duration timeSinceUploadStarted,
+ int blockSize) {
+ incCounter(StreamStatisticNames.STREAM_WRITE_EXCEPTIONS);
}
/**
- * Note exception in a multipart complete.
- * @param count count of exceptions
+ * Intermediate report of bytes uploaded.
+ * Increment counters of bytes upload, reduce the counter and
+ * gauge of pending bytes.;
+ * @param byteCount bytes uploaded
*/
- void exceptionInMultipartComplete(int count) {
+ @Override
+ public void bytesTransferred(long byteCount) {
+ bytesUploaded.addAndGet(byteCount);
+ incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING, -byteCount);
+ }
+
+ @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 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 f09aa82281d..aca1fa5e140 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 12be3a51f81..3729341dbfe 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 4b126673b55..33ae0911996 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 {
- private static final Logger LOG =
- LoggerFactory.getLogger(S3AStorageStatistics.class);
+public class S3AStorageStatistics
+ extends StorageStatisticsFromIOStatistics {
public static final String NAME = "S3AStorageStatistics";
- private final Map opsCount =
- new EnumMap<>(Statistic.class);
+
+ public S3AStorageStatistics(final IOStatistics ioStatistics) {
+ super(NAME, "s3a", ioStatistics);
+ }
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 {
- private Iterator> 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 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";
- }
-
- @Override
- public Iterator getLongStatistics() {
- return new LongIterator();
- }
-
- @Override
- public Iterator 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);
- }
+ 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 65c0c236078..314f13f0352 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 e0a1d780ccf..e04d3b5cbd2 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 1addfbef787..6709382baf5 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}.
+ *
+ * Where possible, stream names come from {@link StreamStatisticNames}
+ * and {@link StoreStatisticNames}
+ *
*/
+@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"),
+ "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),
- 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"),
+ /* 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),
- // S3guard committer stats
+ /* 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"),
+ "S3Guard metadata store authoritative directories updated from S3",
+ TYPE_COUNTER),
- 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"),
+ /* General Store operations */
+ STORE_IO_REQUEST(StoreStatisticNames.STORE_IO_REQUEST,
+ "requests made of the remote store",
+ TYPE_COUNTER),
- 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"),
+ 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 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 6b7c5699a17..e75c09c9f4c 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;
@@ -108,17 +110,26 @@ public class WriteOperationHelper implements WriteOperations {
/** Bucket of the owner FS. */
private final String bucket;
+ /**
+ * statistics context.
+ */
+ private final S3AStatisticsContext statisticsContext;
+
/**
* Constructor.
* @param owner owner FS creating the helper
* @param conf Configuration object
+ * @param statisticsContext statistics context
*
*/
- protected WriteOperationHelper(S3AFileSystem owner, Configuration conf) {
+ protected WriteOperationHelper(S3AFileSystem owner,
+ Configuration conf,
+ S3AStatisticsContext statisticsContext) {
this.owner = owner;
this.invoker = new Invoker(new S3ARetryPolicy(conf),
this::operationRetried);
this.conf = conf;
+ this.statisticsContext = statisticsContext;
bucket = owner.getBucket();
}
@@ -138,19 +149,19 @@ public class WriteOperationHelper implements WriteOperations {
/**
* Execute a function with retry processing.
+ * @param 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 type of return value
* @return the result of the call
* @throws IOException any IOE raised, or translated exception
*/
public T retry(String action,
String path,
boolean idempotent,
- Invoker.Operation operation)
+ CallableRaisingIOE 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 95cbd7e70b7..0b336142ccc 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 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 type of return value
* @return the result of the call
* @throws IOException any IOE raised, or translated exception
*/
T retry(String action,
String path,
boolean idempotent,
- Invoker.Operation operation)
+ CallableRaisingIOE 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 db149de4122..d8fc8b83964 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 token
- = tokenBinding.createDelegationToken(rolePolicy, encryptionSecrets, renewer);
+ Token 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 771d6064c0c..f73689979ec 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, and the HTTP connection pool in the S3A
* committer was large enough for more all the parallel POST requests.
*/
-public abstract class AbstractS3ACommitter extends PathOutputCommitter {
+public abstract class AbstractS3ACommitter extends PathOutputCommitter
+ implements IOStatisticsSource {
private static final Logger LOG =
LoggerFactory.getLogger(AbstractS3ACommitter.class);
@@ -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 filenames)
+ List 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.
@@ -1385,6 +1411,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
@@ -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 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 11eeee9e940..c9fb3806b2c 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 7f9dadf06f3..41f95c86206 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 bbffef3ad22..10440f77e72 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 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 8ad03421058..fd734102566 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.
+ *
+ * The statistics published through the {@link IOStatisticsSource}
+ * interface are the static ones marshalled with the commit data;
+ * they may be empty.
+ *
+ *
+ * As single commits are added via {@link #add(SinglePendingCommit)},
+ * any statistics from those commits are merged into the aggregate
+ * statistics, and those of the single commit cleared.
+ *
*/
@SuppressWarnings("unused")
@InterfaceAudience.Private
@InterfaceStability.Unstable
-public class PendingSet extends PersistentCommitData {
+public class PendingSet extends PersistentCommitData
+ implements IOStatisticsSource {
private static final Logger LOG = LoggerFactory.getLogger(PendingSet.class);
/**
@@ -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 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 ee7c7b236c6..021cc7b34b1 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.
+ *
* It's marked as {@link Serializable} so that it can be passed in RPC
* calls; for this to work it relies on the fact that java.io ArrayList
* and LinkedList are serializable. If any other list type is used for etags,
* it must also be serialized. Jackson expects lists, and it is used
* to persist to disk.
- *
+ *
+ *
+ * The statistics published through the {@link IOStatisticsSource}
+ * interface are the static ones marshalled with the commit data;
+ * they may be empty.
+ *
*/
@SuppressWarnings("unused")
@InterfaceAudience.Private
@InterfaceStability.Unstable
public class SinglePendingCommit extends PersistentCommitData
- implements Iterable {
+ implements Iterable, IOStatisticsSource {
/**
* Serialization ID: {@value}.
@@ -113,6 +122,12 @@ public class SinglePendingCommit extends PersistentCommitData
*/
private Map 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 b7509d6714b..4500e1429e4 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,9 +65,18 @@ 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}.
*/
@@ -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 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 13967645df6..0f1a0a6534d 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 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 b330cee4e6b..34bbfd4ed76 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 00000000000..3ad2bbff3ba
--- /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 b2c1cc6271e..4169a9899cb 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 1c61d30b08a..b788f507735 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 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 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 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 bb025c4ed07..48050563cde 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 d1baf3c8987..f7e06413a37 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
@@ -54,6 +54,18 @@ public class ErrorTranslation {
&& AwsErrorCodes.E_NO_SUCH_BUCKET.equals(e.getErrorCode());
}
+ /**
+ * 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 9ecc1a93c60..0e8d864d4c4 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 e5f9f7d9808..b0ebf1ca169 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 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 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 ca1b09e9bdc..3eeb01f971b 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 e811af94c37..5ff116fe38e 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
* and there has been no unsuccessful attempt to create one.
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploaderBuilder.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploaderBuilder.java
index 3bf1a7ddd91..cb38b82a765 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 b375c31bb53..28be486b438 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 44353b99436..e1f2cb15b82 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 8ec848839e3..eaf9ee22f91 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 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 clearedParentPathSet = new HashSet<>();
// declare the operation to delete a batch as a function so
// as to keep the code consistent across multiple uses.
- FunctionsRaisingIOE.CallableRaisingIOE deleteBatchOperation =
+ CallableRaisingIOE deleteBatchOperation =
() -> {
// lowest path entries get deleted first.
deletionBatch.sort(PathOrderComparators.TOPMOST_PATH_LAST);
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/InternalIterators.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/InternalIterators.java
deleted file mode 100644
index dd6fb5ff115..00000000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/InternalIterators.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.s3a.s3guard;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.fs.s3a.S3AFileStatus;
-
-/**
- * Internal iterators.
- */
-final class InternalIterators {
-
- private InternalIterators() {
- }
-
- /**
- * From a remote status iterator, build a path iterator.
- */
- static final class PathFromRemoteStatusIterator implements
- RemoteIterator {
-
- private final RemoteIterator source;
-
- /**
- * Construct.
- * @param source source iterator.
- */
- PathFromRemoteStatusIterator(final RemoteIterator source) {
- this.source = source;
- }
-
- @Override
- public boolean hasNext() throws IOException {
- return source.hasNext();
- }
-
- @Override
- public Path next() throws IOException {
- return source.next().getPath();
- }
- }
-
- /**
- * From a classic java.util.Iterator, build a Hadoop remote iterator.
- * @param type of iterated value.
- */
- static final class RemoteIteratorFromIterator implements
- RemoteIterator {
-
- private final Iterator source;
-
- /**
- * Construct.
- * @param source source iterator.
- */
- RemoteIteratorFromIterator(final Iterator source) {
- this.source = source;
- }
-
- @Override
- public boolean hasNext() {
- return source.hasNext();
- }
-
- @Override
- public T next() {
- return source.next();
- }
- }
-
-}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RetryingCollection.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RetryingCollection.java
index 394f393c57e..108d205f74b 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 implements Iterable {
/**
* {@inheritDoc}.
- * @throws WrappedIOException for IO failure, including throttling.
+ * @throws UncheckedIOException for IO failure, including throttling.
*/
@Override
@Retries.RetryTranslated
@@ -100,13 +100,13 @@ class RetryingCollection implements Iterable {
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 implements Iterable {
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 69b9aaf7807..02d1e53c7ba 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 00000000000..b1cee718c20
--- /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.
+ *
+ * A final transfer completed event is still expected, so this
+ * does not decrement the active block counter.
+ *
+ * @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/statistics/ChangeTrackerStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/ChangeTrackerStatistics.java
new file mode 100644
index 00000000000..572581e214f
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/ChangeTrackerStatistics.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.statistics;
+
+/**
+ * Interface for change tracking statistics.
+ */
+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 00000000000..fd232a058d0
--- /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 00000000000..61cc0330936
--- /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/statistics/DelegationTokenStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/DelegationTokenStatistics.java
new file mode 100644
index 00000000000..3ab9224f3c5
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/DelegationTokenStatistics.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.statistics;
+
+/**
+ * Instrumentation exported to for S3A Delegation Token support.
+ */
+public interface DelegationTokenStatistics extends S3AStatisticInterface {
+
+ /** A token has been issued. */
+ void tokenIssued();
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AInputStreamStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AInputStreamStatistics.java
new file mode 100644
index 00000000000..328d9f7c4ce
--- /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%
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/S3AMultipartUploaderStatistics.java
index 2cd74ff9cf5..2aa7b341af6 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/statistics/S3AStatisticInterface.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AStatisticInterface.java
new file mode 100644
index 00000000000..836f2038d12
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AStatisticInterface.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.statistics;
+
+import org.apache.hadoop.fs.statistics.DurationTrackerFactory;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+
+/**
+ * Base Interface for S3A Statistics, as a way of managing
+ * them.
+ */
+public interface S3AStatisticInterface extends IOStatisticsSource,
+ DurationTrackerFactory {
+
+
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AStatisticsContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AStatisticsContext.java
new file mode 100644
index 00000000000..27f1398d4ea
--- /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/statistics/StatisticTypeEnum.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/StatisticTypeEnum.java
new file mode 100644
index 00000000000..ac7bd9b5e7b
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/StatisticTypeEnum.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.statistics;
+
+/**
+ * Enum of statistic types.
+ */
+public enum StatisticTypeEnum {
+
+ /**
+ * Counter.
+ */
+ TYPE_COUNTER,
+
+ /**
+ * Duration.
+ */
+ TYPE_DURATION,
+
+ /**
+ * Gauge.
+ */
+ TYPE_GAUGE,
+
+ /**
+ * Quantile.
+ */
+ TYPE_QUANTILE,
+
+}
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 00000000000..2fd7c685ccd
--- /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 00000000000..6b7eb245e49
--- /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.
+ *
+ * A lot of the methods are very terse, because S3AInstrumentation has
+ * verbose methods of similar names; the short ones always
+ * refer to the inner class and not any superclass method.
+ *
+ */
+public abstract class AbstractS3AStatisticsSource implements
+ IOStatisticsSource, DurationTrackerFactory {
+
+ private IOStatisticsStore ioStatistics;
+
+ protected AbstractS3AStatisticsSource() {
+ }
+
+ @Override
+ public IOStatisticsStore getIOStatistics() {
+ return ioStatistics;
+ }
+
+ /**
+ * Setter.
+ * this must be called in the subclass constructor with
+ * whatever
+ * @param statistics statistics to set
+ */
+ protected void setIOStatistics(final IOStatisticsStore statistics) {
+ this.ioStatistics = statistics;
+ }
+
+ /**
+ * 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 00000000000..c002a4a6dee
--- /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.
+ *
+ * See {@code com.facebook.presto.hive.s3.PrestoS3FileSystemMetricCollector}
+ * for the inspiration for this.
+ *
+ * See {@code com.amazonaws.util.AWSRequestMetrics} for metric names.
+ */
+public class AwsStatisticsCollector extends RequestMetricCollector {
+
+ /**
+ * final destination of updates.
+ */
+ private final StatisticsFromAwsSdk collector;
+
+ /**
+ * Instantiate.
+ * @param collector final destination of updates
+ */
+ public AwsStatisticsCollector(final StatisticsFromAwsSdk collector) {
+ this.collector = collector;
+ }
+
+ /**
+ * This is the callback from the AWS SDK where metrics
+ * can be collected.
+ * @param request AWS request
+ * @param response AWS response
+ */
+ @Override
+ public void collectMetrics(
+ final Request> request,
+ final Response> response) {
+
+ TimingInfo timingInfo = request.getAWSRequestMetrics().getTimingInfo();
+
+ counter(timingInfo, HttpClientRetryCount.name(),
+ collector::updateAwsRetryCount);
+ counter(timingInfo, RequestCount.name(),
+ collector::updateAwsRequestCount);
+ counter(timingInfo, ThrottleException.name(),
+ collector::updateAwsThrottleExceptionsCount);
+
+ timing(timingInfo, ClientExecuteTime.name(),
+ collector::noteAwsClientExecuteTime);
+ timing(timingInfo, HttpRequestTime.name(),
+ collector::noteAwsRequestTime);
+ timing(timingInfo, RequestMarshallTime.name(),
+ collector::noteRequestMarshallTime);
+ timing(timingInfo, RequestSigningTime.name(),
+ collector::noteRequestSigningTime);
+ timing(timingInfo, ResponseProcessingTime.name(),
+ collector::noteResponseProcessingTime);
+ }
+
+ /**
+ * Process a timing.
+ * @param timingInfo timing info
+ * @param subMeasurementName sub measurement
+ * @param durationConsumer consumer
+ */
+ private void timing(
+ TimingInfo timingInfo,
+ String subMeasurementName,
+ Consumer durationConsumer) {
+ TimingInfo t1 = timingInfo.getSubMeasurement(subMeasurementName);
+ if (t1 != null && t1.getTimeTakenMillisIfKnown() != null) {
+ durationConsumer.accept(Duration.ofMillis(
+ t1.getTimeTakenMillisIfKnown().longValue()));
+ }
+ }
+
+ /**
+ * Process a counter.
+ * @param timingInfo timing info
+ * @param subMeasurementName sub measurement
+ * @param consumer consumer
+ */
+ private void counter(
+ TimingInfo timingInfo,
+ String subMeasurementName,
+ LongConsumer consumer) {
+ Number n = timingInfo.getCounter(subMeasurementName);
+ if (n != null) {
+ consumer.accept(n.longValue());
+ }
+ }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/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 00000000000..006eb24f72c
--- /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.
+ *
+ * An interface is used to bind to the relevant fields, rather
+ * than have them passed in the constructor because some
+ * production code, specifically, DelegateToFileSystem,
+ * patches the protected field after initialization.
+ *
+ *
+ * All operations are passed through directly to that class.
+ *
+ *
+ * If an instance of FileSystem.Statistics is passed in, it
+ * will be used whenever input stream statistics are created -
+ * However, Internally always increments the statistics in the
+ * current thread.
+ *
+ *
+ * As a result, cross-thread IO will under-report.
+ *
+ *
+ * This is addressed through the stream statistics classes
+ * only updating the stats in the close() call. Provided
+ * they are closed in the worker thread, all stats collected in
+ * helper threads will be included.
+ */
+public class BondedS3AStatisticsContext implements S3AStatisticsContext {
+
+ /** Source of statistics services. */
+ private final S3AFSStatisticsSource statisticsSource;
+
+ /**
+ * Instantiate.
+ * @param statisticsSource integration binding
+ */
+ public BondedS3AStatisticsContext(
+ final S3AFSStatisticsSource statisticsSource) {
+ this.statisticsSource = statisticsSource;
+ }
+
+
+ /**
+ * Get the instrumentation from the FS integration.
+ * @return instrumentation instance.
+ */
+ private S3AInstrumentation getInstrumentation() {
+ return statisticsSource.getInstrumentation();
+ }
+
+ /**
+ * The filesystem statistics: know this is thread-local.
+ * @return FS statistics.
+ */
+ private FileSystem.Statistics getInstanceStatistics() {
+ return statisticsSource.getInstanceStatistics();
+ }
+
+ /**
+ * Get a MetastoreInstrumentation getInstrumentation() instance for this
+ * context.
+ * @return the S3Guard getInstrumentation() point.
+ */
+ @Override
+ public MetastoreInstrumentation getS3GuardInstrumentation() {
+ return getInstrumentation().getS3GuardInstrumentation();
+ }
+
+ /**
+ * Create a stream input statistics instance.
+ * 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.
+ *
+ * No-op if not defined.
+ * @param op operation
+ * @param count increment value
+ */
+ @Override
+ public void incrementCounter(Statistic op, long count) {
+ getInstrumentation().incrementCounter(op, count);
+ }
+
+ /**
+ * Increment a specific gauge.
+ *
+ * No-op if not defined.
+ * @param op operation
+ * @param count increment value
+ * @throws ClassCastException if the metric is of the wrong type
+ */
+ @Override
+ public void incrementGauge(Statistic op, long count) {
+ getInstrumentation().incrementGauge(op, count);
+ }
+
+ /**
+ * Decrement a specific gauge.
+ *
+ * No-op if not defined.
+ * @param op operation
+ * @param count increment value
+ * @throws ClassCastException if the metric is of the wrong type
+ */
+ @Override
+ public void decrementGauge(Statistic op, long count) {
+ getInstrumentation().decrementGauge(op, count);
+ }
+
+ /**
+ * Add a value to a quantiles statistic. No-op if the quantile
+ * isn't found.
+ * @param op operation to look up.
+ * @param value value to add.
+ * @throws ClassCastException if the metric is not a Quantiles.
+ */
+ @Override
+ public void addValueToQuantiles(Statistic op, long value) {
+ getInstrumentation().addValueToQuantiles(op, value);
+ }
+
+ @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/main/java/org/apache/hadoop/fs/s3a/statistics/impl/CountingChangeTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/CountingChangeTracker.java
new file mode 100644
index 00000000000..18157740c8b
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/CountingChangeTracker.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.statistics.impl;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.fs.s3a.statistics.ChangeTrackerStatistics;
+
+/**
+ * A change tracker which increments an atomic long.
+ */
+public class CountingChangeTracker implements
+ ChangeTrackerStatistics {
+
+ /**
+ * The counter which is updated on every mismatch.
+ */
+ private final AtomicLong counter;
+
+ public CountingChangeTracker(final AtomicLong counter) {
+ this.counter = counter;
+ }
+
+ public CountingChangeTracker() {
+ this(new AtomicLong());
+ }
+
+ @Override
+ public void versionMismatchError() {
+ counter.incrementAndGet();
+ }
+
+ @Override
+ public long getVersionMismatches() {
+ return counter.get();
+ }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/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 00000000000..c8cd8059208
--- /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.
+ *
+ * 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 70e4785f62e..7b6d559cf20 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.
+ *
* This takes a function to update some counter and will update
* this value when things change, so it can be bonded to arbitrary
* statistic collectors.
+ *
+ *
+ * Internally it builds a map of the relevant multipart statistics,
+ * increments as appropriate and serves this data back through
+ * the {@code IOStatisticsSource} API.
+ *
*/
-public final class S3AMultipartUploaderStatisticsImpl implements
- S3AMultipartUploaderStatistics {
+public final class S3AMultipartUploaderStatisticsImpl
+ extends AbstractS3AStatisticsSource
+ implements S3AMultipartUploaderStatistics {
/**
* The operation to increment a counter/statistic by a value.
@@ -53,16 +65,28 @@ public final class S3AMultipartUploaderStatisticsImpl implements
*/
public S3AMultipartUploaderStatisticsImpl(
final BiConsumer 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 00000000000..48b0b2bf454
--- /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.
+ *
+ * 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/statistics/impl/package-info.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/package-info.java
new file mode 100644
index 00000000000..6b94ac76752
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/package-info.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Statistics collection for the S3A connector: implementation.
+ * Not for use by anything outside the hadoop-aws source tree.
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.fs.s3a.statistics.impl;
+
+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/statistics/package-info.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/package-info.java
new file mode 100644
index 00000000000..b74bc3e2528
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/package-info.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * 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.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.fs.s3a.statistics;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java
index c7d7fda387e..6d386f250e6 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 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 740f256b62e..ae54dfee000 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 73a503ace49..b1ea45f6865 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,10 +84,22 @@ 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
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 bd6bf2f6cdb..5b6e634a631 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.
+ *
+ * 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);
- @Test
- public void testGlobScan() throws Throwable {
+ /**
+ * Parameterization.
+ */
+ @Parameterized.Parameters(name = "{0}")
+ public static Collection