> iterator = stats.iterator();
+
+ assertThat(iterator.hasNext())
+ .describedAs("iterator.hasNext()")
+ .isFalse();
+ intercept(NoSuchElementException.class, iterator::next);
+ }
+
+ @Test
+ public void testUnknownStatistic() throws Throwable {
+ assertStatisticIsUnknown(stats, "anything");
+ assertStatisticIsUntracked(stats, "anything");
+ }
+
+ @Test
+ public void testStatisticsTrackedAssertion() throws Throwable {
+ // expect an exception to be raised when an assertion
+ // is made that an unknown statistic is tracked,.
+ assertThatThrownBy(() ->
+ assertStatisticIsTracked(stats, "anything"))
+ .isInstanceOf(AssertionError.class);
+ }
+
+ @Test
+ public void testStatisticsValueAssertion() throws Throwable {
+ // expect an exception to be raised when the
+ //
+ assertThatThrownBy(() ->
+ verifyStatisticValue(stats, "anything", 0))
+ .isInstanceOf(AssertionError.class);
+ }
+
+
+
+}
diff --git a/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml b/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml
index b51053603fa7b..1ebf8587e8824 100644
--- a/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml
+++ b/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml
@@ -74,4 +74,14 @@
+
+
+
+
+
+
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java
index 276961bf8b7e1..b57e0e000b7fb 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java
@@ -23,16 +23,25 @@
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.client.builder.AwsClientBuilder;
+import com.amazonaws.metrics.RequestMetricCollector;
import com.amazonaws.services.s3.AmazonS3;
import com.amazonaws.services.s3.AmazonS3Client;
-import com.amazonaws.services.s3.S3ClientOptions;
+import com.amazonaws.services.s3.AmazonS3ClientBuilder;
+import com.amazonaws.services.s3.internal.ServiceUtils;
+import com.amazonaws.util.AwsHostNameUtils;
+import com.amazonaws.util.RuntimeHttpUtils;
+import com.google.common.annotations.VisibleForTesting;
import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.s3a.impl.statistics.AwsStatisticsCollector;
+import org.apache.hadoop.fs.s3a.impl.statistics.StatisticsFromAwsSdk;
import static org.apache.hadoop.fs.s3a.Constants.EXPERIMENTAL_AWS_INTERNAL_THROTTLING;
import static org.apache.hadoop.fs.s3a.Constants.ENDPOINT;
@@ -49,13 +58,22 @@
public class DefaultS3ClientFactory extends Configured
implements S3ClientFactory {
- protected static final Logger LOG = S3AFileSystem.LOG;
+ private static final String S3_SERVICE_NAME = "s3";
+ private static final String S3_SIGNER = "S3SignerType";
+ private static final String S3_V4_SIGNER = "AWSS3V4SignerType";
+
+ /**
+ * Subclasses refer to this.
+ */
+ protected static final Logger LOG =
+ LoggerFactory.getLogger(DefaultS3ClientFactory.class);
@Override
public AmazonS3 createS3Client(URI name,
final String bucket,
final AWSCredentialsProvider credentials,
- final String userAgentSuffix) throws IOException {
+ final String userAgentSuffix,
+ final StatisticsFromAwsSdk statisticsFromAwsSdk) throws IOException {
Configuration conf = getConf();
final ClientConfiguration awsConf = S3AUtils
.createAwsConf(conf, bucket, Constants.AWS_SERVICE_IDENTIFIER_S3);
@@ -72,8 +90,17 @@ public AmazonS3 createS3Client(URI name,
if (!StringUtils.isEmpty(userAgentSuffix)) {
awsConf.setUserAgentSuffix(userAgentSuffix);
}
- return configureAmazonS3Client(
- newAmazonS3Client(credentials, awsConf), conf);
+ // optional metrics
+ RequestMetricCollector metrics = statisticsFromAwsSdk != null
+ ? new AwsStatisticsCollector(statisticsFromAwsSdk)
+ : null;
+
+ return newAmazonS3Client(
+ credentials,
+ awsConf,
+ metrics,
+ conf.getTrimmed(ENDPOINT, ""),
+ conf.getBoolean(PATH_STYLE_ACCESS, false));
}
/**
@@ -81,67 +108,107 @@ public AmazonS3 createS3Client(URI name,
* Override this to provide an extended version of the client
* @param credentials credentials to use
* @param awsConf AWS configuration
- * @return new AmazonS3 client
+ * @param metrics metrics collector or null
+ * @param endpoint endpoint string; may be ""
+ * @param pathStyleAccess enable path style access?
+ * @return new AmazonS3 client
*/
protected AmazonS3 newAmazonS3Client(
- AWSCredentialsProvider credentials, ClientConfiguration awsConf) {
- return new AmazonS3Client(credentials, awsConf);
+ final AWSCredentialsProvider credentials,
+ final ClientConfiguration awsConf,
+ final RequestMetricCollector metrics,
+ final String endpoint,
+ final boolean pathStyleAccess) {
+ AmazonS3ClientBuilder b = AmazonS3Client.builder();
+ b.withCredentials(credentials);
+ b.withClientConfiguration(awsConf);
+ b.withPathStyleAccessEnabled(pathStyleAccess);
+ if (metrics != null) {
+ b.withMetricsCollector(metrics);
+ }
+
+ // endpoint set up is a PITA
+ // client.setEndpoint("") is no longer available
+ AwsClientBuilder.EndpointConfiguration epr
+ = createEndpointConfiguration(endpoint, awsConf);
+ if (epr != null) {
+ // an endpoint binding was constructed: use it.
+ b.withEndpointConfiguration(epr);
+ }
+ final AmazonS3 client = b.build();
+ // if this worked life would be so much simpler
+ // client.setEndpoint(endpoint);
+ return client;
}
/**
- * Configure S3 client from the Hadoop configuration.
- *
- * This includes: endpoint, Path Access and possibly other
- * options.
+ * Patch a classically-constructed s3 instance's endpoint.
+ * @param s3 S3 client
+ * @param endpoint possibly empty endpoint.
*
- * @param conf Hadoop configuration
- * @return S3 client
* @throws IllegalArgumentException if misconfigured
*/
- private static AmazonS3 configureAmazonS3Client(AmazonS3 s3,
- Configuration conf)
+ protected static AmazonS3 setEndpoint(AmazonS3 s3,
+ String endpoint)
throws IllegalArgumentException {
- String endPoint = conf.getTrimmed(ENDPOINT, "");
- if (!endPoint.isEmpty()) {
- try {
- s3.setEndpoint(endPoint);
+ if (!endpoint.isEmpty()) {
+ try {
+ s3.setEndpoint(endpoint);
} catch (IllegalArgumentException e) {
- String msg = "Incorrect endpoint: " + e.getMessage();
+ String msg = "Incorrect endpoint: " + e.getMessage();
LOG.error(msg);
throw new IllegalArgumentException(msg, e);
}
}
- return applyS3ClientOptions(s3, conf);
+ return s3;
}
/**
- * Perform any tuning of the {@code S3ClientOptions} settings based on
- * the Hadoop configuration.
- * This is different from the general AWS configuration creation as
- * it is unique to S3 connections.
+ * Given an endpoint string, return an endpoint config, or null, if none
+ * is needed.
+ * This is a pretty painful piece of code. It is trying to replicate
+ * what AwsClient.setEndpoint() does, because you can't
+ * call that setter on an AwsClient constructed via
+ * the builder, and you can't pass a metrics collector
+ * down except through the builder.
+ *
+ * Note also that AWS signing is a mystery which nobody fully
+ * understands, especially given all problems surface in a
+ * "400 bad request" response, which, like all security systems,
+ * provides minimal diagnostics out of fear of leaking
+ * secrets.
*
- * The {@link Constants#PATH_STYLE_ACCESS} option enables path-style access
- * to S3 buckets if configured. By default, the
- * behavior is to use virtual hosted-style access with URIs of the form
- * {@code http://bucketname.s3.amazonaws.com}
- * Enabling path-style access and a
- * region-specific endpoint switches the behavior to use URIs of the form
- * {@code http://s3-eu-west-1.amazonaws.com/bucketname}.
- * It is common to use this when connecting to private S3 servers, as it
- * avoids the need to play with DNS entries.
- * @param s3 S3 client
- * @param conf Hadoop configuration
- * @return the S3 client
+ * @param endpoint possibly null endpoint.
+ * @param awsConf config to build the URI from.
+ * @return a configuration for the S3 client builder.
*/
- private static AmazonS3 applyS3ClientOptions(AmazonS3 s3,
- Configuration conf) {
- final boolean pathStyleAccess = conf.getBoolean(PATH_STYLE_ACCESS, false);
- if (pathStyleAccess) {
- LOG.debug("Enabling path style access!");
- s3.setS3ClientOptions(S3ClientOptions.builder()
- .setPathStyleAccess(true)
- .build());
+ @VisibleForTesting
+ public static AwsClientBuilder.EndpointConfiguration
+ createEndpointConfiguration(
+ final String endpoint, final ClientConfiguration awsConf) {
+ LOG.debug("Creating endpoint configuration for {}", endpoint);
+ if (endpoint == null || endpoint.isEmpty()) {
+ // the default endpoint...we should be using null at this point.
+ LOG.debug("Using default endpoint -no need to generate a configuration");
+ return null;
}
- return s3;
+
+ final URI epr = RuntimeHttpUtils.toUri(endpoint, awsConf);
+ LOG.debug("Endpoint URI = {}", epr);
+
+ String region;
+ if (!ServiceUtils.isS3USStandardEndpoint(endpoint)) {
+ LOG.debug("Endpoint {} is not the default; parsing", epr);
+ region = AwsHostNameUtils.parseRegion(
+ epr.getHost(),
+ S3_SERVICE_NAME);
+ } else {
+ // US-east, set region == null.
+ LOG.debug("Endpoint {} is the standard one; declare region as null", epr);
+ region = null;
+ }
+ LOG.debug("Region for endpoint {}, URI {} is determined as {}",
+ endpoint, epr, region);
+ return new AwsClientBuilder.EndpointConfiguration(endpoint, region);
}
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java
index 932c472f5bea2..96cab1b65be3e 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java
@@ -20,7 +20,9 @@
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.metrics.RequestMetricCollector;
import com.amazonaws.services.s3.AmazonS3;
+import com.amazonaws.services.s3.S3ClientOptions;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
@@ -40,12 +42,27 @@ public class InconsistentS3ClientFactory extends DefaultS3ClientFactory {
* Logs a warning that this is being done.
* @param credentials credentials to use
* @param awsConf AWS configuration
+ * @param metrics
+ * @param endpoint
+ * @param pathStyleAccess
* @return an inconsistent client.
*/
@Override
protected AmazonS3 newAmazonS3Client(AWSCredentialsProvider credentials,
- ClientConfiguration awsConf) {
+ ClientConfiguration awsConf,
+ final RequestMetricCollector metrics,
+ final String endpoint,
+ final boolean pathStyleAccess) {
LOG.warn("** FAILURE INJECTION ENABLED. Do not run in production! **");
- return new InconsistentAmazonS3Client(credentials, awsConf, getConf());
+ InconsistentAmazonS3Client s3
+ = new InconsistentAmazonS3Client(credentials, awsConf, getConf());
+ if (pathStyleAccess) {
+ LOG.debug("Enabling path style access!");
+ s3.setS3ClientOptions(S3ClientOptions.builder()
+ .setPathStyleAccess(true)
+ .build());
+ }
+ setEndpoint(s3, endpoint);
+ return s3;
}
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
index 66cac99de7baa..0bfa069beb1de 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
@@ -49,6 +49,11 @@
import org.apache.hadoop.fs.StreamCapabilities;
import org.apache.hadoop.fs.s3a.commit.CommitConstants;
import org.apache.hadoop.fs.s3a.commit.PutTracker;
+import org.apache.hadoop.fs.s3a.impl.statistics.BlockOutputStreamStatistics;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsLogging;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+import org.apache.hadoop.fs.statistics.impl.EmptyIOStatistics;
import org.apache.hadoop.util.Progressable;
import static org.apache.hadoop.fs.s3a.S3AUtils.*;
@@ -67,7 +72,7 @@
@InterfaceAudience.Private
@InterfaceStability.Unstable
class S3ABlockOutputStream extends OutputStream implements
- StreamCapabilities {
+ StreamCapabilities, IOStatisticsSource {
private static final Logger LOG =
LoggerFactory.getLogger(S3ABlockOutputStream.class);
@@ -81,6 +86,9 @@ class S3ABlockOutputStream extends OutputStream implements
/** Size of all blocks. */
private final int blockSize;
+ /** IO Statistics. */
+ private final IOStatistics iostatistics;
+
/** Total bytes for uploads submitted so far. */
private long bytesSubmitted;
@@ -109,7 +117,7 @@ class S3ABlockOutputStream extends OutputStream implements
private long blockCount = 0;
/** Statistics to build up. */
- private final S3AInstrumentation.OutputStreamStatistics statistics;
+ private final BlockOutputStreamStatistics statistics;
/**
* Write operation helper; encapsulation of the filesystem operations.
@@ -146,7 +154,7 @@ class S3ABlockOutputStream extends OutputStream implements
Progressable progress,
long blockSize,
S3ADataBlocks.BlockFactory blockFactory,
- S3AInstrumentation.OutputStreamStatistics statistics,
+ BlockOutputStreamStatistics statistics,
WriteOperationHelper writeOperationHelper,
PutTracker putTracker)
throws IOException {
@@ -155,6 +163,10 @@ class S3ABlockOutputStream extends OutputStream implements
this.blockFactory = blockFactory;
this.blockSize = (int) blockSize;
this.statistics = statistics;
+ // test instantiations may not provide statistics;
+ iostatistics = statistics != null
+ ? statistics.createIOStatistics()
+ : EmptyIOStatistics.getInstance();
this.writeOperationHelper = writeOperationHelper;
this.putTracker = putTracker;
Preconditions.checkArgument(blockSize >= Constants.MULTIPART_MIN_SIZE,
@@ -282,6 +294,7 @@ public synchronized void write(byte[] source, int offset, int len)
if (len == 0) {
return;
}
+ statistics.writeBytes(len);
S3ADataBlocks.DataBlock block = createBlockIfNeeded();
int written = block.write(source, offset, len);
int remainingCapacity = block.remainingCapacity();
@@ -466,6 +479,7 @@ public String toString() {
if (block != null) {
sb.append(", activeBlock=").append(block);
}
+ sb.append(IOStatisticsLogging.iostatisticsSourceToString(this));
sb.append('}');
return sb.toString();
}
@@ -486,7 +500,7 @@ private long now() {
* Get the statistics for this stream.
* @return stream statistics
*/
- S3AInstrumentation.OutputStreamStatistics getStatistics() {
+ BlockOutputStreamStatistics getStatistics() {
return statistics;
}
@@ -518,6 +532,11 @@ public boolean hasCapability(String capability) {
}
}
+ @Override
+ public IOStatistics getIOStatistics() {
+ return iostatistics;
+ }
+
/**
* Multiple partition upload.
*/
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java
index 156defb7ca031..c0de7bdaa910c 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java
@@ -37,6 +37,7 @@
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.FSExceptionMessages;
+import org.apache.hadoop.fs.s3a.impl.statistics.BlockOutputStreamStatistics;
import org.apache.hadoop.util.DirectBufferPool;
import static org.apache.hadoop.fs.s3a.S3ADataBlocks.DataBlock.DestState.*;
@@ -180,7 +181,7 @@ protected BlockFactory(S3AFileSystem owner) {
* @return a new block.
*/
abstract DataBlock create(long index, int limit,
- S3AInstrumentation.OutputStreamStatistics statistics)
+ BlockOutputStreamStatistics statistics)
throws IOException;
/**
@@ -210,10 +211,10 @@ enum DestState {Writing, Upload, Closed}
private volatile DestState state = Writing;
protected final long index;
- protected final S3AInstrumentation.OutputStreamStatistics statistics;
+ protected final BlockOutputStreamStatistics statistics;
protected DataBlock(long index,
- S3AInstrumentation.OutputStreamStatistics statistics) {
+ BlockOutputStreamStatistics statistics) {
this.index = index;
this.statistics = statistics;
}
@@ -387,7 +388,7 @@ static class ArrayBlockFactory extends BlockFactory {
@Override
DataBlock create(long index, int limit,
- S3AInstrumentation.OutputStreamStatistics statistics)
+ BlockOutputStreamStatistics statistics)
throws IOException {
return new ByteArrayBlock(0, limit, statistics);
}
@@ -432,7 +433,7 @@ static class ByteArrayBlock extends DataBlock {
ByteArrayBlock(long index,
int limit,
- S3AInstrumentation.OutputStreamStatistics statistics) {
+ BlockOutputStreamStatistics statistics) {
super(index, statistics);
this.limit = limit;
buffer = new S3AByteArrayOutputStream(limit);
@@ -510,7 +511,7 @@ static class ByteBufferBlockFactory extends BlockFactory {
@Override
ByteBufferBlock create(long index, int limit,
- S3AInstrumentation.OutputStreamStatistics statistics)
+ BlockOutputStreamStatistics statistics)
throws IOException {
return new ByteBufferBlock(index, limit, statistics);
}
@@ -560,7 +561,7 @@ class ByteBufferBlock extends DataBlock {
*/
ByteBufferBlock(long index,
int bufferSize,
- S3AInstrumentation.OutputStreamStatistics statistics) {
+ BlockOutputStreamStatistics statistics) {
super(index, statistics);
this.bufferSize = bufferSize;
blockBuffer = requestBuffer(bufferSize);
@@ -805,7 +806,7 @@ static class DiskBlockFactory extends BlockFactory {
@Override
DataBlock create(long index,
int limit,
- S3AInstrumentation.OutputStreamStatistics statistics)
+ BlockOutputStreamStatistics statistics)
throws IOException {
File destFile = getOwner()
.createTmpFileForWrite(String.format("s3ablock-%04d-", index),
@@ -829,7 +830,7 @@ static class DiskBlock extends DataBlock {
DiskBlock(File bufferFile,
int limit,
long index,
- S3AInstrumentation.OutputStreamStatistics statistics)
+ BlockOutputStreamStatistics statistics)
throws FileNotFoundException {
super(index, statistics);
this.limit = limit;
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
index 6d2b3a84ca702..3eefaec1bb253 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
@@ -110,8 +110,14 @@
import org.apache.hadoop.fs.s3a.impl.RenameOperation;
import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum;
import org.apache.hadoop.fs.s3a.impl.StoreContext;
+import org.apache.hadoop.fs.s3a.impl.statistics.CommitterStatistics;
+import org.apache.hadoop.fs.s3a.impl.statistics.IntegratedS3AStatisticsContext;
+import org.apache.hadoop.fs.s3a.impl.statistics.S3AStatisticsContext;
import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
import org.apache.hadoop.fs.s3a.select.InternalSelectConstants;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+import org.apache.hadoop.fs.statistics.impl.IOStatisticsImplementationHelper;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.token.TokenIdentifier;
@@ -195,7 +201,8 @@
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class S3AFileSystem extends FileSystem implements StreamCapabilities,
- AWSPolicyProvider, DelegationTokenProvider {
+ AWSPolicyProvider, DelegationTokenProvider,
+ IOStatisticsSource {
/**
* Default blocksize as used in blocksize and FS status queries.
*/
@@ -250,6 +257,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
private S3AInstrumentation instrumentation;
private final S3AStorageStatistics storageStatistics =
createStorageStatistics();
+
+ private S3AStatisticsContext statisticsContext;
+
private long readAhead;
private S3AInputPolicy inputPolicy;
private ChangeDetectionPolicy changeDetectionPolicy;
@@ -343,6 +353,7 @@ public void initialize(URI name, Configuration originalConf)
invoker = new Invoker(new S3ARetryPolicy(getConf()), onRetry);
instrumentation = new S3AInstrumentation(uri);
+ initializeStatisticsBinding();
// Username is the current user at the time the FS was instantiated.
owner = UserGroupInformation.getCurrentUser();
@@ -352,7 +363,8 @@ public void initialize(URI name, Configuration originalConf)
s3guardInvoker = new Invoker(new S3GuardExistsRetryPolicy(getConf()),
onRetry);
- writeHelper = new WriteOperationHelper(this, getConf());
+ writeHelper = new WriteOperationHelper(this, getConf(),
+ statisticsContext);
failOnMetadataWriteError = conf.getBoolean(FAIL_ON_METADATA_WRITE_ERROR,
FAIL_ON_METADATA_WRITE_ERROR_DEFAULT);
@@ -502,6 +514,31 @@ private void doBucketProbing() throws IOException {
}
}
+ /**
+ * Initialize the statistics binding.
+ * This is done by creating an {@code IntegratedS3AStatisticsContext}
+ * with callbacks to get the FS's instrumentation and FileSystem.statistics
+ * field; the latter may change after {@link #initialize(URI, Configuration)},
+ * so needs to be dynamically adapted.
+ * Protected so that (mock) subclasses can replace it with a
+ * different statistics binding, if desired.
+ */
+ protected void initializeStatisticsBinding() {
+ statisticsContext = new IntegratedS3AStatisticsContext(
+ new IntegratedS3AStatisticsContext.S3AFSStatisticsSource() {
+
+ @Override
+ public S3AInstrumentation getInstrumentation() {
+ return S3AFileSystem.this.getInstrumentation();
+ }
+
+ @Override
+ public Statistics getInstanceStatistics() {
+ return S3AFileSystem.this.statistics;
+ }
+ });
+ }
+
/**
* Initialize the thread pool.
* This must be re-invoked after replacing the S3Client during test
@@ -581,6 +618,7 @@ protected void verifyBucketExistsV2()
* Get S3A Instrumentation. For test purposes.
* @return this instance's instrumentation.
*/
+ @VisibleForTesting
public S3AInstrumentation getInstrumentation() {
return instrumentation;
}
@@ -641,7 +679,8 @@ private void bindAWSClient(URI name, boolean dtEnabled) throws IOException {
S3ClientFactory.class);
s3 = ReflectionUtils.newInstance(s3ClientFactoryClass, conf)
- .createS3Client(getUri(), bucket, credentials, uaSuffix);
+ .createS3Client(getUri(), bucket, credentials, uaSuffix,
+ statisticsContext.newStatisticsFromAwsSdk());
}
/**
@@ -1139,7 +1178,7 @@ private S3AReadOpContext createReadContext(
invoker,
s3guardInvoker,
statistics,
- instrumentation,
+ statisticsContext,
fileStatus,
seekPolicy,
changePolicy,
@@ -1244,7 +1283,7 @@ public FSDataOutputStream create(Path f, FsPermission permission,
progress,
partSize,
blockFactory,
- instrumentation.newOutputStreamStatistics(statistics),
+ statisticsContext.newOutputStreamStatistics(),
getWriteOperationHelper(),
putTracker),
null);
@@ -1706,8 +1745,7 @@ protected void incrementStatistic(Statistic statistic) {
* @param count the count to increment
*/
protected void incrementStatistic(Statistic statistic, long count) {
- instrumentation.incrementCounter(statistic, count);
- storageStatistics.incrementCounter(statistic, count);
+ statisticsContext.incrementCounter(statistic, count);
}
/**
@@ -1716,7 +1754,7 @@ protected void incrementStatistic(Statistic statistic, long count) {
* @param count the count to decrement
*/
protected void decrementGauge(Statistic statistic, long count) {
- instrumentation.decrementGauge(statistic, count);
+ statisticsContext.decrementGauge(statistic, count);
}
/**
@@ -1725,7 +1763,7 @@ protected void decrementGauge(Statistic statistic, long count) {
* @param count the count to increment
*/
protected void incrementGauge(Statistic statistic, long count) {
- instrumentation.incrementGauge(statistic, count);
+ statisticsContext.incrementGauge(statistic, count);
}
/**
@@ -1738,6 +1776,7 @@ public void operationRetried(Exception ex) {
if (isThrottleException(ex)) {
operationThrottled(false);
} else {
+ incrementStatistic(STORE_IO_RETRY);
incrementStatistic(IGNORED_ERRORS);
}
}
@@ -1789,11 +1828,11 @@ private void operationThrottled(boolean metastore) {
LOG.debug("Request throttled on {}", metastore ? "S3": "DynamoDB");
if (metastore) {
incrementStatistic(S3GUARD_METADATASTORE_THROTTLED);
- instrumentation.addValueToQuantiles(S3GUARD_METADATASTORE_THROTTLE_RATE,
+ statisticsContext.addValueToQuantiles(S3GUARD_METADATASTORE_THROTTLE_RATE,
1);
} else {
incrementStatistic(STORE_IO_THROTTLED);
- instrumentation.addValueToQuantiles(STORE_IO_THROTTLE_RATE, 1);
+ statisticsContext.addValueToQuantiles(STORE_IO_THROTTLE_RATE, 1);
}
}
@@ -1806,6 +1845,16 @@ public S3AStorageStatistics getStorageStatistics() {
return storageStatistics;
}
+ /**
+ * Get this filesystem's storage statistics as IO Statistics.
+ * @return statistics
+ */
+ @Override
+ public IOStatistics getIOStatistics() {
+ return IOStatisticsImplementationHelper.createFromStorageStatistics(
+ storageStatistics).getIOStatistics();
+ }
+
/**
* Request object metadata; increments counters in the process.
* Retry policy: retry untranslated.
@@ -3441,8 +3490,9 @@ private CopyResult copyFile(String srcKey, String dstKey, long size,
ChangeTracker changeTracker = new ChangeTracker(
keyToQualifiedPath(srcKey).toString(),
changeDetectionPolicy,
- readContext.instrumentation.newInputStreamStatistics()
- .getVersionMismatchCounter(),
+ readContext.getS3AStatisticsContext()
+ .newInputStreamStatistics()
+ .getChangeTrackerStatistics(),
srcAttributes);
String action = "copyFile(" + srcKey + ", " + dstKey + ")";
@@ -4448,8 +4498,8 @@ void abortMultipartUpload(MultipartUpload upload) {
* Create a new instance of the committer statistics.
* @return a new committer statistics instance
*/
- public S3AInstrumentation.CommitterStatistics newCommitterStatistics() {
- return instrumentation.newCommitterStatistics();
+ public CommitterStatistics newCommitterStatistics() {
+ return statisticsContext.newCommitterStatistics();
}
@SuppressWarnings("deprecation")
@@ -4563,8 +4613,9 @@ private FSDataInputStream select(final Path source,
ChangeTracker changeTracker =
new ChangeTracker(uri.toString(),
changeDetectionPolicy,
- readContext.instrumentation.newInputStreamStatistics()
- .getVersionMismatchCounter(),
+ readContext.getS3AStatisticsContext()
+ .newInputStreamStatistics()
+ .getChangeTrackerStatistics(),
objectAttributes);
// will retry internally if wrong version detected
@@ -4720,7 +4771,7 @@ public StoreContext createStoreContext() {
boundedThreadPool,
executorCapacity,
invoker,
- getInstrumentation(),
+ statisticsContext,
getStorageStatistics(),
getInputPolicy(),
changeDetectionPolicy,
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
index 9c8b9ae7a156e..2596f5bb0cf76 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
@@ -33,10 +33,13 @@
import org.apache.hadoop.fs.CanSetReadahead;
import org.apache.hadoop.fs.CanUnbuffer;
import org.apache.hadoop.fs.FSExceptionMessages;
-import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.fs.s3a.impl.statistics.S3AInputStreamStatistics;
+import org.apache.hadoop.fs.s3a.impl.ChangeTracker;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
import org.apache.hadoop.fs.PathIOException;
import org.apache.hadoop.fs.StreamCapabilities;
-import org.apache.hadoop.fs.s3a.impl.ChangeTracker;
+import org.apache.hadoop.fs.FSInputStream;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -68,7 +71,7 @@
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class S3AInputStream extends FSInputStream implements CanSetReadahead,
- CanUnbuffer, StreamCapabilities {
+ CanUnbuffer, StreamCapabilities, IOStatisticsSource {
public static final String E_NEGATIVE_READAHEAD_VALUE
= "Negative readahead value";
@@ -97,7 +100,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
private final String uri;
private static final Logger LOG =
LoggerFactory.getLogger(S3AInputStream.class);
- private final S3AInstrumentation.InputStreamStatistics streamStatistics;
+ private final S3AInputStreamStatistics streamStatistics;
private S3AEncryptionMethods serverSideEncryptionAlgorithm;
private String serverSideEncryptionKey;
private S3AInputPolicy inputPolicy;
@@ -123,6 +126,11 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
/** change tracker. */
private final ChangeTracker changeTracker;
+ /**
+ * IOStatistics report.
+ */
+ private final IOStatistics ioStatistics;
+
/**
* Create the stream.
* This does not attempt to open it; that is only done on the first
@@ -146,13 +154,15 @@ public S3AInputStream(S3AReadOpContext ctx,
this.contentLength = l;
this.client = client;
this.uri = "s3a://" + this.bucket + "/" + this.key;
- this.streamStatistics = ctx.instrumentation.newInputStreamStatistics();
+ this.streamStatistics = ctx.getS3AStatisticsContext()
+ .newInputStreamStatistics();
+ this.ioStatistics = streamStatistics.createIOStatistics();
this.serverSideEncryptionAlgorithm =
s3Attributes.getServerSideEncryptionAlgorithm();
this.serverSideEncryptionKey = s3Attributes.getServerSideEncryptionKey();
this.changeTracker = new ChangeTracker(uri,
ctx.getChangeDetectionPolicy(),
- streamStatistics.getVersionMismatchCounter(),
+ streamStatistics.getChangeTrackerStatistics(),
s3Attributes);
setInputPolicy(ctx.getInputPolicy());
setReadahead(ctx.getReadahead());
@@ -290,8 +300,6 @@ private void seekInStream(long targetPos, long length) throws IOException {
long skipped = wrappedStream.skip(diff);
if (skipped > 0) {
pos += skipped;
- // as these bytes have been read, they are included in the counter
- incrementBytesRead(diff);
}
if (pos == targetPos) {
@@ -348,7 +356,7 @@ private void lazySeek(long targetPos, long len) throws IOException {
// open. After that, an exception generally means the file has changed
// and there is no point retrying anymore.
Invoker invoker = context.getReadInvoker();
- invoker.maybeRetry(streamStatistics.openOperations == 0,
+ invoker.maybeRetry(streamStatistics.getOpenOperations() == 0,
"lazySeek", pathStr, true,
() -> {
//For lazy seek
@@ -747,7 +755,7 @@ public void readFully(long position, byte[] buffer, int offset, int length)
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
- public S3AInstrumentation.InputStreamStatistics getS3AStreamStatistics() {
+ public S3AInputStreamStatistics getS3AStreamStatistics() {
return streamStatistics;
}
@@ -850,4 +858,9 @@ public boolean hasCapability(String capability) {
boolean isObjectStreamOpen() {
return wrappedStream != null;
}
+
+ @Override
+ public IOStatistics getIOStatistics() {
+ return ioStatistics;
+ }
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
index b9918b5098946..6005dfeb48ae2 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
@@ -18,14 +18,27 @@
package org.apache.hadoop.fs.s3a;
+import javax.annotation.Nullable;
+
import com.google.common.annotations.VisibleForTesting;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.FileSystem.Statistics;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.s3a.impl.statistics.ChangeTrackerStatistics;
+import org.apache.hadoop.fs.s3a.impl.statistics.CommitterStatistics;
+import org.apache.hadoop.fs.s3a.impl.statistics.CountersAndGauges;
+import org.apache.hadoop.fs.s3a.impl.statistics.CountingChangeTracker;
+import org.apache.hadoop.fs.s3a.impl.statistics.DelegationTokenStatistics;
+import org.apache.hadoop.fs.s3a.impl.statistics.S3AInputStreamStatistics;
+import org.apache.hadoop.fs.s3a.impl.statistics.BlockOutputStreamStatistics;
+import org.apache.hadoop.fs.s3a.impl.statistics.StatisticsFromAwsSdk;
import org.apache.hadoop.fs.s3a.s3guard.MetastoreInstrumentation;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.StreamStatisticNames;
+import org.apache.hadoop.fs.statistics.impl.DynamicIOStatisticsBuilder;
import org.apache.hadoop.metrics2.AbstractMetric;
import org.apache.hadoop.metrics2.MetricStringBuilder;
import org.apache.hadoop.metrics2.MetricsCollector;
@@ -43,12 +56,14 @@
import java.io.Closeable;
import java.net.URI;
+import java.time.Duration;
import java.util.HashMap;
import java.util.Map;
import java.util.UUID;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsImplementationHelper.createDynamicIOStatistics;
import static org.apache.hadoop.fs.s3a.Statistic.*;
/**
@@ -63,32 +78,33 @@
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
-public class S3AInstrumentation implements Closeable, MetricsSource {
+public class S3AInstrumentation implements Closeable, MetricsSource,
+ CountersAndGauges {
private static final Logger LOG = LoggerFactory.getLogger(
S3AInstrumentation.class);
private static final String METRICS_SOURCE_BASENAME = "S3AMetrics";
/**
- * {@value #METRICS_SYSTEM_NAME} The name of the s3a-specific metrics
+ * {@value} The name of the s3a-specific metrics
* system instance used for s3a metrics.
*/
public static final String METRICS_SYSTEM_NAME = "s3a-file-system";
/**
- * {@value #CONTEXT} Currently all s3a metrics are placed in a single
+ * {@value} Currently all s3a metrics are placed in a single
* "context". Distinct contexts may be used in the future.
*/
public static final String CONTEXT = "s3aFileSystem";
/**
- * {@value #METRIC_TAG_FILESYSTEM_ID} The name of a field added to metrics
+ * {@value} The name of a field added to metrics
* records that uniquely identifies a specific FileSystem instance.
*/
public static final String METRIC_TAG_FILESYSTEM_ID = "s3aFileSystemId";
/**
- * {@value #METRIC_TAG_BUCKET} The name of a field added to metrics records
+ * {@value} The name of a field added to metrics records
* that indicates the hostname portion of the FS URL.
*/
public static final String METRIC_TAG_BUCKET = "bucket";
@@ -192,6 +208,8 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
S3GUARD_METADATASTORE_THROTTLED,
S3GUARD_METADATASTORE_AUTHORITATIVE_DIRECTORIES_UPDATED,
STORE_IO_THROTTLED,
+ STORE_IO_REQUEST,
+ STORE_IO_RETRY,
DELEGATION_TOKENS_ISSUED,
FILES_DELETE_REJECTED
};
@@ -561,9 +579,11 @@ public void decrementGauge(Statistic op, long count) {
/**
* Create a stream input statistics instance.
* @return the new instance
+ * @param filesystemStatistics FS Stats.
*/
- public InputStreamStatistics newInputStreamStatistics() {
- return new InputStreamStatistics();
+ public S3AInputStreamStatistics newInputStreamStatistics(
+ final FileSystem.Statistics filesystemStatistics) {
+ return new InputStreamStatisticsImpl(filesystemStatistics);
}
/**
@@ -579,8 +599,8 @@ public MetastoreInstrumentation getS3GuardInstrumentation() {
* Create a new instance of the committer statistics.
* @return a new committer statistics instance
*/
- CommitterStatistics newCommitterStatistics() {
- return new CommitterStatistics();
+ public CommitterStatistics newCommitterStatistics() {
+ return new CommitterStatisticsImpl();
}
/**
@@ -588,23 +608,23 @@ CommitterStatistics newCommitterStatistics() {
* the filesystem-wide statistics.
* @param statistics stream statistics
*/
- private void mergeInputStreamStatistics(InputStreamStatistics statistics) {
- streamOpenOperations.incr(statistics.openOperations);
- streamCloseOperations.incr(statistics.closeOperations);
- streamClosed.incr(statistics.closed);
- streamAborted.incr(statistics.aborted);
- streamSeekOperations.incr(statistics.seekOperations);
- streamReadExceptions.incr(statistics.readExceptions);
- streamForwardSeekOperations.incr(statistics.forwardSeekOperations);
- streamBytesSkippedOnSeek.incr(statistics.bytesSkippedOnSeek);
- streamBackwardSeekOperations.incr(statistics.backwardSeekOperations);
- streamBytesBackwardsOnSeek.incr(statistics.bytesBackwardsOnSeek);
- streamBytesRead.incr(statistics.bytesRead);
- streamReadOperations.incr(statistics.readOperations);
- streamReadFullyOperations.incr(statistics.readFullyOperations);
- streamReadsIncomplete.incr(statistics.readsIncomplete);
- streamBytesReadInClose.incr(statistics.bytesReadInClose);
- streamBytesDiscardedInAbort.incr(statistics.bytesDiscardedInAbort);
+ private void mergeInputStreamStatistics(InputStreamStatisticsImpl statistics) {
+ streamOpenOperations.incr(statistics.openOperations.get());
+ streamCloseOperations.incr(statistics.closeOperations.get());
+ streamClosed.incr(statistics.closed.get());
+ streamAborted.incr(statistics.aborted.get());
+ streamSeekOperations.incr(statistics.seekOperations.get());
+ streamReadExceptions.incr(statistics.readExceptions.get());
+ streamForwardSeekOperations.incr(statistics.forwardSeekOperations.get());
+ streamBytesSkippedOnSeek.incr(statistics.bytesSkippedOnSeek.get());
+ streamBackwardSeekOperations.incr(statistics.backwardSeekOperations.get());
+ streamBytesBackwardsOnSeek.incr(statistics.bytesBackwardsOnSeek.get());
+ streamBytesRead.incr(statistics.bytesRead.get());
+ streamReadOperations.incr(statistics.readOperations.get());
+ streamReadFullyOperations.incr(statistics.readFullyOperations.get());
+ streamReadsIncomplete.incr(statistics.readsIncomplete.get());
+ streamBytesReadInClose.incr(statistics.bytesReadInClose.get());
+ streamBytesDiscardedInAbort.incr(statistics.bytesDiscardedInAbort.get());
incrementCounter(STREAM_READ_VERSION_MISMATCHES,
statistics.versionMismatches.get());
}
@@ -622,7 +642,8 @@ public void close() {
throttleRateQuantile.stop();
s3GuardThrottleRateQuantile.stop();
metricsSystem.unregisterSource(metricsSourceName);
- int activeSources = --metricsSourceActiveCounter;
+ metricsSourceActiveCounter--;
+ int activeSources = metricsSourceActiveCounter;
if (activeSources == 0) {
LOG.debug("Shutting down metrics publisher");
metricsSystem.publishMetricsNow();
@@ -634,35 +655,44 @@ public void close() {
/**
* Statistics updated by an input stream during its actual operation.
- * These counters not thread-safe and are for use in a single instance
- * of a stream.
+ * These counters are marked as volatile so that IOStatistics on the stream
+ * will get the latest values.
+ * They are only to be incremented within synchronized blocks.
*/
- @InterfaceAudience.Private
- @InterfaceStability.Unstable
- public final class InputStreamStatistics implements AutoCloseable {
- public long openOperations;
- public long closeOperations;
- public long closed;
- public long aborted;
- public long seekOperations;
- public long readExceptions;
- public long forwardSeekOperations;
- public long backwardSeekOperations;
- public long bytesRead;
- public long bytesSkippedOnSeek;
- public long bytesBackwardsOnSeek;
- public long readOperations;
- public long readFullyOperations;
- public long readsIncomplete;
- public long bytesReadInClose;
- public long bytesDiscardedInAbort;
- public long policySetCount;
- public long inputPolicy;
- /** This is atomic so that it can be passed as a reference. */
+ private final class InputStreamStatisticsImpl implements
+ S3AInputStreamStatistics {
+
+ /**
+ * Distance used when incrementing FS stats.
+ */
+ private static final int DISTANCE = 5;
+
+ private final FileSystem.Statistics filesystemStatistics;
+
+ public final AtomicLong openOperations= new AtomicLong(0);
+ public final AtomicLong closeOperations= new AtomicLong(0);
+ public final AtomicLong closed= new AtomicLong(0);
+ public final AtomicLong aborted= new AtomicLong(0);
+ public final AtomicLong seekOperations= new AtomicLong(0);
+ public final AtomicLong readExceptions= new AtomicLong(0);
+ public final AtomicLong forwardSeekOperations= new AtomicLong(0);
+ public final AtomicLong backwardSeekOperations= new AtomicLong(0);
+ public final AtomicLong bytesRead= new AtomicLong(0);
+ public final AtomicLong bytesSkippedOnSeek= new AtomicLong(0);
+ public final AtomicLong bytesBackwardsOnSeek= new AtomicLong(0);
+ public final AtomicLong readOperations= new AtomicLong(0);
+ public final AtomicLong readFullyOperations= new AtomicLong(0);
+ public final AtomicLong readsIncomplete= new AtomicLong(0);
+ public final AtomicLong bytesReadInClose= new AtomicLong(0);
+ public final AtomicLong bytesDiscardedInAbort= new AtomicLong(0);
+ public final AtomicLong policySetCount= new AtomicLong(0);
+ public volatile long inputPolicy;
private final AtomicLong versionMismatches = new AtomicLong(0);
- private InputStreamStatistics mergedStats;
+ private InputStreamStatisticsImpl mergedStats;
- private InputStreamStatistics() {
+ private InputStreamStatisticsImpl(
+ FileSystem.Statistics filesystemStatistics) {
+ this.filesystemStatistics = filesystemStatistics;
}
/**
@@ -670,10 +700,11 @@ private InputStreamStatistics() {
* @param negativeOffset how far was the seek?
* This is expected to be negative.
*/
+ @Override
public void seekBackwards(long negativeOffset) {
- seekOperations++;
- backwardSeekOperations++;
- bytesBackwardsOnSeek -= negativeOffset;
+ seekOperations.incrementAndGet();
+ backwardSeekOperations.incrementAndGet();
+ bytesBackwardsOnSeek.addAndGet(-negativeOffset);
}
/**
@@ -682,11 +713,12 @@ public void seekBackwards(long negativeOffset) {
* @param skipped number of bytes skipped by reading from the stream.
* If the seek was implemented by a close + reopen, set this to zero.
*/
+ @Override
public void seekForwards(long skipped) {
- seekOperations++;
- forwardSeekOperations++;
+ seekOperations.incrementAndGet();
+ forwardSeekOperations.incrementAndGet();
if (skipped > 0) {
- bytesSkippedOnSeek += skipped;
+ bytesSkippedOnSeek.addAndGet(skipped);
}
}
@@ -694,10 +726,9 @@ public void seekForwards(long skipped) {
* The inner stream was opened.
* @return the previous count
*/
+ @Override
public long streamOpened() {
- long count = openOperations;
- openOperations++;
- return count;
+ return openOperations.getAndIncrement();
}
/**
@@ -707,23 +738,25 @@ public long streamOpened() {
* @param remainingInCurrentRequest the number of bytes remaining in
* the current request.
*/
+ @Override
public void streamClose(boolean abortedConnection,
long remainingInCurrentRequest) {
- closeOperations++;
+ closeOperations.incrementAndGet();
if (abortedConnection) {
- this.aborted++;
- bytesDiscardedInAbort += remainingInCurrentRequest;
+ aborted.incrementAndGet();
+ bytesDiscardedInAbort.addAndGet(remainingInCurrentRequest);
} else {
- closed++;
- bytesReadInClose += remainingInCurrentRequest;
+ closed.incrementAndGet();
+ bytesReadInClose.addAndGet(remainingInCurrentRequest);
}
}
/**
* An ignored stream read exception was received.
*/
+ @Override
public void readException() {
- readExceptions++;
+ readExceptions.incrementAndGet();
}
/**
@@ -731,9 +764,10 @@ public void readException() {
* no-op if the argument is negative.
* @param bytes number of bytes read
*/
+ @Override
public void bytesRead(long bytes) {
if (bytes > 0) {
- bytesRead += bytes;
+ bytesRead.addAndGet(bytes);
}
}
@@ -742,8 +776,9 @@ public void bytesRead(long bytes) {
* @param pos starting position of the read
* @param len length of bytes to read
*/
+ @Override
public void readOperationStarted(long pos, long len) {
- readOperations++;
+ readOperations.incrementAndGet();
}
/**
@@ -752,8 +787,9 @@ public void readOperationStarted(long pos, long len) {
* @param pos starting position of the read
* @param len length of bytes to read
*/
+ @Override
public void readFullyOperationStarted(long pos, long len) {
- readFullyOperations++;
+ readFullyOperations.incrementAndGet();
}
/**
@@ -761,9 +797,10 @@ public void readFullyOperationStarted(long pos, long len) {
* @param requested number of requested bytes
* @param actual the actual number of bytes
*/
+ @Override
public void readOperationCompleted(int requested, int actual) {
if (requested > actual) {
- readsIncomplete++;
+ readsIncomplete.incrementAndGet();
}
}
@@ -780,17 +817,20 @@ public void close() {
* The input policy has been switched.
* @param updatedPolicy enum value of new policy.
*/
+ @Override
public void inputPolicySet(int updatedPolicy) {
- policySetCount++;
+ policySetCount.incrementAndGet();
inputPolicy = updatedPolicy;
}
/**
- * Get a reference to the version mismatch counter.
- * @return a counter which can be incremented.
+ * The change tracker increments {@code versionMismatches} on any
+ * mismatch.
+ * @return change tracking.
*/
- public AtomicLong getVersionMismatchCounter() {
- return versionMismatches;
+ @Override
+ public ChangeTrackerStatistics getChangeTrackerStatistics() {
+ return new CountingChangeTracker(versionMismatches);
}
/**
@@ -804,26 +844,26 @@ public AtomicLong getVersionMismatchCounter() {
public String toString() {
final StringBuilder sb = new StringBuilder(
"StreamStatistics{");
- sb.append("OpenOperations=").append(openOperations);
- sb.append(", CloseOperations=").append(closeOperations);
- sb.append(", Closed=").append(closed);
- sb.append(", Aborted=").append(aborted);
- sb.append(", SeekOperations=").append(seekOperations);
- sb.append(", ReadExceptions=").append(readExceptions);
+ sb.append("OpenOperations=").append(openOperations.get());
+ sb.append(", CloseOperations=").append(closeOperations.get());
+ sb.append(", Closed=").append(closed.get());
+ sb.append(", Aborted=").append(aborted.get());
+ sb.append(", SeekOperations=").append(seekOperations.get());
+ sb.append(", ReadExceptions=").append(readExceptions.get());
sb.append(", ForwardSeekOperations=")
- .append(forwardSeekOperations);
+ .append(forwardSeekOperations.get());
sb.append(", BackwardSeekOperations=")
- .append(backwardSeekOperations);
- sb.append(", BytesSkippedOnSeek=").append(bytesSkippedOnSeek);
- sb.append(", BytesBackwardsOnSeek=").append(bytesBackwardsOnSeek);
- sb.append(", BytesRead=").append(bytesRead);
+ .append(backwardSeekOperations.get());
+ sb.append(", BytesSkippedOnSeek=").append(bytesSkippedOnSeek.get());
+ sb.append(", BytesBackwardsOnSeek=").append(bytesBackwardsOnSeek.get());
+ sb.append(", BytesRead=").append(bytesRead.get());
sb.append(", BytesRead excluding skipped=")
- .append(bytesRead - bytesSkippedOnSeek);
- sb.append(", ReadOperations=").append(readOperations);
- sb.append(", ReadFullyOperations=").append(readFullyOperations);
- sb.append(", ReadsIncomplete=").append(readsIncomplete);
- sb.append(", BytesReadInClose=").append(bytesReadInClose);
- sb.append(", BytesDiscardedInAbort=").append(bytesDiscardedInAbort);
+ .append(bytesRead.get() - bytesSkippedOnSeek.get());
+ sb.append(", ReadOperations=").append(readOperations.get());
+ sb.append(", ReadFullyOperations=").append(readFullyOperations.get());
+ sb.append(", ReadsIncomplete=").append(readsIncomplete.get());
+ sb.append(", BytesReadInClose=").append(bytesReadInClose.get());
+ sb.append(", BytesDiscardedInAbort=").append(bytesDiscardedInAbort.get());
sb.append(", InputPolicy=").append(inputPolicy);
sb.append(", InputPolicySetCount=").append(policySetCount);
sb.append(", versionMismatches=").append(versionMismatches.get());
@@ -839,110 +879,251 @@ public String toString() {
* stats into the fs-wide stats. Behavior is undefined if called on a
* closed instance.
*/
- void merge(boolean isClosed) {
+ @Override
+ public void merge(boolean isClosed) {
if (mergedStats != null) {
- mergeInputStreamStatistics(diff(mergedStats));
+ mergeInputStreamStatistics(setd(mergedStats));
} else {
mergeInputStreamStatistics(this);
}
// If stats are closed, no need to create another copy
if (!isClosed) {
mergedStats = copy();
+ } else {
+ // stream is being closed.
+ // increment the filesystem statistics for this thread.
+ if (filesystemStatistics != null) {
+ filesystemStatistics.incrementBytesReadByDistance(DISTANCE,
+ bytesRead.get() + bytesReadInClose.get());
+ }
}
}
/**
- * Returns a diff between this {@link InputStreamStatistics} instance and
- * the given {@link InputStreamStatistics} instance.
+ * Set the dest variable to the difference of the two
+ * other values
+ * @param dest destination
+ * @param l left side
+ * @param r right side
+ */
+ private void setd(AtomicLong dest, AtomicLong l, AtomicLong r) {
+ dest.set(l.get() - r.get());
+ }
+
+ /**
+ * Returns a diff between this {@link InputStreamStatisticsImpl} instance and
+ * the given {@link InputStreamStatisticsImpl} instance.
*/
- private InputStreamStatistics diff(InputStreamStatistics inputStats) {
- InputStreamStatistics diff = new InputStreamStatistics();
- diff.openOperations = openOperations - inputStats.openOperations;
- diff.closeOperations = closeOperations - inputStats.closeOperations;
- diff.closed = closed - inputStats.closed;
- diff.aborted = aborted - inputStats.aborted;
- diff.seekOperations = seekOperations - inputStats.seekOperations;
- diff.readExceptions = readExceptions - inputStats.readExceptions;
- diff.forwardSeekOperations =
- forwardSeekOperations - inputStats.forwardSeekOperations;
- diff.backwardSeekOperations =
- backwardSeekOperations - inputStats.backwardSeekOperations;
- diff.bytesRead = bytesRead - inputStats.bytesRead;
- diff.bytesSkippedOnSeek =
- bytesSkippedOnSeek - inputStats.bytesSkippedOnSeek;
- diff.bytesBackwardsOnSeek =
- bytesBackwardsOnSeek - inputStats.bytesBackwardsOnSeek;
- diff.readOperations = readOperations - inputStats.readOperations;
- diff.readFullyOperations =
- readFullyOperations - inputStats.readFullyOperations;
- diff.readsIncomplete = readsIncomplete - inputStats.readsIncomplete;
- diff.bytesReadInClose = bytesReadInClose - inputStats.bytesReadInClose;
- diff.bytesDiscardedInAbort =
- bytesDiscardedInAbort - inputStats.bytesDiscardedInAbort;
- diff.policySetCount = policySetCount - inputStats.policySetCount;
- diff.inputPolicy = inputPolicy - inputStats.inputPolicy;
+ private InputStreamStatisticsImpl setd(InputStreamStatisticsImpl inputStats) {
+ InputStreamStatisticsImpl diff =
+ new InputStreamStatisticsImpl(filesystemStatistics);
+ setd(diff.openOperations, openOperations, inputStats.openOperations);
+ setd(diff.closeOperations, closeOperations, inputStats.closeOperations);
+ setd(diff.closed, closed, inputStats.closed);
+ setd(diff.aborted, aborted, inputStats.aborted);
+ setd(diff.seekOperations, seekOperations, inputStats.seekOperations);
+ setd(diff.readExceptions, readExceptions, inputStats.readExceptions);
+ setd(diff.forwardSeekOperations, forwardSeekOperations, inputStats.forwardSeekOperations);
+ setd(diff.backwardSeekOperations, backwardSeekOperations, inputStats.backwardSeekOperations);
+ setd(diff.bytesRead, bytesRead, inputStats.bytesRead);
+ setd(diff.bytesSkippedOnSeek, bytesSkippedOnSeek, inputStats.bytesSkippedOnSeek);
+ setd(diff.bytesBackwardsOnSeek, bytesBackwardsOnSeek, inputStats.bytesBackwardsOnSeek);
+ setd(diff.readOperations, readOperations, inputStats.readOperations);
+ setd(diff.readFullyOperations, readFullyOperations, inputStats.readFullyOperations);
+ setd(diff.readsIncomplete, readsIncomplete, inputStats.readsIncomplete);
+ setd(diff.bytesReadInClose, bytesReadInClose, inputStats.bytesReadInClose);
+ setd(diff.bytesDiscardedInAbort, bytesDiscardedInAbort, inputStats.bytesDiscardedInAbort);
+ setd(diff.policySetCount, policySetCount, inputStats.policySetCount);
+ diff.inputPolicy = inputPolicy -inputStats.inputPolicy;
diff.versionMismatches.set(versionMismatches.longValue() -
inputStats.versionMismatches.longValue());
return diff;
}
/**
- * Returns a new {@link InputStreamStatistics} instance with all the same
- * values as this {@link InputStreamStatistics}.
+ * Returns a new {@link InputStreamStatisticsImpl} instance with
+ * all the same values as this {@link InputStreamStatisticsImpl}.
*/
- private InputStreamStatistics copy() {
- InputStreamStatistics copy = new InputStreamStatistics();
- copy.openOperations = openOperations;
- copy.closeOperations = closeOperations;
- copy.closed = closed;
- copy.aborted = aborted;
- copy.seekOperations = seekOperations;
- copy.readExceptions = readExceptions;
- copy.forwardSeekOperations = forwardSeekOperations;
- copy.backwardSeekOperations = backwardSeekOperations;
- copy.bytesRead = bytesRead;
- copy.bytesSkippedOnSeek = bytesSkippedOnSeek;
- copy.bytesBackwardsOnSeek = bytesBackwardsOnSeek;
- copy.readOperations = readOperations;
- copy.readFullyOperations = readFullyOperations;
- copy.readsIncomplete = readsIncomplete;
- copy.bytesReadInClose = bytesReadInClose;
- copy.bytesDiscardedInAbort = bytesDiscardedInAbort;
- copy.policySetCount = policySetCount;
+ private InputStreamStatisticsImpl copy() {
+ InputStreamStatisticsImpl copy = new InputStreamStatisticsImpl(filesystemStatistics);
+ copy.openOperations.set(openOperations.get());
+ copy.closeOperations.set(closeOperations.get());
+ copy.closed.set(closed.get());
+ copy.aborted.set(aborted.get());
+ copy.seekOperations.set(seekOperations.get());
+ copy.readExceptions.set(readExceptions.get());
+ copy.forwardSeekOperations.set(forwardSeekOperations.get());
+ copy.backwardSeekOperations.set(backwardSeekOperations.get());
+ copy.bytesRead.set(bytesRead.get());
+ copy.bytesSkippedOnSeek.set(bytesSkippedOnSeek.get());
+ copy.bytesBackwardsOnSeek.set(bytesBackwardsOnSeek.get());
+ copy.readOperations.set(readOperations.get());
+ copy.readFullyOperations.set(readFullyOperations.get());
+ copy.readsIncomplete.set(readsIncomplete.get());
+ copy.bytesReadInClose.set(bytesReadInClose.get());
+ copy.bytesDiscardedInAbort.set(bytesDiscardedInAbort.get());
+ copy.policySetCount.set(policySetCount.get());
copy.inputPolicy = inputPolicy;
return copy;
}
+
+ /**
+ * Convert to an IOStatistics source which is dynamically updated.
+ * @return statistics
+ */
+ @Override
+ public IOStatistics createIOStatistics() {
+ DynamicIOStatisticsBuilder builder
+ = createDynamicIOStatistics();
+
+ builder.add(StreamStatisticNames.STREAM_BYTES_DISCARDED_ABORT,
+ bytesDiscardedInAbort);
+ builder.add(StreamStatisticNames.STREAM_CLOSED,
+ closed);
+ builder.add(StreamStatisticNames.STREAM_CLOSE_OPERATIONS,
+ closeOperations);
+ builder.add(StreamStatisticNames.STREAM_OPENED,
+ openOperations);
+ builder.add(StreamStatisticNames.STREAM_READ_BYTES,
+ bytesRead);
+ builder.add(StreamStatisticNames.STREAM_READ_EXCEPTIONS,
+ readExceptions);
+ builder.add(StreamStatisticNames.STREAM_READ_FULLY_OPERATIONS,
+ readFullyOperations);
+ builder.add(StreamStatisticNames.STREAM_READ_OPERATIONS,
+ readOperations);
+ builder.add(StreamStatisticNames.STREAM_READ_OPERATIONS_INCOMPLETE,
+ readsIncomplete);
+ return builder.build();
+ }
+
+ @Override
+ public long getCloseOperations() {
+ return closeOperations.get();
+ }
+
+ @Override
+ public long getClosed() {
+ return closed.get();
+ }
+
+ @Override
+ public long getAborted() {
+ return aborted.get();
+ }
+
+ @Override
+ public long getForwardSeekOperations() {
+ return forwardSeekOperations.get();
+ }
+
+ @Override
+ public long getBackwardSeekOperations() {
+ return backwardSeekOperations.get();
+ }
+
+ @Override
+ public long getBytesRead() {
+ return bytesRead.get();
+ }
+
+ @Override
+ public long getBytesSkippedOnSeek() {
+ return bytesSkippedOnSeek.get();
+ }
+
+ @Override
+ public long getBytesBackwardsOnSeek() {
+ return bytesBackwardsOnSeek.get();
+ }
+
+ @Override
+ public long getBytesReadInClose() {
+ return bytesReadInClose.get();
+ }
+
+ @Override
+ public long getBytesDiscardedInAbort() {
+ return bytesDiscardedInAbort.get();
+ }
+
+ @Override
+ public long getOpenOperations() {
+ return openOperations.get();
+ }
+
+ @Override
+ public long getSeekOperations() {
+ return seekOperations.get();
+ }
+
+ @Override
+ public long getReadExceptions() {
+ return readExceptions.get();
+ }
+
+ @Override
+ public long getReadOperations() {
+ return readOperations.get();
+ }
+
+ @Override
+ public long getReadFullyOperations() {
+ return readFullyOperations.get();
+ }
+
+ @Override
+ public long getReadsIncomplete() {
+ return readsIncomplete.get();
+ }
+
+ @Override
+ public long getPolicySetCount() {
+ return policySetCount.get();
+ }
+
+ @Override
+ public long getVersionMismatches() {
+ return versionMismatches.get();
+ }
+
+ @Override
+ public long getInputPolicy() {
+ return inputPolicy;
+ }
}
/**
* Create a stream output statistics instance.
* @return the new instance
*/
- OutputStreamStatistics newOutputStreamStatistics(Statistics statistics) {
- return new OutputStreamStatistics(statistics);
+ public BlockOutputStreamStatistics newOutputStreamStatistics(
+ FileSystem.Statistics filesystemStatistics) {
+ return new BlockOutputStreamStatisticsImpl(filesystemStatistics);
}
/**
* Merge in the statistics of a single output stream into
* the filesystem-wide statistics.
- * @param statistics stream statistics
+ * @param source stream statistics
*/
- private void mergeOutputStreamStatistics(OutputStreamStatistics statistics) {
- incrementCounter(STREAM_WRITE_TOTAL_TIME, statistics.totalUploadDuration());
- incrementCounter(STREAM_WRITE_QUEUE_DURATION, statistics.queueDuration);
- incrementCounter(STREAM_WRITE_TOTAL_DATA, statistics.bytesUploaded);
+ private void mergeOutputStreamStatistics(BlockOutputStreamStatisticsImpl source) {
+ incrementCounter(STREAM_WRITE_TOTAL_TIME, source.totalUploadDuration());
+ incrementCounter(STREAM_WRITE_QUEUE_DURATION, source.queueDuration);
+ incrementCounter(STREAM_WRITE_TOTAL_DATA, source.bytesUploaded);
incrementCounter(STREAM_WRITE_BLOCK_UPLOADS,
- statistics.blockUploadsCompleted);
+ source.blockUploadsCompleted);
+ incrementCounter(STREAM_WRITE_FAILURES, source.blockUploadsFailed);
}
/**
* Statistics updated by an output stream during its actual operation.
- * Some of these stats may be relayed. However, as block upload is
- * spans multiple
+ * Some of these stats are propagated to any passed in
+ * {@link FileSystem.Statistics} instance; this is only done
+ * in close() for better cross-thread accounting.
*/
- @InterfaceAudience.Private
- @InterfaceStability.Unstable
- public final class OutputStreamStatistics implements Closeable {
+ private final class BlockOutputStreamStatisticsImpl implements
+ BlockOutputStreamStatistics {
private final AtomicLong blocksSubmitted = new AtomicLong(0);
private final AtomicLong blocksInQueue = new AtomicLong(0);
private final AtomicLong blocksActive = new AtomicLong(0);
@@ -950,6 +1131,7 @@ public final class OutputStreamStatistics implements Closeable {
private final AtomicLong blockUploadsFailed = new AtomicLong(0);
private final AtomicLong bytesPendingUpload = new AtomicLong(0);
+ private final AtomicLong bytesWritten = new AtomicLong(0);
private final AtomicLong bytesUploaded = new AtomicLong(0);
private final AtomicLong transferDuration = new AtomicLong(0);
private final AtomicLong queueDuration = new AtomicLong(0);
@@ -957,30 +1139,34 @@ public final class OutputStreamStatistics implements Closeable {
private final AtomicInteger blocksAllocated = new AtomicInteger(0);
private final AtomicInteger blocksReleased = new AtomicInteger(0);
- private Statistics statistics;
+ private final FileSystem.Statistics filesystemStatistics;
- public OutputStreamStatistics(Statistics statistics){
- this.statistics = statistics;
+ private BlockOutputStreamStatisticsImpl(
+ @Nullable FileSystem.Statistics filesystemStatistics) {
+ this.filesystemStatistics = filesystemStatistics;
}
/**
* A block has been allocated.
*/
- void blockAllocated() {
+ @Override
+ public void blockAllocated() {
blocksAllocated.incrementAndGet();
}
/**
* A block has been released.
*/
- void blockReleased() {
+ @Override
+ public void blockReleased() {
blocksReleased.incrementAndGet();
}
/**
* Block is queued for upload.
*/
- void blockUploadQueued(int blockSize) {
+ @Override
+ public void blockUploadQueued(int blockSize) {
blocksSubmitted.incrementAndGet();
blocksInQueue.incrementAndGet();
bytesPendingUpload.addAndGet(blockSize);
@@ -989,7 +1175,8 @@ void blockUploadQueued(int blockSize) {
}
/** Queued block has been scheduled for upload. */
- void blockUploadStarted(long duration, int blockSize) {
+ @Override
+ public void blockUploadStarted(long duration, int blockSize) {
queueDuration.addAndGet(duration);
blocksInQueue.decrementAndGet();
blocksActive.incrementAndGet();
@@ -998,8 +1185,9 @@ void blockUploadStarted(long duration, int blockSize) {
}
/** A block upload has completed. */
- void blockUploadCompleted(long duration, int blockSize) {
- this.transferDuration.addAndGet(duration);
+ @Override
+ public void blockUploadCompleted(long duration, int blockSize) {
+ transferDuration.addAndGet(duration);
incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_ACTIVE, -1);
blocksActive.decrementAndGet();
blockUploadsCompleted.incrementAndGet();
@@ -1010,14 +1198,15 @@ void blockUploadCompleted(long duration, int blockSize) {
* A final transfer completed event is still expected, so this
* does not decrement the active block counter.
*/
- void blockUploadFailed(long duration, int blockSize) {
+ @Override
+ public void blockUploadFailed(long duration, int blockSize) {
blockUploadsFailed.incrementAndGet();
}
/** Intermediate report of bytes uploaded. */
- void bytesTransferred(long byteCount) {
+ @Override
+ public void bytesTransferred(long byteCount) {
bytesUploaded.addAndGet(byteCount);
- statistics.incrementBytesWritten(byteCount);
bytesPendingUpload.addAndGet(-byteCount);
incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING, -byteCount);
}
@@ -1026,7 +1215,8 @@ void bytesTransferred(long byteCount) {
* Note exception in a multipart complete.
* @param count count of exceptions
*/
- void exceptionInMultipartComplete(int count) {
+ @Override
+ public void exceptionInMultipartComplete(int count) {
if (count > 0) {
exceptionsInMultipartFinalize.addAndGet(count);
}
@@ -1035,7 +1225,8 @@ void exceptionInMultipartComplete(int count) {
/**
* Note an exception in a multipart abort.
*/
- void exceptionInMultipartAbort() {
+ @Override
+ public void exceptionInMultipartAbort() {
exceptionsInMultipartFinalize.incrementAndGet();
}
@@ -1043,6 +1234,7 @@ void exceptionInMultipartAbort() {
* Get the number of bytes pending upload.
* @return the number of bytes in the pending upload state.
*/
+ @Override
public long getBytesPendingUpload() {
return bytesPendingUpload.get();
}
@@ -1052,6 +1244,7 @@ public long getBytesPendingUpload() {
* to be called at the end of the write.
* @param size size in bytes
*/
+ @Override
public void commitUploaded(long size) {
incrementCounter(COMMITTER_BYTES_UPLOADED, size);
}
@@ -1067,6 +1260,12 @@ public void close() {
" as pending upload in {}", this);
}
mergeOutputStreamStatistics(this);
+ // and patch the FS statistics.
+ // provided the stream is closed in the worker thread, this will
+ // ensure that the thread-specific worker stats are updated.
+ if (filesystemStatistics != null) {
+ filesystemStatistics.incrementBytesWritten(bytesUploaded.get());
+ }
}
long averageQueueTime() {
@@ -1084,11 +1283,13 @@ long totalUploadDuration() {
return queueDuration.get() + transferDuration.get();
}
- public int blocksAllocated() {
+ @Override
+ public int getBlocksAllocated() {
return blocksAllocated.get();
}
- public int blocksReleased() {
+ @Override
+ public int getBlocksReleased() {
return blocksReleased.get();
}
@@ -1097,10 +1298,28 @@ public int blocksReleased() {
* if the numbers change during the (non-synchronized) calculation.
* @return the number of actively allocated blocks.
*/
- public int blocksActivelyAllocated() {
+ @Override
+ public int getBlocksActivelyAllocated() {
return blocksAllocated.get() - blocksReleased.get();
}
+ /**
+ * Record bytes written.
+ * @param count number of bytes
+ */
+ @Override
+ public void writeBytes(long count) {
+ bytesWritten.addAndGet(count);
+ }
+
+ /**
+ * Get the current count of bytes written.
+ * @return the counter value.
+ */
+ @Override
+ public long getBytesWritten() {
+ return bytesWritten.get();
+ }
@Override
public String toString() {
@@ -1113,9 +1332,10 @@ public String toString() {
sb.append(", blockUploadsFailed=").append(blockUploadsFailed);
sb.append(", bytesPendingUpload=").append(bytesPendingUpload);
sb.append(", bytesUploaded=").append(bytesUploaded);
+ sb.append(", bytesWritten=").append(bytesWritten);
sb.append(", blocksAllocated=").append(blocksAllocated);
sb.append(", blocksReleased=").append(blocksReleased);
- sb.append(", blocksActivelyAllocated=").append(blocksActivelyAllocated());
+ sb.append(", blocksActivelyAllocated=").append(getBlocksActivelyAllocated());
sb.append(", exceptionsInMultipartFinalize=").append(
exceptionsInMultipartFinalize);
sb.append(", transferDuration=").append(transferDuration).append(" ms");
@@ -1128,6 +1348,23 @@ public String toString() {
sb.append('}');
return sb.toString();
}
+
+ /**
+ * Convert to an IOStatistics source which is dynamically updated.
+ * @return statistics
+ */
+ @Override
+ public IOStatistics createIOStatistics() {
+ DynamicIOStatisticsBuilder builder = createDynamicIOStatistics();
+
+ builder.add(StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS,
+ blocksSubmitted);
+ builder.add(StreamStatisticNames.STREAM_WRITE_BYTES,
+ bytesWritten);
+ builder.add(StreamStatisticNames.STREAM_WRITE_FAILURES,
+ blockUploadsFailed);
+ return builder.build();
+ }
}
/**
@@ -1192,13 +1429,12 @@ public void entryAdded(final long durationNanos) {
}
/**
- * Instrumentation exported to S3Guard Committers.
+ * Instrumentation exported to S3A Committers.
*/
- @InterfaceAudience.Private
- @InterfaceStability.Unstable
- public final class CommitterStatistics {
+ private final class CommitterStatisticsImpl implements CommitterStatistics {
/** A commit has been created. */
+ @Override
public void commitCreated() {
incrementCounter(COMMITTER_COMMITS_CREATED, 1);
}
@@ -1207,6 +1443,7 @@ public void commitCreated() {
* Data has been uploaded to be committed in a subsequent operation.
* @param size size in bytes
*/
+ @Override
public void commitUploaded(long size) {
incrementCounter(COMMITTER_BYTES_UPLOADED, size);
}
@@ -1215,24 +1452,29 @@ public void commitUploaded(long size) {
* A commit has been completed.
* @param size size in bytes
*/
+ @Override
public void commitCompleted(long size) {
incrementCounter(COMMITTER_COMMITS_COMPLETED, 1);
incrementCounter(COMMITTER_BYTES_COMMITTED, size);
}
/** A commit has been aborted. */
+ @Override
public void commitAborted() {
incrementCounter(COMMITTER_COMMITS_ABORTED, 1);
}
+ @Override
public void commitReverted() {
incrementCounter(COMMITTER_COMMITS_REVERTED, 1);
}
+ @Override
public void commitFailed() {
incrementCounter(COMMITTER_COMMITS_FAILED, 1);
}
+ @Override
public void taskCompleted(boolean success) {
incrementCounter(
success ? COMMITTER_TASKS_SUCCEEDED
@@ -1240,6 +1482,7 @@ public void taskCompleted(boolean success) {
1);
}
+ @Override
public void jobCompleted(boolean success) {
incrementCounter(
success ? COMMITTER_JOBS_SUCCEEDED
@@ -1253,20 +1496,20 @@ public void jobCompleted(boolean success) {
* @return an instance of delegation token statistics
*/
public DelegationTokenStatistics newDelegationTokenStatistics() {
- return new DelegationTokenStatistics();
+ return new DelegationTokenStatisticsImpl();
}
/**
* Instrumentation exported to S3A Delegation Token support.
*/
- @InterfaceAudience.Private
- @InterfaceStability.Unstable
- public final class DelegationTokenStatistics {
+ private final class DelegationTokenStatisticsImpl implements
+ DelegationTokenStatistics {
- private DelegationTokenStatistics() {
+ private DelegationTokenStatisticsImpl() {
}
/** A token has been issued. */
+ @Override
public void tokenIssued() {
incrementCounter(DELEGATION_TOKENS_ISSUED, 1);
}
@@ -1366,4 +1609,66 @@ public Map getMap() {
return map;
}
}
+
+ public StatisticsFromAwsSdk newStatisticsFromAwsSdk() {
+ return new StatisticsFromAwsSdkImpl(this);
+ }
+
+ /**
+ * Hook up AWS SDK Statistics to the S3 counters.
+ * Durations are not currently being used; that could be
+ * changed in future once an effective strategy for reporting
+ * them is determined.
+ */
+ private static final class StatisticsFromAwsSdkImpl implements
+ StatisticsFromAwsSdk {
+
+ private final CountersAndGauges countersAndGauges;
+
+ private StatisticsFromAwsSdkImpl(final CountersAndGauges countersAndGauges) {
+ this.countersAndGauges = countersAndGauges;
+ }
+
+ @Override
+ public void updateAwsRequestCount(final long count) {
+ countersAndGauges.incrementCounter(STORE_IO_REQUEST, count);
+ }
+
+ @Override
+ public void updateAwsRetryCount(final long count) {
+ countersAndGauges.incrementCounter(STORE_IO_RETRY, count);
+
+ }
+
+ @Override
+ public void updateAwsThrottleExceptionsCount(final long count) {
+ countersAndGauges.incrementCounter(STORE_IO_THROTTLED, count);
+ countersAndGauges.addValueToQuantiles(STORE_IO_THROTTLE_RATE, count);
+ }
+
+ @Override
+ public void noteAwsRequestTime(final Duration duration) {
+
+ }
+
+ @Override
+ public void noteAwsClientExecuteTime(final Duration duration) {
+
+ }
+
+ @Override
+ public void noteRequestMarshallTime(final Duration duration) {
+
+ }
+
+ @Override
+ public void noteRequestSigningTime(final Duration duration) {
+
+ }
+
+ @Override
+ public void noteResponseProcessingTime(final Duration duration) {
+
+ }
+ }
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOpContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOpContext.java
index 4e0aac5138eea..f39d4731f8689 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOpContext.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOpContext.java
@@ -24,19 +24,23 @@
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.s3a.impl.ActiveOperationContext;
+import org.apache.hadoop.fs.s3a.impl.statistics.S3AStatisticsContext;
/**
- * Base class for operation context struct passed through codepaths for main
+ * Class for operation context struct passed through codepaths for main
* S3AFileSystem operations.
* Anything op-specific should be moved to a subclass of this.
+ *
+ * This was originally a base class, but {@link ActiveOperationContext} was
+ * created to be more minimal and cover many more operation type.
*/
-@SuppressWarnings("visibilitymodifier") // I want a struct of finals, for real.
-public class S3AOpContext {
+@SuppressWarnings("visibilitymodifier")
+public class S3AOpContext extends ActiveOperationContext {
final boolean isS3GuardEnabled;
final Invoker invoker;
@Nullable final FileSystem.Statistics stats;
- final S3AInstrumentation instrumentation;
@Nullable final Invoker s3guardInvoker;
/** FileStatus for "destination" path being operated on. */
@@ -53,9 +57,14 @@ public class S3AOpContext {
* @param dstFileStatus file status from existence check
*/
public S3AOpContext(boolean isS3GuardEnabled, Invoker invoker,
- Invoker s3guardInvoker, @Nullable FileSystem.Statistics stats,
- S3AInstrumentation instrumentation, FileStatus dstFileStatus) {
+ @Nullable Invoker s3guardInvoker,
+ @Nullable FileSystem.Statistics stats,
+ S3AStatisticsContext instrumentation,
+ FileStatus dstFileStatus) {
+ super(newOperationId(),
+ instrumentation,
+ null);
Preconditions.checkNotNull(invoker, "Null invoker arg");
Preconditions.checkNotNull(instrumentation, "Null instrumentation arg");
Preconditions.checkNotNull(dstFileStatus, "Null dstFileStatus arg");
@@ -65,7 +74,6 @@ public S3AOpContext(boolean isS3GuardEnabled, Invoker invoker,
this.invoker = invoker;
this.s3guardInvoker = s3guardInvoker;
this.stats = stats;
- this.instrumentation = instrumentation;
this.dstFileStatus = dstFileStatus;
}
@@ -77,8 +85,10 @@ public S3AOpContext(boolean isS3GuardEnabled, Invoker invoker,
* @param instrumentation instrumentation to use
* @param dstFileStatus file status from existence check
*/
- public S3AOpContext(boolean isS3GuardEnabled, Invoker invoker,
- @Nullable FileSystem.Statistics stats, S3AInstrumentation instrumentation,
+ public S3AOpContext(boolean isS3GuardEnabled,
+ Invoker invoker,
+ @Nullable FileSystem.Statistics stats,
+ S3AStatisticsContext instrumentation,
FileStatus dstFileStatus) {
this(isS3GuardEnabled, invoker, null, stats, instrumentation,
dstFileStatus);
@@ -97,10 +107,6 @@ public FileSystem.Statistics getStats() {
return stats;
}
- public S3AInstrumentation getInstrumentation() {
- return instrumentation;
- }
-
@Nullable
public Invoker getS3guardInvoker() {
return s3guardInvoker;
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java
index a7317c945127c..af92d61d713f1 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java
@@ -22,6 +22,7 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy;
+import org.apache.hadoop.fs.s3a.impl.statistics.S3AStatisticsContext;
import javax.annotation.Nullable;
@@ -60,8 +61,8 @@ public class S3AReadOpContext extends S3AOpContext {
* @param isS3GuardEnabled true iff S3Guard is enabled.
* @param invoker invoker for normal retries.
* @param s3guardInvoker S3Guard-specific retry invoker.
- * @param stats statistics (may be null)
- * @param instrumentation FS instrumentation
+ * @param stats Fileystem statistics (may be null)
+ * @param instrumentation statistics context
* @param dstFileStatus target file status
* @param inputPolicy the input policy
* @param readahead readahead for GET operations/skip, etc.
@@ -71,13 +72,14 @@ public S3AReadOpContext(
final Path path,
boolean isS3GuardEnabled,
Invoker invoker,
- Invoker s3guardInvoker,
+ @Nullable Invoker s3guardInvoker,
@Nullable FileSystem.Statistics stats,
- S3AInstrumentation instrumentation,
+ S3AStatisticsContext instrumentation,
FileStatus dstFileStatus,
S3AInputPolicy inputPolicy,
ChangeDetectionPolicy changeDetectionPolicy,
final long readahead) {
+
super(isS3GuardEnabled, invoker, s3guardInvoker, stats, instrumentation,
dstFileStatus);
this.path = checkNotNull(path);
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java
index e0a1d780ccf5f..60149ced38beb 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java
@@ -22,10 +22,12 @@
import java.net.URI;
import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.metrics.RequestMetricCollector;
import com.amazonaws.services.s3.AmazonS3;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.s3a.impl.statistics.StatisticsFromAwsSdk;
/**
* Factory for creation of {@link AmazonS3} client instances.
@@ -41,12 +43,14 @@ public interface S3ClientFactory {
* @param bucket Optional bucket to use to look up per-bucket proxy secrets
* @param credentialSet credentials to use
* @param userAgentSuffix optional suffix for the UA field.
+ * @param statisticsFromAwsSdk binding for AWS stats
* @return S3 client
* @throws IOException IO problem
*/
AmazonS3 createS3Client(URI name,
String bucket,
AWSCredentialsProvider credentialSet,
- String userAgentSuffix) throws IOException;
+ String userAgentSuffix,
+ StatisticsFromAwsSdk statisticsFromAwsSdk) throws IOException;
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
index 1d3d4758028c6..54f17cbdcb7c9 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
@@ -18,7 +18,8 @@
package org.apache.hadoop.fs.s3a;
-import org.apache.hadoop.fs.StorageStatistics.CommonStatisticNames;
+import org.apache.hadoop.fs.statistics.StoreStatisticNames;
+import org.apache.hadoop.fs.statistics.StreamStatisticNames;
import java.util.HashMap;
import java.util.Map;
@@ -49,39 +50,56 @@ public enum Statistic {
FAKE_DIRECTORIES_DELETED("fake_directories_deleted",
"Total number of fake directory deletes submitted to object store."),
IGNORED_ERRORS("ignored_errors", "Errors caught and ignored"),
- INVOCATION_COPY_FROM_LOCAL_FILE(CommonStatisticNames.OP_COPY_FROM_LOCAL_FILE,
+ INVOCATION_COPY_FROM_LOCAL_FILE(
+ StoreStatisticNames.OP_COPY_FROM_LOCAL_FILE,
"Calls of copyFromLocalFile()"),
- INVOCATION_CREATE(CommonStatisticNames.OP_CREATE,
+ INVOCATION_CREATE(
+ StoreStatisticNames.OP_CREATE,
"Calls of create()"),
- INVOCATION_CREATE_NON_RECURSIVE(CommonStatisticNames.OP_CREATE_NON_RECURSIVE,
+ INVOCATION_CREATE_NON_RECURSIVE(
+ StoreStatisticNames.OP_CREATE_NON_RECURSIVE,
"Calls of createNonRecursive()"),
- INVOCATION_DELETE(CommonStatisticNames.OP_DELETE,
+ INVOCATION_DELETE(
+ StoreStatisticNames.OP_DELETE,
"Calls of delete()"),
- INVOCATION_EXISTS(CommonStatisticNames.OP_EXISTS,
+ INVOCATION_EXISTS(
+ StoreStatisticNames.OP_EXISTS,
"Calls of exists()"),
- INVOCATION_GET_DELEGATION_TOKEN(CommonStatisticNames.OP_GET_DELEGATION_TOKEN,
+ INVOCATION_GET_DELEGATION_TOKEN(
+ StoreStatisticNames.OP_GET_DELEGATION_TOKEN,
"Calls of getDelegationToken()"),
- INVOCATION_GET_FILE_CHECKSUM(CommonStatisticNames.OP_GET_FILE_CHECKSUM,
+ INVOCATION_GET_FILE_CHECKSUM(
+ StoreStatisticNames.OP_GET_FILE_CHECKSUM,
"Calls of getFileChecksum()"),
- INVOCATION_GET_FILE_STATUS(CommonStatisticNames.OP_GET_FILE_STATUS,
+ INVOCATION_GET_FILE_STATUS(
+ StoreStatisticNames.OP_GET_FILE_STATUS,
"Calls of getFileStatus()"),
- INVOCATION_GLOB_STATUS(CommonStatisticNames.OP_GLOB_STATUS,
+ INVOCATION_GLOB_STATUS(
+ StoreStatisticNames.OP_GLOB_STATUS,
"Calls of globStatus()"),
- INVOCATION_IS_DIRECTORY(CommonStatisticNames.OP_IS_DIRECTORY,
+ INVOCATION_IS_DIRECTORY(
+ StoreStatisticNames.OP_IS_DIRECTORY,
"Calls of isDirectory()"),
- INVOCATION_IS_FILE(CommonStatisticNames.OP_IS_FILE,
+ INVOCATION_IS_FILE(
+ StoreStatisticNames.OP_IS_FILE,
"Calls of isFile()"),
- INVOCATION_LIST_FILES(CommonStatisticNames.OP_LIST_FILES,
+ INVOCATION_LIST_FILES(
+ StoreStatisticNames.OP_LIST_FILES,
"Calls of listFiles()"),
- INVOCATION_LIST_LOCATED_STATUS(CommonStatisticNames.OP_LIST_LOCATED_STATUS,
+ INVOCATION_LIST_LOCATED_STATUS(
+ StoreStatisticNames.OP_LIST_LOCATED_STATUS,
"Calls of listLocatedStatus()"),
- INVOCATION_LIST_STATUS(CommonStatisticNames.OP_LIST_STATUS,
+ INVOCATION_LIST_STATUS(
+ StoreStatisticNames.OP_LIST_STATUS,
"Calls of listStatus()"),
- INVOCATION_MKDIRS(CommonStatisticNames.OP_MKDIRS,
+ INVOCATION_MKDIRS(
+ StoreStatisticNames.OP_MKDIRS,
"Calls of mkdirs()"),
- INVOCATION_OPEN(CommonStatisticNames.OP_OPEN,
+ INVOCATION_OPEN(
+ StoreStatisticNames.OP_OPEN,
"Calls of open()"),
- INVOCATION_RENAME(CommonStatisticNames.OP_RENAME,
+ INVOCATION_RENAME(
+ StoreStatisticNames.OP_RENAME,
"Calls of rename()"),
OBJECT_COPY_REQUESTS("object_copy_requests", "Object copy requests"),
OBJECT_DELETE_REQUESTS("object_delete_requests", "Object delete requests"),
@@ -106,51 +124,75 @@ public enum Statistic {
"number of bytes queued for upload/being actively uploaded"),
OBJECT_SELECT_REQUESTS("object_select_requests",
"Count of S3 Select requests issued"),
- STREAM_ABORTED("stream_aborted",
+ STREAM_ABORTED(
+ StreamStatisticNames.STREAM_ABORTED,
"Count of times the TCP stream was aborted"),
- STREAM_BACKWARD_SEEK_OPERATIONS("stream_backward_seek_operations",
+ STREAM_BACKWARD_SEEK_OPERATIONS(
+ StreamStatisticNames.STREAM_READ_SEEK_BACKWARD_OPERATIONS,
"Number of executed seek operations which went backwards in a stream"),
- STREAM_CLOSED("stream_closed", "Count of times the TCP stream was closed"),
- STREAM_CLOSE_OPERATIONS("stream_close_operations",
+ STREAM_CLOSED(
+ StreamStatisticNames.STREAM_CLOSED,
+ "Count of times the TCP stream was closed"),
+ STREAM_CLOSE_OPERATIONS(
+ StreamStatisticNames.STREAM_CLOSE_OPERATIONS,
"Total count of times an attempt to close a data stream was made"),
- STREAM_FORWARD_SEEK_OPERATIONS("stream_forward_seek_operations",
+ STREAM_FORWARD_SEEK_OPERATIONS(
+ StreamStatisticNames.STREAM_READ_SEEK_FORWARD_OPERATIONS,
"Number of executed seek operations which went forward in a stream"),
- STREAM_OPENED("stream_opened",
- "Total count of times an input stream to object store was opened"),
- STREAM_READ_EXCEPTIONS("stream_read_exceptions",
- "Number of exceptions invoked on input streams"),
- STREAM_READ_FULLY_OPERATIONS("stream_read_fully_operations",
+ STREAM_OPENED(
+ StreamStatisticNames.STREAM_OPENED,
+ "Total count of times an input stream to object store data was opened"),
+ STREAM_READ_EXCEPTIONS(
+ StreamStatisticNames.STREAM_READ_EXCEPTIONS,
+ "Number of exceptions raised during input stream reads"),
+ STREAM_READ_FULLY_OPERATIONS(
+ StreamStatisticNames.STREAM_READ_FULLY_OPERATIONS,
"Count of readFully() operations in streams"),
- STREAM_READ_OPERATIONS("stream_read_operations",
+ STREAM_READ_OPERATIONS(
+ StreamStatisticNames.STREAM_READ_OPERATIONS,
"Count of read() operations in streams"),
- STREAM_READ_OPERATIONS_INCOMPLETE("stream_read_operations_incomplete",
+ STREAM_READ_OPERATIONS_INCOMPLETE(
+ StreamStatisticNames.STREAM_READ_OPERATIONS_INCOMPLETE,
"Count of incomplete read() operations in streams"),
- STREAM_READ_VERSION_MISMATCHES("stream_read_version_mismatches",
+ STREAM_READ_VERSION_MISMATCHES(
+ StreamStatisticNames.STREAM_READ_VERSION_MISMATCHES,
"Count of version mismatches encountered while reading streams"),
- STREAM_SEEK_BYTES_BACKWARDS("stream_bytes_backwards_on_seek",
+ STREAM_SEEK_BYTES_BACKWARDS(
+ StreamStatisticNames.STREAM_READ_SEEK_BYTES_BACKWARDS,
"Count of bytes moved backwards during seek operations"),
- STREAM_SEEK_BYTES_READ("stream_bytes_read",
+ STREAM_SEEK_BYTES_READ(
+ StreamStatisticNames.STREAM_READ_SEEK_BYTES_READ,
"Count of bytes read during seek() in stream operations"),
- STREAM_SEEK_BYTES_SKIPPED("stream_bytes_skipped_on_seek",
+ STREAM_SEEK_BYTES_SKIPPED(
+ StreamStatisticNames.STREAM_READ_SEEK_BYTES_SKIPPED,
"Count of bytes skipped during forward seek operation"),
- STREAM_SEEK_OPERATIONS("stream_seek_operations",
+ STREAM_SEEK_OPERATIONS(
+ StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS,
"Number of seek operations during stream IO."),
- STREAM_CLOSE_BYTES_READ("stream_bytes_read_in_close",
+ STREAM_CLOSE_BYTES_READ(
+ StreamStatisticNames.STREAM_CLOSE_BYTES_READ,
"Count of bytes read when closing streams during seek operations."),
- STREAM_ABORT_BYTES_DISCARDED("stream_bytes_discarded_in_abort",
+ STREAM_ABORT_BYTES_DISCARDED(
+ StreamStatisticNames.STREAM_BYTES_DISCARDED_ABORT,
"Count of bytes discarded by aborting the stream"),
- STREAM_WRITE_FAILURES("stream_write_failures",
+ STREAM_WRITE_FAILURES(
+ StreamStatisticNames.STREAM_WRITE_FAILURES,
"Count of stream write failures reported"),
- STREAM_WRITE_BLOCK_UPLOADS("stream_write_block_uploads",
+ STREAM_WRITE_BLOCK_UPLOADS(
+ StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS,
"Count of block/partition uploads completed"),
- STREAM_WRITE_BLOCK_UPLOADS_ACTIVE("stream_write_block_uploads_active",
+ STREAM_WRITE_BLOCK_UPLOADS_ACTIVE(
+ StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS_ACTIVE,
"Count of block/partition uploads completed"),
- STREAM_WRITE_BLOCK_UPLOADS_COMMITTED("stream_write_block_uploads_committed",
+ STREAM_WRITE_BLOCK_UPLOADS_COMMITTED(
+ StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS_COMMITTED,
"Count of number of block uploads committed"),
- STREAM_WRITE_BLOCK_UPLOADS_ABORTED("stream_write_block_uploads_aborted",
+ STREAM_WRITE_BLOCK_UPLOADS_ABORTED(
+ StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS_ABORTED,
"Count of number of block uploads aborted"),
- STREAM_WRITE_BLOCK_UPLOADS_PENDING("stream_write_block_uploads_pending",
+ STREAM_WRITE_BLOCK_UPLOADS_PENDING(
+ StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS_PENDING,
"Gauge of block/partitions uploads queued to be written"),
STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING(
"stream_write_block_uploads_data_pending",
@@ -159,6 +201,9 @@ public enum Statistic {
"Count of total time taken for uploads to complete"),
STREAM_WRITE_TOTAL_DATA("stream_write_total_data",
"Count of total data uploaded in block output"),
+ STREAM_WRITE_BYTES(
+ StreamStatisticNames.STREAM_WRITE_BYTES,
+ "Count of bytes written to output stream (including all not yet uploaded"),
STREAM_WRITE_QUEUE_DURATION("stream_write_queue_duration",
"Total queue duration of all block uploads"),
@@ -233,8 +278,17 @@ public enum Statistic {
STORE_IO_THROTTLE_RATE("store_io_throttle_rate",
"Rate of S3 request throttling"),
- DELEGATION_TOKENS_ISSUED("delegation_tokens_issued",
- "Number of delegation tokens issued");
+ DELEGATION_TOKENS_ISSUED(
+ StoreStatisticNames.DELEGATION_TOKENS_ISSUED,
+ "Number of delegation tokens issued"),
+
+ STORE_IO_REQUEST(StoreStatisticNames.STORE_IO_REQUEST,
+ "requests made of the remote store"),
+
+ STORE_IO_RETRY(StoreStatisticNames.STORE_IO_RETRY,
+ "retried requests made of the remote store"),
+
+ ;
private static final Map SYMBOL_MAP =
new HashMap<>(Statistic.values().length);
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java
index 602732b6d3250..552a5449a55a7 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java
@@ -49,6 +49,7 @@
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.impl.statistics.S3AStatisticsContext;
import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
import org.apache.hadoop.fs.s3a.s3guard.S3Guard;
import org.apache.hadoop.fs.s3a.select.SelectBinding;
@@ -104,17 +105,26 @@ public class WriteOperationHelper {
/** Bucket of the owner FS. */
private final String bucket;
+ /**
+ * statistics context.
+ */
+ private final S3AStatisticsContext statisticsContext;
+
/**
* Constructor.
* @param owner owner FS creating the helper
* @param conf Configuration object
+ * @param statisticsContext statistics context
*
*/
- protected WriteOperationHelper(S3AFileSystem owner, Configuration conf) {
+ protected WriteOperationHelper(S3AFileSystem owner,
+ Configuration conf,
+ S3AStatisticsContext statisticsContext) {
this.owner = owner;
this.invoker = new Invoker(new S3ARetryPolicy(conf),
this::operationRetried);
this.conf = conf;
+ this.statisticsContext = statisticsContext;
bucket = owner.getBucket();
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/S3ADelegationTokens.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/S3ADelegationTokens.java
index 5005436c8242a..6811d3d9b3825 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/S3ADelegationTokens.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/S3ADelegationTokens.java
@@ -34,9 +34,9 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.s3a.AWSCredentialProviderList;
-import org.apache.hadoop.fs.s3a.S3AInstrumentation;
import org.apache.hadoop.fs.s3a.auth.RoleModel;
import org.apache.hadoop.fs.s3a.impl.StoreContext;
+import org.apache.hadoop.fs.s3a.impl.statistics.DelegationTokenStatistics;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.UserGroupInformation;
@@ -134,9 +134,9 @@ public class S3ADelegationTokens extends AbstractDTService {
AWSPolicyProvider.AccessLevel.WRITE);
/**
- * Statistics for the owner FS.
+ * Statistics for the operations.
*/
- private S3AInstrumentation.DelegationTokenStatistics stats;
+ private DelegationTokenStatistics stats;
/**
* Name of the token binding as extracted from token kind; used for
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java
index 8592ad490184e..27bc52591466f 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java
@@ -44,12 +44,12 @@
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
-import org.apache.hadoop.fs.s3a.S3AInstrumentation;
import org.apache.hadoop.fs.s3a.S3AUtils;
import org.apache.hadoop.fs.s3a.WriteOperationHelper;
import org.apache.hadoop.fs.s3a.commit.files.PendingSet;
import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit;
import org.apache.hadoop.fs.s3a.commit.files.SuccessData;
+import org.apache.hadoop.fs.s3a.impl.statistics.CommitterStatistics;
import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.util.DurationInfo;
@@ -79,7 +79,7 @@ public class CommitOperations {
private final S3AFileSystem fs;
/** Statistics. */
- private final S3AInstrumentation.CommitterStatistics statistics;
+ private final CommitterStatistics statistics;
/**
* Write operations for the destination fs.
@@ -126,7 +126,7 @@ public String toString() {
}
/** @return statistics. */
- protected S3AInstrumentation.CommitterStatistics getStatistics() {
+ protected CommitterStatistics getStatistics() {
return statistics;
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java
index 7f9dadf06f320..1de2df0278303 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java
@@ -27,6 +27,7 @@
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.fs.s3a.Statistic;
import org.apache.hadoop.fs.s3a.commit.magic.MagicCommitTracker;
+import org.apache.hadoop.fs.s3a.impl.StoreContext;
import static org.apache.hadoop.fs.s3a.commit.MagicCommitPaths.*;
@@ -50,6 +51,8 @@ public class MagicCommitIntegration {
private final S3AFileSystem owner;
private final boolean magicCommitEnabled;
+ private final StoreContext storeContext;
+
/**
* Instantiate.
* @param owner owner class
@@ -59,6 +62,7 @@ public MagicCommitIntegration(S3AFileSystem owner,
boolean magicCommitEnabled) {
this.owner = owner;
this.magicCommitEnabled = magicCommitEnabled;
+ storeContext = owner.createStoreContext();
}
/**
@@ -94,10 +98,9 @@ public PutTracker createTracker(Path path, String key) {
if (isMagicCommitPath(elements)) {
final String destKey = keyOfFinalDestination(elements, key);
String pendingsetPath = key + CommitConstants.PENDING_SUFFIX;
- owner.getInstrumentation()
- .incrementCounter(Statistic.COMMITTER_MAGIC_FILES_CREATED, 1);
+ storeContext.incrementStatistic(Statistic.COMMITTER_MAGIC_FILES_CREATED);
tracker = new MagicCommitTracker(path,
- owner.getBucket(),
+ storeContext.getBucket(),
key,
destKey,
pendingsetPath,
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ActiveOperationContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ActiveOperationContext.java
new file mode 100644
index 0000000000000..946c27ecc1eb7
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ActiveOperationContext.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.impl;
+
+import javax.annotation.Nullable;
+
+import java.util.Objects;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.fs.s3a.impl.statistics.S3AStatisticsContext;
+import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
+
+/**
+ * Context for any active operation.
+ */
+public class ActiveOperationContext {
+
+ /**
+ * An operation ID; currently just for logging...proper tracing needs more.
+ */
+ private final long operationId;
+
+ /**
+ * Statistics context
+ */
+ private final S3AStatisticsContext statisticsContext;
+
+ /**
+ * S3Guard bulk operation state, if (currently) set.
+ */
+ @Nullable private BulkOperationState bulkOperationState;
+
+ public ActiveOperationContext(
+ final long operationId,
+ final S3AStatisticsContext statisticsContext,
+ @Nullable final BulkOperationState bulkOperationState) {
+ this.operationId = operationId;
+ this.statisticsContext = Objects.requireNonNull(statisticsContext,
+ "null statistics context");
+ this.bulkOperationState = bulkOperationState;
+ }
+
+ @Override
+ public String toString() {
+ final StringBuilder sb = new StringBuilder(
+ "ActiveOperation{");
+ sb.append("operationId=").append(operationId);
+ sb.append(", bulkOperationState=").append(bulkOperationState);
+ sb.append('}');
+ return sb.toString();
+ }
+
+ @Nullable
+ public BulkOperationState getBulkOperationState() {
+ return bulkOperationState;
+ }
+
+ public long getOperationId() {
+ return operationId;
+ }
+
+ public S3AStatisticsContext getS3AStatisticsContext() {
+ return statisticsContext;
+ }
+
+ private static final AtomicLong nextOperationId = new AtomicLong(0);
+
+ /**
+ * Create an operation ID. The nature of it should be opaque.
+ * @return an ID for the constructor.
+ */
+ protected static long newOperationId() {
+ return nextOperationId.incrementAndGet();
+ }
+
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteRetryHandler.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteRetryHandler.java
index b2c1cc6271ec7..7c9189d2353d3 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteRetryHandler.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteRetryHandler.java
@@ -26,9 +26,9 @@
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.s3a.AWSClientIOException;
-import org.apache.hadoop.fs.s3a.S3AInstrumentation;
import org.apache.hadoop.fs.s3a.S3AStorageStatistics;
import org.apache.hadoop.fs.s3a.Statistic;
+import org.apache.hadoop.fs.s3a.impl.statistics.S3AStatisticsContext;
import static org.apache.hadoop.fs.s3a.S3AUtils.isThrottleException;
import static org.apache.hadoop.fs.s3a.Statistic.IGNORED_ERRORS;
@@ -53,7 +53,7 @@ public class BulkDeleteRetryHandler extends AbstractStoreOperation {
*/
public static final String XML_PARSE_BROKEN = "Failed to parse XML document";
- private final S3AInstrumentation instrumentation;
+ private final S3AStatisticsContext instrumentation;
private final S3AStorageStatistics storageStatistics;
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java
index d34328ccbfe7c..649532cfac14b 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java
@@ -18,8 +18,6 @@
package org.apache.hadoop.fs.s3a.impl;
-import java.util.concurrent.atomic.AtomicLong;
-
import com.amazonaws.AmazonServiceException;
import com.amazonaws.SdkBaseException;
import com.amazonaws.services.s3.model.CopyObjectRequest;
@@ -39,6 +37,7 @@
import org.apache.hadoop.fs.s3a.NoVersionAttributeException;
import org.apache.hadoop.fs.s3a.RemoteFileChangedException;
import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
+import org.apache.hadoop.fs.s3a.impl.statistics.ChangeTrackerStatistics;
import static com.google.common.base.Preconditions.checkNotNull;
@@ -72,7 +71,7 @@ public class ChangeTracker {
* Mismatch counter; expected to be wired up to StreamStatistics except
* during testing.
*/
- private final AtomicLong versionMismatches;
+ private final ChangeTrackerStatistics versionMismatches;
/**
* Revision identifier (e.g. eTag or versionId, depending on change
@@ -90,7 +89,7 @@ public class ChangeTracker {
*/
public ChangeTracker(final String uri,
final ChangeDetectionPolicy policy,
- final AtomicLong versionMismatches,
+ final ChangeTrackerStatistics versionMismatches,
final S3ObjectAttributes s3ObjectAttributes) {
this.policy = checkNotNull(policy);
this.uri = uri;
@@ -111,8 +110,8 @@ public ChangeDetectionPolicy.Source getSource() {
}
@VisibleForTesting
- public AtomicLong getVersionMismatches() {
- return versionMismatches;
+ public long getVersionMismatches() {
+ return versionMismatches.getVersionMismatches();
}
/**
@@ -177,7 +176,7 @@ public void processResponse(final S3Object object,
if (revisionId != null) {
// the requirements of the change detection policy wasn't met: the
// object was not returned.
- versionMismatches.incrementAndGet();
+ versionMismatches.versionMismatchError();
throw new RemoteFileChangedException(uri, operation,
String.format(CHANGE_REPORTED_BY_S3
+ " during %s"
@@ -235,7 +234,7 @@ public void processException(SdkBaseException e, String operation) throws
// This isn't really going to be hit due to
// https://github.com/aws/aws-sdk-java/issues/1644
if (serviceException.getStatusCode() == SC_PRECONDITION_FAILED) {
- versionMismatches.incrementAndGet();
+ versionMismatches.versionMismatchError();
throw new RemoteFileChangedException(uri, operation, String.format(
RemoteFileChangedException.PRECONDITIONS_FAILED
+ " on %s."
@@ -292,10 +291,10 @@ private void processNewRevision(final String newRevisionId,
uri,
pos,
operation,
- versionMismatches.get());
+ versionMismatches.getVersionMismatches());
if (pair.left) {
// an mismatch has occurred: note it.
- versionMismatches.incrementAndGet();
+ versionMismatches.versionMismatchError();
}
if (pair.right != null) {
// there's an exception to raise: do it
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java
index 88480db753515..7e0b4fe39cfc4 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java
@@ -31,9 +31,9 @@
import org.apache.hadoop.fs.s3a.Invoker;
import org.apache.hadoop.fs.s3a.S3AFileStatus;
import org.apache.hadoop.fs.s3a.S3AInputPolicy;
-import org.apache.hadoop.fs.s3a.S3AInstrumentation;
import org.apache.hadoop.fs.s3a.S3AStorageStatistics;
import org.apache.hadoop.fs.s3a.Statistic;
+import org.apache.hadoop.fs.s3a.impl.statistics.S3AStatisticsContext;
import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider;
import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
import org.apache.hadoop.security.UserGroupInformation;
@@ -84,7 +84,7 @@ public class StoreContext {
private final Invoker invoker;
/** Instrumentation and statistics. */
- private final S3AInstrumentation instrumentation;
+ private final S3AStatisticsContext instrumentation;
private final S3AStorageStatistics storageStatistics;
/** Seek policy. */
@@ -126,7 +126,7 @@ public StoreContext(
final ListeningExecutorService executor,
final int executorCapacity,
final Invoker invoker,
- final S3AInstrumentation instrumentation,
+ final S3AStatisticsContext instrumentation,
final S3AStorageStatistics storageStatistics,
final S3AInputPolicy inputPolicy,
final ChangeDetectionPolicy changeDetectionPolicy,
@@ -183,7 +183,7 @@ public Invoker getInvoker() {
return invoker;
}
- public S3AInstrumentation getInstrumentation() {
+ public S3AStatisticsContext getInstrumentation() {
return instrumentation;
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/AwsStatisticsCollector.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/AwsStatisticsCollector.java
new file mode 100644
index 0000000000000..7b45258f38c97
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/AwsStatisticsCollector.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.impl.statistics;
+
+import java.time.Duration;
+import java.util.function.Consumer;
+import java.util.function.LongConsumer;
+
+import com.amazonaws.Request;
+import com.amazonaws.Response;
+import com.amazonaws.metrics.RequestMetricCollector;
+import com.amazonaws.util.TimingInfo;
+
+import static com.amazonaws.util.AWSRequestMetrics.Field.ClientExecuteTime;
+import static com.amazonaws.util.AWSRequestMetrics.Field.HttpClientRetryCount;
+import static com.amazonaws.util.AWSRequestMetrics.Field.HttpRequestTime;
+import static com.amazonaws.util.AWSRequestMetrics.Field.RequestCount;
+import static com.amazonaws.util.AWSRequestMetrics.Field.RequestMarshallTime;
+import static com.amazonaws.util.AWSRequestMetrics.Field.RequestSigningTime;
+import static com.amazonaws.util.AWSRequestMetrics.Field.ResponseProcessingTime;
+import static com.amazonaws.util.AWSRequestMetrics.Field.ThrottleException;
+
+/**
+ * Collect statistics from the AWS SDK and update our statistics.
+ *
+ * See {@code com.facebook.presto.hive.s3.PrestoS3FileSystemMetricCollector}
+ * for the inspiration for this.
+ * See {@code com.amazonaws.util.AWSRequestMetrics} for metric names.
+ */
+public class AwsStatisticsCollector extends RequestMetricCollector {
+
+ /**
+ * final destination of updates.
+ */
+ private final StatisticsFromAwsSdk collector;
+
+ /**
+ * Instantiate.
+ * @param collector final destination of updates
+ */
+ public AwsStatisticsCollector(final StatisticsFromAwsSdk collector) {
+ this.collector = collector;
+ }
+
+ @Override
+ public void collectMetrics(
+ final Request> request,
+ final Response> response) {
+
+ TimingInfo timingInfo = request.getAWSRequestMetrics().getTimingInfo();
+
+ counter(timingInfo, HttpClientRetryCount.name(),
+ collector::updateAwsRetryCount);
+ counter(timingInfo, RequestCount.name(),
+ collector::updateAwsRequestCount);
+ counter(timingInfo, ThrottleException.name(),
+ collector::updateAwsThrottleExceptionsCount);
+
+ timing(timingInfo, ClientExecuteTime.name(),
+ collector::noteAwsClientExecuteTime);
+ timing(timingInfo, HttpRequestTime.name(),
+ collector::noteAwsRequestTime);
+ timing(timingInfo, RequestMarshallTime.name(),
+ collector::noteRequestMarshallTime);
+ timing(timingInfo, RequestSigningTime.name(),
+ collector::noteRequestSigningTime);
+ timing(timingInfo, ResponseProcessingTime.name(),
+ collector::noteResponseProcessingTime);
+ }
+
+ /**
+ * Process a timing.
+ * @param timingInfo timing info
+ * @param subMeasurementName sub measurement
+ * @param durationConsumer consumer
+ */
+ private void timing(
+ TimingInfo timingInfo,
+ String subMeasurementName,
+ Consumer durationConsumer) {
+ TimingInfo t1 = timingInfo.getSubMeasurement(subMeasurementName);
+ if (t1 != null && t1.getTimeTakenMillisIfKnown() != null) {
+ durationConsumer.accept(Duration.ofMillis(
+ t1.getTimeTakenMillisIfKnown().longValue()));
+ }
+ }
+
+ /**
+ * Process a counter.
+ * @param timingInfo timing info
+ * @param subMeasurementName sub measurement
+ * @param consumer consumer
+ */
+ private void counter(
+ TimingInfo timingInfo,
+ String subMeasurementName,
+ LongConsumer consumer) {
+ Number n = timingInfo.getCounter(subMeasurementName);
+ if (n != null) {
+ consumer.accept(n.longValue());
+ }
+ }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/BlockOutputStreamStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/BlockOutputStreamStatistics.java
new file mode 100644
index 0000000000000..77d73e115392a
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/BlockOutputStreamStatistics.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.impl.statistics;
+
+import java.io.Closeable;
+
+import org.apache.hadoop.fs.statistics.IOStatistics;
+
+/**
+ * Block output stream statistics.
+ */
+public interface BlockOutputStreamStatistics extends Closeable {
+
+ /**
+ * Block is queued for upload.
+ */
+ void blockUploadQueued(int blockSize);
+
+ /** Queued block has been scheduled for upload. */
+ void blockUploadStarted(long duration, int blockSize);
+
+ /** A block upload has completed. */
+ void blockUploadCompleted(long duration, int blockSize);
+
+ /**
+ * A block upload has failed.
+ * A final transfer completed event is still expected, so this
+ * does not decrement the active block counter.
+ */
+ void blockUploadFailed(long duration, int blockSize);
+
+ /** Intermediate report of bytes uploaded. */
+ void bytesTransferred(long byteCount);
+
+ /**
+ * Note exception in a multipart complete.
+ * @param count count of exceptions
+ */
+ void exceptionInMultipartComplete(int count);
+
+ /**
+ * Note an exception in a multipart abort.
+ */
+ void exceptionInMultipartAbort();
+
+ /**
+ * Get the number of bytes pending upload.
+ * @return the number of bytes in the pending upload state.
+ */
+ long getBytesPendingUpload();
+
+ /**
+ * Data has been uploaded to be committed in a subsequent operation;
+ * to be called at the end of the write.
+ * @param size size in bytes
+ */
+ void commitUploaded(long size);
+
+ int getBlocksAllocated();
+
+ int getBlocksReleased();
+
+ /**
+ * Get counters of blocks actively allocated; my be inaccurate
+ * if the numbers change during the (non-synchronized) calculation.
+ * @return the number of actively allocated blocks.
+ */
+ int getBlocksActivelyAllocated();
+
+ /**
+ * Record bytes written.
+ * @param count number of bytes
+ */
+ void writeBytes(long count);
+
+ /**
+ * Get the current count of bytes written.
+ * @return the counter value.
+ */
+ long getBytesWritten();
+
+ /**
+ * Convert to an IOStatistics source which is
+ * dynamically updated.
+ * @return statistics
+ */
+ IOStatistics createIOStatistics();
+
+ /**
+ * A block has been allocated.
+ */
+ void blockAllocated();
+
+ /**
+ * A block has been released.
+ */
+ void blockReleased();
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/ChangeTrackerStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/ChangeTrackerStatistics.java
new file mode 100644
index 0000000000000..6b58fa8b06c9b
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/ChangeTrackerStatistics.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.impl.statistics;
+
+/**
+ * Interface for change tracking.
+ */
+public interface ChangeTrackerStatistics {
+
+ /**
+ * A version mismatch was detected.
+ */
+ void versionMismatchError();
+
+ /**
+ * How many version mismatches have occurred.
+ */
+ long getVersionMismatches();
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/CommitterStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/CommitterStatistics.java
new file mode 100644
index 0000000000000..b3fa552bef7e4
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/CommitterStatistics.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.impl.statistics;
+
+/**
+ * Statistics for committed work.
+ */
+public interface CommitterStatistics {
+
+ /** A commit has been created. */
+ void commitCreated();
+
+ /**
+ * Data has been uploaded to be committed in a subsequent operation.
+ * @param size size in bytes
+ */
+ void commitUploaded(long size);
+
+ /**
+ * A commit has been completed.
+ * @param size size in bytes
+ */
+ void commitCompleted(long size);
+
+ /** A commit has been aborted. */
+ void commitAborted();
+
+ /**
+ * A commit was reverted.
+ */
+ void commitReverted();
+
+ /**
+ * A commit failed.
+ */
+ void commitFailed();
+
+ /**
+ * Note that a task has completed.
+ * @param success success flag
+ */
+ void taskCompleted(boolean success);
+
+ /**
+ * Note that a job has completed.
+ * @param success success flag
+ */
+ void jobCompleted(boolean success);
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/CountersAndGauges.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/CountersAndGauges.java
new file mode 100644
index 0000000000000..5a0651f0878bf
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/CountersAndGauges.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.impl.statistics;
+
+import org.apache.hadoop.fs.s3a.Statistic;
+
+/**
+ * This is the foundational API for collecting S3A statistics.
+ */
+public interface CountersAndGauges {
+
+ /**
+ * Increment a specific counter.
+ * No-op if not defined.
+ * @param op operation
+ * @param count increment value
+ */
+ void incrementCounter(Statistic op, long count);
+
+ /**
+ * Increment a specific gauge.
+ * No-op if not defined.
+ * @param op operation
+ * @param count increment value
+ * @throws ClassCastException if the metric is of the wrong type
+ */
+ void incrementGauge(Statistic op, long count);
+
+ /**
+ * Decrement a specific gauge.
+ * No-op if not defined.
+ * @param op operation
+ * @param count increment value
+ * @throws ClassCastException if the metric is of the wrong type
+ */
+ void decrementGauge(Statistic op, long count);
+
+ /**
+ * Add a value to a quantiles statistic. No-op if the quantile
+ * isn't found.
+ * @param op operation to look up.
+ * @param value value to add.
+ * @throws ClassCastException if the metric is not a Quantiles.
+ */
+ void addValueToQuantiles(Statistic op, long value);
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/CountingChangeTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/CountingChangeTracker.java
new file mode 100644
index 0000000000000..3581267b354ef
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/CountingChangeTracker.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.impl.statistics;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * A change tracker which increments an atomic long.
+ */
+public class CountingChangeTracker implements
+ ChangeTrackerStatistics {
+
+ /**
+ * The counter which is updated on every mismatch.
+ */
+ private final AtomicLong counter;
+
+ public CountingChangeTracker(final AtomicLong counter) {
+ this.counter = counter;
+ }
+
+ public CountingChangeTracker() {
+ this(new AtomicLong());
+ }
+
+ @Override
+ public void versionMismatchError() {
+ counter.incrementAndGet();
+ }
+
+ @Override
+ public long getVersionMismatches() {
+ return counter.get();
+ }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/DelegationTokenStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/DelegationTokenStatistics.java
new file mode 100644
index 0000000000000..d918b8743985c
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/DelegationTokenStatistics.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.impl.statistics;
+
+/**
+ * Instrumentation exported to for S3A Delegation Token support.
+ */
+public interface DelegationTokenStatistics {
+
+ /** A token has been issued. */
+ void tokenIssued();
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/EmptyS3AStatisticsContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/EmptyS3AStatisticsContext.java
new file mode 100644
index 0000000000000..03ce2aed5b5a2
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/EmptyS3AStatisticsContext.java
@@ -0,0 +1,457 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.impl.statistics;
+
+import java.io.IOException;
+import java.time.Duration;
+
+import org.apache.hadoop.fs.s3a.Statistic;
+import org.apache.hadoop.fs.s3a.s3guard.MetastoreInstrumentation;
+import org.apache.hadoop.fs.s3a.s3guard.MetastoreInstrumentationImpl;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.impl.EmptyIOStatistics;
+
+/**
+ * Special statistics context, all of whose context operations are no-ops.
+ * All statistics instances it returns are also empty.
+ *
+ * This class is here primarily to aid in testing, but it also allows for
+ * classes to require a non-empty statistics context in their constructor -yet
+ * still be instantiated without one bound to any filesystem.
+ */
+public final class EmptyS3AStatisticsContext implements S3AStatisticsContext {
+
+ @Override
+ public MetastoreInstrumentation getMetastoreInstrumentation() {
+ return new MetastoreInstrumentationImpl();
+ }
+
+ @Override
+ public S3AInputStreamStatistics newInputStreamStatistics() {
+ return new EmptyInputStreamStatistics();
+ }
+
+ @Override
+ public CommitterStatistics newCommitterStatistics() {
+ return new EmptyCommitterStatistics();
+ }
+
+ @Override
+ public BlockOutputStreamStatistics newOutputStreamStatistics() {
+ return new EmptyBlockOutputStreamStatistics();
+ }
+
+ @Override
+ public DelegationTokenStatistics newDelegationTokenStatistics() {
+ return new EmptyDelegationTokenStatistics();
+ }
+
+ @Override
+ public StatisticsFromAwsSdk newStatisticsFromAwsSdk() {
+ return new EmptyStatisticsFromAwsSdk();
+ }
+
+ @Override
+ public void incrementCounter(final Statistic op, final long count) {
+
+ }
+
+ @Override
+ public void incrementGauge(final Statistic op, final long count) {
+
+ }
+
+ @Override
+ public void decrementGauge(final Statistic op, final long count) {
+
+ }
+
+ @Override
+ public void addValueToQuantiles(final Statistic op, final long value) {
+
+ }
+
+ /**
+ * Input Stream statistics callbacks.
+ */
+ private static final class EmptyInputStreamStatistics
+ implements S3AInputStreamStatistics {
+
+ @Override
+ public void seekBackwards(final long negativeOffset) {
+
+ }
+
+ @Override
+ public void seekForwards(final long skipped) {
+
+ }
+
+ @Override
+ public long streamOpened() {
+ return 0;
+ }
+
+ @Override
+ public void streamClose(final boolean abortedConnection,
+ final long remainingInCurrentRequest) {
+
+ }
+
+ @Override
+ public void readException() {
+
+ }
+
+ @Override
+ public void bytesRead(final long bytes) {
+
+ }
+
+ @Override
+ public void readOperationStarted(final long pos, final long len) {
+
+ }
+
+ @Override
+ public void readFullyOperationStarted(final long pos, final long len) {
+
+ }
+
+ @Override
+ public void readOperationCompleted(final int requested, final int actual) {
+
+ }
+
+ @Override
+ public void close() {
+
+ }
+
+ @Override
+ public void inputPolicySet(final int updatedPolicy) {
+
+ }
+
+ @Override
+ public void merge(final boolean isClosed) {
+
+ }
+
+ /**
+ * Return an IO statistics instance.
+ * @return an empty IO statistics instance.
+ */
+ @Override
+ public IOStatistics createIOStatistics() {
+ return EmptyIOStatistics.getInstance();
+ }
+
+ @Override
+ public long getCloseOperations() {
+ return 0;
+ }
+
+ @Override
+ public long getClosed() {
+ return 0;
+ }
+
+ @Override
+ public long getAborted() {
+ return 0;
+ }
+
+ @Override
+ public long getForwardSeekOperations() {
+ return 0;
+ }
+
+ @Override
+ public long getBackwardSeekOperations() {
+ return 0;
+ }
+
+ @Override
+ public long getBytesRead() {
+ return 0;
+ }
+
+ @Override
+ public long getBytesSkippedOnSeek() {
+ return 0;
+ }
+
+ @Override
+ public long getBytesBackwardsOnSeek() {
+ return 0;
+ }
+
+ @Override
+ public long getBytesReadInClose() {
+ return 0;
+ }
+
+ @Override
+ public long getBytesDiscardedInAbort() {
+ return 0;
+ }
+
+ @Override
+ public long getOpenOperations() {
+ return 0;
+ }
+
+ @Override
+ public long getSeekOperations() {
+ return 0;
+ }
+
+ @Override
+ public long getReadExceptions() {
+ return 0;
+ }
+
+ @Override
+ public long getReadOperations() {
+ return 0;
+ }
+
+ @Override
+ public long getReadFullyOperations() {
+ return 0;
+ }
+
+ @Override
+ public long getReadsIncomplete() {
+ return 0;
+ }
+
+ @Override
+ public long getPolicySetCount() {
+ return 0;
+ }
+
+ @Override
+ public long getVersionMismatches() {
+ return 0;
+ }
+
+ @Override
+ public long getInputPolicy() {
+ return 0;
+ }
+
+ @Override
+ public ChangeTrackerStatistics getChangeTrackerStatistics() {
+ return new CountingChangeTracker();
+ }
+ }
+
+ private static final class EmptyCommitterStatistics
+ implements CommitterStatistics {
+
+ @Override
+ public void commitCreated() {
+
+ }
+
+ @Override
+ public void commitUploaded(final long size) {
+
+ }
+
+ @Override
+ public void commitCompleted(final long size) {
+
+ }
+
+ @Override
+ public void commitAborted() {
+
+ }
+
+ @Override
+ public void commitReverted() {
+
+ }
+
+ @Override
+ public void commitFailed() {
+
+ }
+
+ @Override
+ public void taskCompleted(final boolean success) {
+
+ }
+
+ @Override
+ public void jobCompleted(final boolean success) {
+
+ }
+ }
+
+ private static final class EmptyBlockOutputStreamStatistics
+ implements BlockOutputStreamStatistics {
+
+ @Override
+ public void blockUploadQueued(final int blockSize) {
+
+ }
+
+ @Override
+ public void blockUploadStarted(final long duration, final int blockSize) {
+
+ }
+
+ @Override
+ public void blockUploadCompleted(final long duration, final int blockSize) {
+
+ }
+
+ @Override
+ public void blockUploadFailed(final long duration, final int blockSize) {
+
+ }
+
+ @Override
+ public void bytesTransferred(final long byteCount) {
+
+ }
+
+ @Override
+ public void exceptionInMultipartComplete(final int count) {
+
+ }
+
+ @Override
+ public void exceptionInMultipartAbort() {
+
+ }
+
+ @Override
+ public long getBytesPendingUpload() {
+ return 0;
+ }
+
+ @Override
+ public void commitUploaded(final long size) {
+
+ }
+
+ @Override
+ public int getBlocksAllocated() {
+ return 0;
+ }
+
+ @Override
+ public int getBlocksReleased() {
+ return 0;
+ }
+
+ @Override
+ public int getBlocksActivelyAllocated() {
+ return 0;
+ }
+
+ @Override
+ public IOStatistics createIOStatistics() {
+ return EmptyIOStatistics.getInstance();
+ }
+
+ @Override
+ public void blockAllocated() {
+
+ }
+
+ @Override
+ public void blockReleased() {
+
+ }
+
+ @Override
+ public void writeBytes(final long count) {
+ }
+
+ @Override
+ public long getBytesWritten() {
+ return 0;
+ }
+
+ @Override
+ public void close() throws IOException {
+
+ }
+ }
+
+ private static final class EmptyDelegationTokenStatistics
+ implements DelegationTokenStatistics {
+
+ @Override
+ public void tokenIssued() {
+
+ }
+ }
+
+ private static final class EmptyStatisticsFromAwsSdk implements
+ StatisticsFromAwsSdk {
+
+ @Override
+ public void updateAwsRequestCount(final long longValue) {
+
+ }
+
+ @Override
+ public void updateAwsRetryCount(final long longValue) {
+
+ }
+
+ @Override
+ public void updateAwsThrottleExceptionsCount(final long longValue) {
+
+ }
+
+ @Override
+ public void noteAwsRequestTime(final Duration ofMillis) {
+
+ }
+
+ @Override
+ public void noteAwsClientExecuteTime(final Duration ofMillis) {
+
+ }
+
+ @Override
+ public void noteRequestMarshallTime(final Duration duration) {
+
+ }
+
+ @Override
+ public void noteRequestSigningTime(final Duration duration) {
+
+ }
+
+ @Override
+ public void noteResponseProcessingTime(final Duration duration) {
+
+ }
+ }
+
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/IntegratedS3AStatisticsContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/IntegratedS3AStatisticsContext.java
new file mode 100644
index 0000000000000..faeb9b434e331
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/IntegratedS3AStatisticsContext.java
@@ -0,0 +1,201 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.impl.statistics;
+
+import javax.annotation.Nullable;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.s3a.S3AInstrumentation;
+import org.apache.hadoop.fs.s3a.Statistic;
+import org.apache.hadoop.fs.s3a.s3guard.MetastoreInstrumentation;
+
+/**
+ * An S3A statistics context which is bonded to a
+ * S3AInstrumentation instance -inevitably that of an S3AFileSystem
+ * instance.
+ * An interface is used to bind to the relevant fields, rather
+ * than have them passed in the constructor because some
+ * production code, specifically, DelegateToFileSystem,
+ * patches the protected field after initialization.
+ *
+ * All operations are passed through directly to that class.
+ *
+ * If an instance of FileSystem.Statistics is passed in, it
+ * will be used whenever input stream statistics are created -
+ * However, Internally always increments the statistics in the
+ * current thread.
+ * As a result, cross-thread IO will under-report.
+ *
+ * This is addressed through the stream statistics classes
+ * only updating the stats in the close() call. Provided
+ * they are closed in the worker thread, all stats collected in
+ * helper threads will be included.
+ */
+public class IntegratedS3AStatisticsContext implements S3AStatisticsContext {
+
+ private final S3AFSStatisticsSource statisticsSource;
+
+ /**
+ * Instantiate.
+ * @param statisticsSource integration binding
+ */
+ public IntegratedS3AStatisticsContext(
+ final S3AFSStatisticsSource statisticsSource) {
+ this.statisticsSource = statisticsSource;
+ }
+
+
+ /**
+ * Get the instrumentation from the FS integraation.
+ * @return instrumentation instance.
+ */
+ private S3AInstrumentation getInstrumentation() {
+ return statisticsSource.getInstrumentation();
+ }
+
+ /**
+ * The filesystem statistics: know this is thread-local.
+ * @return FS statistics.
+ */
+ private FileSystem.Statistics getInstanceStatistics() {
+ return statisticsSource.getInstanceStatistics();
+ }
+
+ /**
+ * Get a MetastoreInstrumentation getInstrumentation() instance for this
+ * context.
+ * @return the S3Guard getInstrumentation() point.
+ */
+ @Override
+ public MetastoreInstrumentation getMetastoreInstrumentation() {
+ return getInstrumentation().getS3GuardInstrumentation();
+ }
+
+ /**
+ * Create a stream input statistics instance.
+ * @return the new instance
+ */
+ @Override
+ public S3AInputStreamStatistics newInputStreamStatistics() {
+ return getInstrumentation().newInputStreamStatistics(
+ statisticsSource.getInstanceStatistics());
+ }
+
+ /**
+ * Create a new instance of the committer statistics.
+ * @return a new committer statistics instance
+ */
+ @Override
+ public CommitterStatistics newCommitterStatistics() {
+ return getInstrumentation().newCommitterStatistics();
+ }
+
+ /**
+ * Create a stream output statistics instance.
+ * @return the new instance
+ */
+ @Override
+ public BlockOutputStreamStatistics newOutputStreamStatistics() {
+ return getInstrumentation()
+ .newOutputStreamStatistics(getInstanceStatistics());
+ }
+
+ /**
+ * Increment a specific counter.
+ * No-op if not defined.
+ * @param op operation
+ * @param count increment value
+ */
+ @Override
+ public void incrementCounter(Statistic op, long count) {
+ getInstrumentation().incrementCounter(op, count);
+ }
+
+ /**
+ * Increment a specific gauge.
+ * No-op if not defined.
+ * @param op operation
+ * @param count increment value
+ * @throws ClassCastException if the metric is of the wrong type
+ */
+ @Override
+ public void incrementGauge(Statistic op, long count) {
+ getInstrumentation().incrementGauge(op, count);
+ }
+
+ /**
+ * Decrement a specific gauge.
+ * No-op if not defined.
+ * @param op operation
+ * @param count increment value
+ * @throws ClassCastException if the metric is of the wrong type
+ */
+ @Override
+ public void decrementGauge(Statistic op, long count) {
+ getInstrumentation().decrementGauge(op, count);
+ }
+
+ /**
+ * Add a value to a quantiles statistic. No-op if the quantile
+ * isn't found.
+ * @param op operation to look up.
+ * @param value value to add.
+ * @throws ClassCastException if the metric is not a Quantiles.
+ */
+ @Override
+ public void addValueToQuantiles(Statistic op, long value) {
+ getInstrumentation().addValueToQuantiles(op, value);
+ }
+
+ /**
+ * Create a delegation token statistics instance.
+ * @return an instance of delegation token statistics
+ */
+ @Override
+ public DelegationTokenStatistics newDelegationTokenStatistics() {
+ return getInstrumentation().newDelegationTokenStatistics();
+ }
+
+ @Override
+ public StatisticsFromAwsSdk newStatisticsFromAwsSdk() {
+ return getInstrumentation().newStatisticsFromAwsSdk();
+ }
+
+ /**
+ * This is the interface which an integration source must implement
+ * for the integration.
+ * Note that the FileSystem.statistics field may be null for a class;
+ */
+ public interface S3AFSStatisticsSource {
+
+ /**
+ * Get the S3A Instrumentation.
+ * @return a non-null instrumentation instance
+ */
+ S3AInstrumentation getInstrumentation();
+
+ /**
+ * Get the statistics of the FS instance, shared across all threads.
+ * @return filesystem statistics
+ */
+ @Nullable
+ FileSystem.Statistics getInstanceStatistics();
+
+ }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AInputStreamStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AInputStreamStatistics.java
new file mode 100644
index 0000000000000..3c1023f653f1b
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AInputStreamStatistics.java
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.impl.statistics;
+
+import org.apache.hadoop.fs.statistics.IOStatistics;
+
+/**
+ * Statistics updated by an input stream during its actual operation.
+ * It also contains getters for tests.
+ */
+public interface S3AInputStreamStatistics extends AutoCloseable {
+
+ /**
+ * Seek backwards, incrementing the seek and backward seek counters.
+ * @param negativeOffset how far was the seek?
+ * This is expected to be negative.
+ */
+ void seekBackwards(long negativeOffset);
+
+ /**
+ * Record a forward seek, adding a seek operation, a forward
+ * seek operation, and any bytes skipped.
+ * @param skipped number of bytes skipped by reading from the stream.
+ * If the seek was implemented by a close + reopen, set this to zero.
+ */
+ void seekForwards(long skipped);
+
+ /**
+ * The inner stream was opened.
+ * @return the previous count
+ */
+ long streamOpened();
+
+ /**
+ * The inner stream was closed.
+ * @param abortedConnection flag to indicate the stream was aborted,
+ * rather than closed cleanly
+ * @param remainingInCurrentRequest the number of bytes remaining in
+ * the current request.
+ */
+ void streamClose(boolean abortedConnection,
+ long remainingInCurrentRequest);
+
+ /**
+ * An ignored stream read exception was received.
+ */
+ void readException();
+
+ /**
+ * Increment the bytes read counter by the number of bytes;
+ * no-op if the argument is negative.
+ * @param bytes number of bytes read
+ */
+ void bytesRead(long bytes);
+
+ /**
+ * A {@code read(byte[] buf, int off, int len)} operation has started.
+ * @param pos starting position of the read
+ * @param len length of bytes to read
+ */
+ void readOperationStarted(long pos, long len);
+
+ /**
+ * A {@code PositionedRead.read(position, buffer, offset, length)}
+ * operation has just started.
+ * @param pos starting position of the read
+ * @param len length of bytes to read
+ */
+ void readFullyOperationStarted(long pos, long len);
+
+ /**
+ * A read operation has completed.
+ * @param requested number of requested bytes
+ * @param actual the actual number of bytes
+ */
+ void readOperationCompleted(int requested, int actual);
+
+ @Override
+ void close();
+
+ /**
+ * The input policy has been switched.
+ * @param updatedPolicy enum value of new policy.
+ */
+ void inputPolicySet(int updatedPolicy);
+
+ /**
+ * Get a reference to the change tracker statistics for this
+ * stream.
+ * @return a reference to the change tracker statistics
+ */
+ ChangeTrackerStatistics getChangeTrackerStatistics();
+
+ /**
+ * Merge the statistics into the filesystem's instrumentation instance.
+ * Takes a diff between the current version of the stats and the
+ * version of the stats when merge was last called, and merges the diff
+ * into the instrumentation instance. Used to periodically merge the
+ * stats into the fs-wide stats. Behavior is undefined if called on a
+ * closed instance.
+ */
+ void merge(boolean isClosed);
+
+ /**
+ * Convert to an IOStatistics source which is
+ * dynamically updated.
+ * @return statistics
+ */
+ IOStatistics createIOStatistics();
+
+ long getCloseOperations();
+
+ long getClosed();
+
+ long getAborted();
+
+ long getForwardSeekOperations();
+
+ long getBackwardSeekOperations();
+
+ long getBytesRead();
+
+ long getBytesSkippedOnSeek();
+
+ long getBytesBackwardsOnSeek();
+
+ long getBytesReadInClose();
+
+ long getBytesDiscardedInAbort();
+
+ long getOpenOperations();
+
+ long getSeekOperations();
+
+ long getReadExceptions();
+
+ long getReadOperations();
+
+ long getReadFullyOperations();
+
+ long getReadsIncomplete();
+
+ long getPolicySetCount();
+
+ long getVersionMismatches();
+
+ long getInputPolicy();
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AStatisticsContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AStatisticsContext.java
new file mode 100644
index 0000000000000..80cb4bd8bd44d
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/S3AStatisticsContext.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.impl.statistics;
+
+import org.apache.hadoop.fs.s3a.s3guard.MetastoreInstrumentation;
+
+/**
+ * This is the statistics context for ongoing operations in S3A.
+ */
+public interface S3AStatisticsContext extends CountersAndGauges {
+
+ /**
+ * Get the metastore instrumentation.
+ * @return an instance of the metastore statistics tracking.
+ */
+ MetastoreInstrumentation getMetastoreInstrumentation();
+
+ /**
+ * Create a stream input statistics instance.
+ * @return the new instance
+ */
+ S3AInputStreamStatistics newInputStreamStatistics();
+
+ /**
+ * Create a new instance of the committer statistics.
+ * @return a new committer statistics instance
+ */
+ CommitterStatistics newCommitterStatistics();
+
+ /**
+ * Create a stream output statistics instance.
+ * @return the new instance
+ */
+ BlockOutputStreamStatistics newOutputStreamStatistics();
+
+ /**
+ * Create a delegation token statistics instance.
+ * @return an instance of delegation token statistics
+ */
+ DelegationTokenStatistics newDelegationTokenStatistics();
+
+ /**
+ * Create a StatisticsFromAwsSdk instance.
+ * @return an instance of StatisticsFromAwsSdk
+ */
+ StatisticsFromAwsSdk newStatisticsFromAwsSdk();
+
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/StatisticsFromAwsSdk.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/StatisticsFromAwsSdk.java
new file mode 100644
index 0000000000000..333e2d6649560
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/StatisticsFromAwsSdk.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.impl.statistics;
+
+import java.time.Duration;
+
+/**
+ * interface to receive statistics events from the AWS SDK
+ * by way of {@link AwsStatisticsCollector}.
+ */
+public interface StatisticsFromAwsSdk {
+
+ /**
+ * Record a number of AWS requests.
+ * @param count number of events.
+ */
+ void updateAwsRequestCount(long count);
+
+ /**
+ * Record a number of AWS request retries.
+ * @param count number of events.
+ */
+ void updateAwsRetryCount(long count);
+
+ /**
+ * Record a number of throttle exceptions received.
+ * @param count number of events.
+ */
+ void updateAwsThrottleExceptionsCount(long count);
+
+ /**
+ * Record how long a request took overall.
+ * @param duration duration of operation.
+ */
+ void noteAwsRequestTime(Duration duration);
+
+ /**
+ * Record how long a request took to execute on the
+ * client.
+ * @param duration duration of operation.
+ */
+ void noteAwsClientExecuteTime(Duration duration);
+
+ /**
+ * Record how long a request took to marshall into
+ * XML.
+ * @param duration duration of operation.
+ */
+ void noteRequestMarshallTime(Duration duration);
+
+ /**
+ * Record how long a request took to sign, including
+ * any calls to EC2 credential endpoints.
+ * @param duration duration of operation.
+ */
+ void noteRequestSigningTime(Duration duration);
+
+ /**
+ * Record how long it took to process the response.
+ * @param duration duration of operation.
+ */
+ void noteResponseProcessingTime(Duration duration);
+}
+
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/package-info.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/package-info.java
new file mode 100644
index 0000000000000..5d9d3b4d8d81a
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/statistics/package-info.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Statistics collection for the S3A connector.
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.fs.s3a.impl.statistics;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java
index 38b38fb7f93e7..7bf1b829c92bb 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java
@@ -450,7 +450,8 @@ void bindToOwnerFilesystem(final S3AFileSystem fs) {
owner = fs;
conf = owner.getConf();
StoreContext context = owner.createStoreContext();
- instrumentation = context.getInstrumentation().getS3GuardInstrumentation();
+ instrumentation = context.getInstrumentation()
+ .getMetastoreInstrumentation();
username = context.getUsername();
executor = context.createThrottledExecutor();
ttlTimeProvider = Preconditions.checkNotNull(
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectInputStream.java
index f4bd8d11708ef..b99ddb830c785 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectInputStream.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectInputStream.java
@@ -39,9 +39,9 @@
import org.apache.hadoop.fs.FSInputStream;
import org.apache.hadoop.fs.PathIOException;
import org.apache.hadoop.fs.s3a.Retries;
-import org.apache.hadoop.fs.s3a.S3AInstrumentation;
import org.apache.hadoop.fs.s3a.S3AReadOpContext;
import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
+import org.apache.hadoop.fs.s3a.impl.statistics.S3AInputStreamStatistics;
import org.apache.hadoop.io.IOUtils;
import static com.google.common.base.Preconditions.checkNotNull;
@@ -103,7 +103,7 @@ public class SelectInputStream extends FSInputStream implements
private final S3AReadOpContext readContext;
- private final S3AInstrumentation.InputStreamStatistics streamStatistics;
+ private final S3AInputStreamStatistics streamStatistics;
private long readahead;
@@ -130,7 +130,7 @@ public SelectInputStream(
this.uri = "s3a://" + this.bucket + "/" + this.key;
this.readContext = readContext;
this.readahead = readContext.getReadahead();
- this.streamStatistics = readContext.getInstrumentation()
+ this.streamStatistics = readContext.getS3AStatisticsContext()
.newInputStreamStatistics();
SelectRecordsInputStream stream = once(
"S3 Select",
@@ -428,7 +428,7 @@ private void incrementBytesRead(long bytesRead) {
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
- public S3AInstrumentation.InputStreamStatistics getS3AStreamStatistics() {
+ public S3AInputStreamStatistics getS3AStreamStatistics() {
return streamStatistics;
}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractStreamIOStatistics.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractStreamIOStatistics.java
new file mode 100644
index 0000000000000..024c970233fc4
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractStreamIOStatistics.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.contract.s3a;
+
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractStreamIOStatisticsTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeEnableS3Guard;
+import static org.apache.hadoop.fs.statistics.StreamStatisticNames.*;
+
+/**
+ * Test the S3A Streams IOStatistics support.
+ */
+public class ITestS3AContractStreamIOStatistics extends
+ AbstractContractStreamIOStatisticsTest {
+
+ @Override
+ protected Configuration createConfiguration() {
+ Configuration conf = super.createConfiguration();
+ // patch in S3Guard options
+ maybeEnableS3Guard(conf);
+ return conf;
+ }
+
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new S3AContract(conf);
+ }
+
+ /**
+ * Keys which the output stream must support.
+ * @return a list of keys
+ */
+ @Override
+ public List outputStreamStatisticKeys() {
+ return Arrays.asList(STREAM_WRITE_BYTES,
+ STREAM_WRITE_BLOCK_UPLOADS,
+ STREAM_WRITE_FAILURES);
+ }
+
+}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java
index afd3ec2bd3473..3e2786dbe475f 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java
@@ -22,6 +22,7 @@
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.s3a.impl.statistics.BlockOutputStreamStatistics;
import org.apache.hadoop.io.IOUtils;
import org.junit.BeforeClass;
@@ -94,16 +95,16 @@ public void testBlocksClosed() throws Throwable {
Path dest = path("testBlocksClosed");
describe(" testBlocksClosed");
FSDataOutputStream stream = getFileSystem().create(dest, true);
- S3AInstrumentation.OutputStreamStatistics statistics
+ BlockOutputStreamStatistics statistics
= S3ATestUtils.getOutputStreamStatistics(stream);
byte[] data = ContractTestUtils.dataset(16, 'a', 26);
stream.write(data);
LOG.info("closing output stream");
stream.close();
assertEquals("total allocated blocks in " + statistics,
- 1, statistics.blocksAllocated());
+ 1, statistics.getBlocksAllocated());
assertEquals("actively allocated blocks in " + statistics,
- 0, statistics.blocksActivelyAllocated());
+ 0, statistics.getBlocksActivelyAllocated());
LOG.info("end of test case");
}
@@ -129,7 +130,7 @@ private void markAndResetDatablock(S3ADataBlocks.BlockFactory factory)
throws Exception {
S3AInstrumentation instrumentation =
new S3AInstrumentation(new URI("s3a://example"));
- S3AInstrumentation.OutputStreamStatistics outstats
+ BlockOutputStreamStatistics outstats
= instrumentation.newOutputStreamStatistics(null);
S3ADataBlocks.DataBlock block = factory.create(1, BLOCK_SIZE, outstats);
block.write(dataset, 0, dataset.length);
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AUnbuffer.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AUnbuffer.java
index 2ba3fd7a65cde..e95a6d1f4bd62 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AUnbuffer.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AUnbuffer.java
@@ -71,7 +71,7 @@ public void testUnbuffer() throws IOException {
/**
* Test that calling {@link S3AInputStream#unbuffer()} merges a stream's
- * {@link org.apache.hadoop.fs.s3a.S3AInstrumentation.InputStreamStatistics}
+ * {@code InputStreamStatistics}
* into the {@link S3AFileSystem}'s {@link S3AInstrumentation} instance.
*/
@Test
@@ -108,7 +108,7 @@ public void testUnbufferStreamStatistics() throws IOException {
// Validate that the input stream stats are correct when the file is closed
assertEquals("S3AInputStream statistics were not updated properly", 12,
((S3AInputStream) inputStream.getWrappedStream())
- .getS3AStreamStatistics().bytesRead);
+ .getS3AStreamStatistics().getBytesRead());
}
private boolean isObjectStreamOpen(FSDataInputStream inputStream) {
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java
index e30269298111d..0bddd264a6c15 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java
@@ -39,11 +39,12 @@
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets;
import org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase;
+import org.apache.hadoop.fs.s3a.impl.statistics.CommitterStatistics;
+import org.apache.hadoop.fs.s3a.impl.statistics.EmptyS3AStatisticsContext;
import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
import org.apache.hadoop.util.Progressable;
import static com.google.common.base.Preconditions.checkNotNull;
-import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
/**
* Relays FS calls to the mocked FS, allows for some extra logging with
@@ -83,8 +84,6 @@ public class MockS3AFileSystem extends S3AFileSystem {
* mock FS.
*/
private int logEvents = LOG_NAME;
- private final S3AInstrumentation instrumentation =
- new S3AInstrumentation(FS_URI);
private Configuration conf;
private WriteOperationHelper writeHelper;
@@ -146,12 +145,12 @@ public Path qualify(final Path path) {
public void initialize(URI name, Configuration originalConf)
throws IOException {
conf = originalConf;
- writeHelper = new WriteOperationHelper(this, conf);
+ writeHelper = new WriteOperationHelper(this, conf,
+ new EmptyS3AStatisticsContext());
}
@Override
public void close() {
- cleanupWithLogger(LOG, instrumentation);
}
@Override
@@ -359,12 +358,56 @@ public String toString() {
}
@Override
- public S3AInstrumentation.CommitterStatistics newCommitterStatistics() {
- return instrumentation.newCommitterStatistics();
+ public CommitterStatistics newCommitterStatistics() {
+ return new StubCommitterStatistics();
}
@Override
public void operationRetried(Exception ex) {
/** no-op */
}
+
+ private final class StubCommitterStatistics implements CommitterStatistics {
+
+ @Override
+ public void commitCreated() {
+
+ }
+
+ @Override
+ public void commitUploaded(final long size) {
+
+ }
+
+ @Override
+ public void commitCompleted(final long size) {
+
+ }
+
+ @Override
+ public void commitAborted() {
+
+ }
+
+ @Override
+ public void commitReverted() {
+
+ }
+
+ @Override
+ public void commitFailed() {
+
+ }
+
+ @Override
+ public void taskCompleted(final boolean success) {
+
+ }
+
+ @Override
+ public void jobCompleted(final boolean success) {
+
+ }
+ }
+
}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java
index 4644cf24764ae..deb36c7719f24 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java
@@ -24,10 +24,13 @@
import java.util.ArrayList;
import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.metrics.RequestMetricCollector;
import com.amazonaws.services.s3.AmazonS3;
import com.amazonaws.services.s3.model.MultipartUploadListing;
import com.amazonaws.services.s3.model.Region;
+import org.apache.hadoop.fs.s3a.impl.statistics.StatisticsFromAwsSdk;
+
/**
* An {@link S3ClientFactory} that returns Mockito mocks of the {@link AmazonS3}
* interface suitable for unit testing.
@@ -38,7 +41,8 @@ public class MockS3ClientFactory implements S3ClientFactory {
public AmazonS3 createS3Client(URI name,
final String bucket,
final AWSCredentialsProvider credentialSet,
- final String userAgentSuffix) {
+ final String userAgentSuffix,
+ final StatisticsFromAwsSdk statisticsFromAwsSdks) {
AmazonS3 s3 = mock(AmazonS3.class);
when(s3.doesBucketExist(bucket)).thenReturn(true);
when(s3.doesBucketExistV2(bucket)).thenReturn(true);
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
index aa5979dbf751e..79e82de4537ed 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
@@ -34,8 +34,8 @@
import org.apache.hadoop.fs.s3a.auth.MarshalledCredentialBinding;
import org.apache.hadoop.fs.s3a.auth.MarshalledCredentials;
import org.apache.hadoop.fs.s3a.commit.CommitConstants;
-
import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum;
+import org.apache.hadoop.fs.s3a.impl.statistics.BlockOutputStreamStatistics;
import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
import org.apache.hadoop.fs.s3a.s3guard.MetadataStoreCapabilities;
import org.apache.hadoop.fs.s3native.S3xLoginHelper;
@@ -1162,7 +1162,7 @@ public static void assume(String message, boolean condition) {
* @param out output stream
* @return the (active) stats of the write
*/
- public static S3AInstrumentation.OutputStreamStatistics
+ public static BlockOutputStreamStatistics
getOutputStreamStatistics(FSDataOutputStream out) {
S3ABlockOutputStream blockOutputStream
= (S3ABlockOutputStream) out.getWrappedStream();
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java
index ff176f58da67d..a15b75c63134f 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java
@@ -45,11 +45,10 @@ public void setUp() throws Exception {
S3ADataBlocks.BlockFactory blockFactory =
mock(S3ADataBlocks.BlockFactory.class);
long blockSize = Constants.DEFAULT_MULTIPART_SIZE;
- S3AInstrumentation.OutputStreamStatistics statistics = null;
WriteOperationHelper oHelper = mock(WriteOperationHelper.class);
PutTracker putTracker = mock(PutTracker.class);
stream = spy(new S3ABlockOutputStream(fs, "", executorService,
- progressable, blockSize, blockFactory, statistics, oHelper,
+ progressable, blockSize, blockFactory, null, oHelper,
putTracker));
}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java
index 3d7cdfc08dec4..41ff4c2fbf3fd 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java
@@ -18,8 +18,6 @@
package org.apache.hadoop.fs.s3a;
-import java.util.concurrent.atomic.AtomicLong;
-
import com.amazonaws.AmazonServiceException;
import com.amazonaws.SdkBaseException;
import com.amazonaws.services.s3.Headers;
@@ -36,6 +34,7 @@
import org.apache.hadoop.fs.PathIOException;
import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy;
import org.apache.hadoop.fs.s3a.impl.ChangeTracker;
+import org.apache.hadoop.fs.s3a.impl.statistics.CountingChangeTracker;
import org.apache.hadoop.test.HadoopTestBase;
import static org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy.CHANGE_DETECTED;
@@ -359,7 +358,7 @@ protected void assertTrackerMismatchCount(
final ChangeTracker tracker,
final int expectedCount) {
assertEquals("counter in tracker " + tracker,
- expectedCount, tracker.getVersionMismatches().get());
+ expectedCount, tracker.getVersionMismatches());
}
/**
@@ -386,7 +385,7 @@ protected ChangeTracker newTracker(final ChangeDetectionPolicy.Mode mode,
source,
requireVersion);
ChangeTracker tracker = new ChangeTracker(URI, policy,
- new AtomicLong(0), objectAttributes);
+ new CountingChangeTracker(), objectAttributes);
if (objectAttributes.getVersionId() == null
&& objectAttributes.getETag() == null) {
assertFalse("Tracker should not have applied constraints " + tracker,
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationInFileystem.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationInFileystem.java
index f6c908ea2f7e6..4077257ad764e 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationInFileystem.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationInFileystem.java
@@ -305,6 +305,9 @@ public void testDelegatedFileSystem() throws Throwable {
describe("Delegation tokens can be passed to a new filesystem;"
+ " if role restricted, permissions are tightened.");
S3AFileSystem fs = getFileSystem();
+ // force a probe of the remote FS to make sure its endpoint is valid
+ // (this always hits S3, even when S3Guard is enabled)
+ fs.getObjectMetadata(new Path("/"));
readLandsatMetadata(fs);
URI uri = fs.getUri();
@@ -562,7 +565,7 @@ protected ObjectMetadata readLandsatMetadata(final S3AFileSystem delegatedFS)
factory.setConf(conf);
String host = landsat.getHost();
AmazonS3 s3 = factory.createS3Client(landsat, host, testing,
- "ITestSessionDelegationInFileystem");
+ "ITestSessionDelegationInFileystem", null);
return Invoker.once("HEAD", host,
() -> s3.getObjectMetadata(host, landsat.getPath().substring(1)));
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNeworkBinding.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNetworkBinding.java
similarity index 52%
rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNeworkBinding.java
rename to hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNetworkBinding.java
index eebc3bfdf2fe4..882e12c5665a1 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNeworkBinding.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNetworkBinding.java
@@ -18,19 +18,23 @@
package org.apache.hadoop.fs.s3a.impl;
-import org.assertj.core.api.Assertions;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.client.builder.AwsClientBuilder;
import org.junit.Test;
-import org.apache.hadoop.test.HadoopTestBase;
+import org.apache.hadoop.test.AbstractHadoopTestBase;
+import static org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.createEndpointConfiguration;
import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.fixBucketRegion;
+import static org.assertj.core.api.Assertions.assertThat;
/**
* Unit tests related to the {@link NetworkBinding} class.
*/
-public class TestNeworkBinding extends HadoopTestBase {
+public class TestNetworkBinding extends AbstractHadoopTestBase {
private static final String US_EAST_1 = "us-east-1";
+
private static final String US_WEST_2 = "us-west-2";
@Test
@@ -54,8 +58,45 @@ public void testRegionNullToUSEast() {
}
private static void assertRegionFixup(String region, String expected) {
- Assertions.assertThat(fixBucketRegion(region))
+ assertThat(fixBucketRegion(region))
.describedAs("Fixup of %s", region)
.isEqualTo(expected);
}
+
+ @Test
+ public void testNull() throws Throwable {
+ expectEndpoint("", true, "unused");
+ }
+
+ @Test
+ public void testUSEastEndpoint() throws Throwable {
+ expectEndpoint(US_EAST_1, false, US_EAST_1);
+ }
+
+ @Test
+ public void testUSWestEndpoint() throws Throwable {
+ expectEndpoint(US_WEST_2, false, US_WEST_2);
+ }
+
+ public void expectEndpoint(final String src,
+ final boolean expectNull,
+ final String expectRegion) {
+ AwsClientBuilder.EndpointConfiguration epr =
+ createEndpointConfiguration(src, new ClientConfiguration());
+ String eprStr = epr == null
+ ? "(empty)"
+ : ("(" + epr.getServiceEndpoint() + " " + epr.getSigningRegion());
+ if (expectNull) {
+ assertThat(epr)
+ .describedAs("Endpoint configuration of %s =",
+ src, eprStr)
+ .isNull();
+ } else {
+ assertThat(epr)
+ .describedAs("Endpoint configuration of %s =",
+ src, eprStr)
+ .hasFieldOrPropertyWithValue("serviceEndpoint", src)
+ .hasFieldOrPropertyWithValue("signingRegion", expectRegion);
+ }
+ }
}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java
index 244d2eed324c7..bfac41a48a48e 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java
@@ -48,8 +48,8 @@
import org.apache.hadoop.fs.s3a.Invoker;
import org.apache.hadoop.fs.s3a.S3AFileStatus;
import org.apache.hadoop.fs.s3a.S3AInputPolicy;
-import org.apache.hadoop.fs.s3a.S3AInstrumentation;
import org.apache.hadoop.fs.s3a.S3AStorageStatistics;
+import org.apache.hadoop.fs.s3a.impl.statistics.EmptyS3AStatisticsContext;
import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
import org.apache.hadoop.fs.s3a.s3guard.DirListingMetadata;
import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider;
@@ -216,7 +216,7 @@ private StoreContext createMockStoreContext(boolean multiDelete,
"s3a-transfer-shared"),
Constants.DEFAULT_EXECUTOR_CAPACITY,
new Invoker(RetryPolicies.TRY_ONCE_THEN_FAIL, Invoker.LOG_EVENT),
- new S3AInstrumentation(name),
+ new EmptyS3AStatisticsContext(),
new S3AStorageStatistics(),
S3AInputPolicy.Normal,
ChangeDetectionPolicy.createPolicy(ChangeDetectionPolicy.Mode.None,
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java
index 315d1fe7285be..827e34168f357 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java
@@ -25,8 +25,6 @@
import com.amazonaws.event.ProgressEvent;
import com.amazonaws.event.ProgressEventType;
import com.amazonaws.event.ProgressListener;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.s3a.S3ATestUtils;
import org.junit.FixMethodOrder;
import org.junit.Test;
import org.junit.runners.MethodSorters;
@@ -36,12 +34,14 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.StorageStatistics;
import org.apache.hadoop.fs.contract.ContractTestUtils;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
-import org.apache.hadoop.fs.s3a.S3AInstrumentation;
+import org.apache.hadoop.fs.s3a.S3ATestUtils;
import org.apache.hadoop.fs.s3a.Statistic;
+import org.apache.hadoop.fs.s3a.impl.statistics.BlockOutputStreamStatistics;
import org.apache.hadoop.util.Progressable;
import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
@@ -170,7 +170,7 @@ public void test_010_CreateHugeFile() throws IOException {
Statistic putBytesPending = Statistic.OBJECT_PUT_BYTES_PENDING;
ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
- S3AInstrumentation.OutputStreamStatistics streamStatistics;
+ BlockOutputStreamStatistics streamStatistics;
long blocksPer10MB = blocksPerMB * 10;
ProgressCallback progress = new ProgressCallback(timer);
try (FSDataOutputStream out = fs.create(fileToCreate,
@@ -234,7 +234,7 @@ public void test_010_CreateHugeFile() throws IOException {
"Put file " + fileToCreate + " of size " + filesize);
if (streamStatistics != null) {
assertEquals("actively allocated blocks in " + streamStatistics,
- 0, streamStatistics.blocksActivelyAllocated());
+ 0, streamStatistics.getBlocksActivelyAllocated());
}
}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java
index efd96c4e7387e..e444b463af876 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java
@@ -27,7 +27,7 @@
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.fs.s3a.S3AInputPolicy;
import org.apache.hadoop.fs.s3a.S3AInputStream;
-import org.apache.hadoop.fs.s3a.S3AInstrumentation;
+import org.apache.hadoop.fs.s3a.impl.statistics.S3AInputStreamStatistics;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.compress.CompressionCodec;
@@ -58,7 +58,7 @@ public class ITestS3AInputStreamPerformance extends S3AScaleTestBase {
private Path testData;
private FileStatus testDataStatus;
private FSDataInputStream in;
- private S3AInstrumentation.InputStreamStatistics streamStatistics;
+ private S3AInputStreamStatistics streamStatistics;
public static final int BLOCK_SIZE = 32 * 1024;
public static final int BIG_BLOCK_SIZE = 256 * 1024;
@@ -187,7 +187,7 @@ protected void assertStreamOpenedExactlyOnce() {
*/
private void assertOpenOperationCount(long expected) {
assertEquals("open operations in\n" + in,
- expected, streamStatistics.openOperations);
+ expected, streamStatistics.getOpenOperations());
}
/**
@@ -295,7 +295,7 @@ public void testLazySeekEnabled() throws Throwable {
logTimePerIOP("seek()", timer, blockCount);
logStreamStatistics();
assertOpenOperationCount(0);
- assertEquals("bytes read", 0, streamStatistics.bytesRead);
+ assertEquals("bytes read", 0, streamStatistics.getBytesRead());
}
@Test
@@ -391,8 +391,8 @@ protected void executeSeekReadSequence(long blockSize,
readahead);
logTimePerIOP("seek(pos + " + blockCount+"); read()", timer, blockCount);
LOG.info("Effective bandwidth {} MB/S",
- timer.bandwidthDescription(streamStatistics.bytesRead -
- streamStatistics.bytesSkippedOnSeek));
+ timer.bandwidthDescription(streamStatistics.getBytesRead() -
+ streamStatistics.getBytesSkippedOnSeek()));
logStreamStatistics();
}
@@ -419,7 +419,7 @@ protected void executeSeekReadSequence(long blockSize,
public void testRandomIORandomPolicy() throws Throwable {
executeRandomIO(S3AInputPolicy.Random, (long) RANDOM_IO_SEQUENCE.length);
assertEquals("streams aborted in " + streamStatistics,
- 0, streamStatistics.aborted);
+ 0, streamStatistics.getAborted());
}
@Test
@@ -427,11 +427,12 @@ public void testRandomIONormalPolicy() throws Throwable {
long expectedOpenCount = RANDOM_IO_SEQUENCE.length;
executeRandomIO(S3AInputPolicy.Normal, expectedOpenCount);
assertEquals("streams aborted in " + streamStatistics,
- 1, streamStatistics.aborted);
+ 1, streamStatistics.getAborted());
assertEquals("policy changes in " + streamStatistics,
- 2, streamStatistics.policySetCount);
+ 2, streamStatistics.getPolicySetCount());
assertEquals("input policy in " + streamStatistics,
- S3AInputPolicy.Random.ordinal(), streamStatistics.inputPolicy);
+ S3AInputPolicy.Random.ordinal(),
+ streamStatistics.getInputPolicy());
}
/**
@@ -466,8 +467,8 @@ private ContractTestUtils.NanoTimer executeRandomIO(S3AInputPolicy policy,
assertOpenOperationCount(expectedOpenCount);
logTimePerIOP("byte read", timer, totalBytesRead);
LOG.info("Effective bandwidth {} MB/S",
- timer.bandwidthDescription(streamStatistics.bytesRead -
- streamStatistics.bytesSkippedOnSeek));
+ timer.bandwidthDescription(streamStatistics.getBytesRead() -
+ streamStatistics.getBytesSkippedOnSeek()));
logStreamStatistics();
return timer;
}
@@ -525,7 +526,7 @@ public void testRandomReadOverBuffer() throws Throwable {
+ " current position in stream " + currentPos
+ " in\n" + fs
+ "\n " + in,
- 1, streamStatistics.openOperations);
+ 1, streamStatistics.getOpenOperations());
for (int i = currentPos; i < currentPos + read; i++) {
assertEquals("Wrong value from byte " + i,
sourceData[i], buffer[i]);
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java
index eb80bc579f6c1..1980f2ba3d51d 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java
@@ -26,6 +26,7 @@
import org.apache.hadoop.fs.s3a.S3AInstrumentation;
import org.apache.hadoop.fs.s3a.S3ATestConstants;
import org.apache.hadoop.fs.s3a.Statistic;
+import org.apache.hadoop.fs.s3a.impl.statistics.S3AInputStreamStatistics;
import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
import org.slf4j.Logger;
@@ -162,7 +163,7 @@ protected int getTestTimeoutMillis() {
* @param in wrapper
* @return the statistics for the inner stream
*/
- protected S3AInstrumentation.InputStreamStatistics getInputStreamStatistics(
+ protected S3AInputStreamStatistics getInputStreamStatistics(
FSDataInputStream in) {
return getS3AInputStream(in).getS3AStreamStatistics();
}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3Select.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3Select.java
index 64974db5a466c..378ce4a69e29f 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3Select.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3Select.java
@@ -47,9 +47,9 @@
import org.apache.hadoop.fs.s3a.Constants;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.fs.s3a.S3AInputStream;
-import org.apache.hadoop.fs.s3a.S3AInstrumentation;
import org.apache.hadoop.fs.s3a.S3ATestUtils;
import org.apache.hadoop.fs.s3a.Statistic;
+import org.apache.hadoop.fs.s3a.impl.statistics.S3AInputStreamStatistics;
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapred.JobConf;
@@ -302,8 +302,8 @@ public void testSelectSeek() throws Throwable {
SELECT_EVERYTHING)) {
SelectInputStream sis
= (SelectInputStream) seekStream.getWrappedStream();
- S3AInstrumentation.InputStreamStatistics streamStats
- = sis.getS3AStreamStatistics();
+ S3AInputStreamStatistics streamStats =
+ sis.getS3AStreamStatistics();
// lazy seek doesn't raise a problem here
seekStream.seek(0);
assertEquals("first byte read", fullData[0], seekStream.read());
@@ -344,7 +344,7 @@ public void testSelectSeek() throws Throwable {
assertEquals("byte at seek position",
fullData[(int)seekStream.getPos()], seekStream.read());
assertEquals("Seek bytes skipped in " + streamStats,
- seekRange, streamStats.bytesSkippedOnSeek);
+ seekRange, streamStats.getBytesSkippedOnSeek());
// try an invalid readahead range
intercept(IllegalArgumentException.class,
@@ -588,13 +588,14 @@ public void testCloseWithAbort() throws Throwable {
stream.setReadahead(1L);
assertEquals("Readahead on " + sis, 1, sis.getReadahead());
stream.read();
- S3AInstrumentation.InputStreamStatistics stats
- = sis.getS3AStreamStatistics();
+ S3AInputStreamStatistics stats
+ = (S3AInputStreamStatistics)
+ sis.getS3AStreamStatistics();
assertEquals("Read count in " + sis,
- 1, stats.bytesRead);
+ 1, stats.getBytesRead());
stream.close();
assertEquals("Abort count in " + sis,
- 1, stats.aborted);
+ 1, stats.getAborted());
readOps.assertDiffEquals("Read operations are still considered active",
0);
intercept(PathIOException.class, FSExceptionMessages.STREAM_IS_CLOSED,
@@ -608,12 +609,14 @@ public void testCloseWithNoAbort() throws Throwable {
"SELECT * FROM S3OBJECT s");
stream.setReadahead(0x1000L);
SelectInputStream sis = (SelectInputStream) stream.getWrappedStream();
- S3AInstrumentation.InputStreamStatistics stats
- = sis.getS3AStreamStatistics();
+ S3AInputStreamStatistics stats
+ = (S3AInputStreamStatistics)
+ sis.getS3AStreamStatistics();
stream.close();
- assertEquals("Close count in " + sis, 1, stats.closed);
- assertEquals("Abort count in " + sis, 0, stats.aborted);
- assertTrue("No bytes read in close of " + sis, stats.bytesReadInClose > 0);
+ assertEquals("Close count in " + sis, 1, stats.getClosed());
+ assertEquals("Abort count in " + sis, 0, stats.getAborted());
+ assertTrue("No bytes read in close of " + sis,
+ stats.getBytesReadInClose() > 0);
}
@Test
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectLandsat.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectLandsat.java
index 2099edd248b63..0322ff142db49 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectLandsat.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectLandsat.java
@@ -35,8 +35,8 @@
import org.apache.hadoop.fs.PathIOException;
import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy;
import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy.Source;
+import org.apache.hadoop.fs.s3a.impl.statistics.S3AInputStreamStatistics;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
-import org.apache.hadoop.fs.s3a.S3AInstrumentation;
import org.apache.hadoop.fs.s3a.S3ATestUtils;
import org.apache.hadoop.fs.s3a.Statistic;
import org.apache.hadoop.mapred.JobConf;
@@ -381,7 +381,7 @@ public void testSelectSeekFullLandsat() throws Throwable {
SELECT_EVERYTHING)) {
SelectInputStream sis
= (SelectInputStream) seekStream.getWrappedStream();
- S3AInstrumentation.InputStreamStatistics streamStats
+ S3AInputStreamStatistics streamStats
= sis.getS3AStreamStatistics();
// lazy seek doesn't raise a problem here
seekStream.seek(0);
@@ -410,7 +410,7 @@ public void testSelectSeekFullLandsat() throws Throwable {
assertEquals("byte at seek position",
dataset[(int) seekStream.getPos()], seekStream.read());
assertEquals("Seek bytes skipped in " + streamStats,
- seekRange, streamStats.bytesSkippedOnSeek);
+ seekRange, streamStats.getBytesSkippedOnSeek());
long offset;
long increment = 64 * _1KB;
diff --git a/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties
index 6e20fbcda7efd..8228e3151907e 100644
--- a/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties
+++ b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties
@@ -75,3 +75,6 @@ log4j.logger.org.apache.hadoop.fs.s3a.s3guard.Operations=DEBUG
log4j.logger.org.apache.hadoop.mapreduce.lib.output=DEBUG
log4j.logger.org.apache.hadoop.fs.s3a.S3AStorageStatistics=INFO
+
+# Set to debug if you need to debug S3A endpoint problems.
+#log4j.logger.org.apache.hadoop.fs.s3a.DefaultS3ClientFactory=DEBUG