HADOOP-17271. S3A connector to support IOStatistics. (#2580)
S3A connector to support the IOStatistics API of HADOOP-16830, This is a major rework of the S3A Statistics collection to * Embrace the IOStatistics APIs * Move from direct references of S3AInstrumention statistics collectors to interface/implementation classes in new packages. * Ubiquitous support of IOStatistics, including: S3AFileSystem, input and output streams, RemoteIterator instances provided in list calls. * Adoption of new statistic names from hadoop-common Regarding statistic collection, as well as all existing statistics, the connector now records min/max/mean durations of HTTP GET and HEAD requests, and those of LIST operations. Contributed by Steve Loughran.
This commit is contained in:
parent
9b2956e254
commit
617af28e80
|
@ -74,4 +74,14 @@
|
||||||
<Bug pattern="SF_SWITCH_FALLTHROUGH"/>
|
<Bug pattern="SF_SWITCH_FALLTHROUGH"/>
|
||||||
</Match>
|
</Match>
|
||||||
|
|
||||||
|
<!--
|
||||||
|
Some of the S3A Instrumentation classes increment volatile references from
|
||||||
|
within synchronized contexts; they use volatile to keep the cost
|
||||||
|
of these updates and reading them down.
|
||||||
|
-->
|
||||||
|
<Match>
|
||||||
|
<Class name="org.apache.hadoop.fs.s3a.S3AInstrumentation$InputStreamStatisticsImpl"/>
|
||||||
|
<Bug pattern="VO_VOLATILE_INCREMENT"/>
|
||||||
|
</Match>
|
||||||
|
|
||||||
</FindBugsFilter>
|
</FindBugsFilter>
|
||||||
|
|
|
@ -201,6 +201,8 @@
|
||||||
<exclude>**/ITestMarkerToolRootOperations.java</exclude>
|
<exclude>**/ITestMarkerToolRootOperations.java</exclude>
|
||||||
<!-- operations across the metastore -->
|
<!-- operations across the metastore -->
|
||||||
<exclude>**/ITestS3GuardDDBRootOperations.java</exclude>
|
<exclude>**/ITestS3GuardDDBRootOperations.java</exclude>
|
||||||
|
<!-- leave this until the end for better statistics -->
|
||||||
|
<exclude>**/ITestAggregateIOStatistics.java</exclude>
|
||||||
</excludes>
|
</excludes>
|
||||||
</configuration>
|
</configuration>
|
||||||
</execution>
|
</execution>
|
||||||
|
@ -250,6 +252,8 @@
|
||||||
<!-- operations across the metastore -->
|
<!-- operations across the metastore -->
|
||||||
<include>**/ITestS3AContractRootDir.java</include>
|
<include>**/ITestS3AContractRootDir.java</include>
|
||||||
<include>**/ITestS3GuardDDBRootOperations.java</include>
|
<include>**/ITestS3GuardDDBRootOperations.java</include>
|
||||||
|
<!-- leave this until the end for better statistics -->
|
||||||
|
<include>**/ITestAggregateIOStatistics.java</include>
|
||||||
</includes>
|
</includes>
|
||||||
</configuration>
|
</configuration>
|
||||||
</execution>
|
</execution>
|
||||||
|
|
|
@ -549,6 +549,13 @@ public final class Constants {
|
||||||
@InterfaceStability.Unstable
|
@InterfaceStability.Unstable
|
||||||
public static final String INPUT_FADV_RANDOM = "random";
|
public static final String INPUT_FADV_RANDOM = "random";
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Gauge name for the input policy : {@value}.
|
||||||
|
* This references an enum currently exclusive to the S3A stream.
|
||||||
|
*/
|
||||||
|
public static final String STREAM_READ_GAUGE_INPUT_POLICY =
|
||||||
|
"stream_read_gauge_input_policy";
|
||||||
|
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
@InterfaceStability.Unstable
|
@InterfaceStability.Unstable
|
||||||
public static final String S3_CLIENT_FACTORY_IMPL =
|
public static final String S3_CLIENT_FACTORY_IMPL =
|
||||||
|
|
|
@ -23,16 +23,26 @@ import java.net.URI;
|
||||||
|
|
||||||
import com.amazonaws.ClientConfiguration;
|
import com.amazonaws.ClientConfiguration;
|
||||||
import com.amazonaws.auth.AWSCredentialsProvider;
|
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.AmazonS3;
|
||||||
import com.amazonaws.services.s3.AmazonS3Client;
|
import com.amazonaws.services.s3.AmazonS3Client;
|
||||||
|
import com.amazonaws.services.s3.AmazonS3ClientBuilder;
|
||||||
import com.amazonaws.services.s3.S3ClientOptions;
|
import com.amazonaws.services.s3.S3ClientOptions;
|
||||||
|
import com.amazonaws.services.s3.internal.ServiceUtils;
|
||||||
|
import com.amazonaws.util.AwsHostNameUtils;
|
||||||
|
import com.amazonaws.util.RuntimeHttpUtils;
|
||||||
|
import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import org.apache.commons.lang3.StringUtils;
|
import org.apache.commons.lang3.StringUtils;
|
||||||
import org.apache.hadoop.classification.InterfaceAudience;
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.classification.InterfaceStability;
|
import org.apache.hadoop.classification.InterfaceStability;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.conf.Configured;
|
import org.apache.hadoop.conf.Configured;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.StatisticsFromAwsSdk;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.impl.AwsStatisticsCollector;
|
||||||
|
|
||||||
import static org.apache.hadoop.fs.s3a.Constants.EXPERIMENTAL_AWS_INTERNAL_THROTTLING;
|
import static org.apache.hadoop.fs.s3a.Constants.EXPERIMENTAL_AWS_INTERNAL_THROTTLING;
|
||||||
import static org.apache.hadoop.fs.s3a.Constants.ENDPOINT;
|
import static org.apache.hadoop.fs.s3a.Constants.ENDPOINT;
|
||||||
|
@ -41,7 +51,7 @@ import static org.apache.hadoop.fs.s3a.Constants.PATH_STYLE_ACCESS;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The default {@link S3ClientFactory} implementation.
|
* The default {@link S3ClientFactory} implementation.
|
||||||
* This which calls the AWS SDK to configure and create an
|
* This calls the AWS SDK to configure and create an
|
||||||
* {@link AmazonS3Client} that communicates with the S3 service.
|
* {@link AmazonS3Client} that communicates with the S3 service.
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
|
@ -49,13 +59,30 @@ import static org.apache.hadoop.fs.s3a.Constants.PATH_STYLE_ACCESS;
|
||||||
public class DefaultS3ClientFactory extends Configured
|
public class DefaultS3ClientFactory extends Configured
|
||||||
implements S3ClientFactory {
|
implements S3ClientFactory {
|
||||||
|
|
||||||
protected static final Logger LOG = S3AFileSystem.LOG;
|
private static final String S3_SERVICE_NAME = "s3";
|
||||||
|
private static final String S3_SIGNER = "S3SignerType";
|
||||||
|
private static final String S3_V4_SIGNER = "AWSS3V4SignerType";
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Subclasses refer to this.
|
||||||
|
*/
|
||||||
|
protected static final Logger LOG =
|
||||||
|
LoggerFactory.getLogger(DefaultS3ClientFactory.class);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create the client.
|
||||||
|
* <p>
|
||||||
|
* If the AWS stats are not null then a {@link AwsStatisticsCollector}.
|
||||||
|
* is created to bind to the two.
|
||||||
|
* <i>Important: until this binding works properly across regions,
|
||||||
|
* this should be null.</i>
|
||||||
|
*/
|
||||||
@Override
|
@Override
|
||||||
public AmazonS3 createS3Client(URI name,
|
public AmazonS3 createS3Client(URI name,
|
||||||
final String bucket,
|
final String bucket,
|
||||||
final AWSCredentialsProvider credentials,
|
final AWSCredentialsProvider credentials,
|
||||||
final String userAgentSuffix) throws IOException {
|
final String userAgentSuffix,
|
||||||
|
final StatisticsFromAwsSdk statisticsFromAwsSdk) throws IOException {
|
||||||
Configuration conf = getConf();
|
Configuration conf = getConf();
|
||||||
final ClientConfiguration awsConf = S3AUtils
|
final ClientConfiguration awsConf = S3AUtils
|
||||||
.createAwsConf(conf, bucket, Constants.AWS_SERVICE_IDENTIFIER_S3);
|
.createAwsConf(conf, bucket, Constants.AWS_SERVICE_IDENTIFIER_S3);
|
||||||
|
@ -72,36 +99,124 @@ public class DefaultS3ClientFactory extends Configured
|
||||||
if (!StringUtils.isEmpty(userAgentSuffix)) {
|
if (!StringUtils.isEmpty(userAgentSuffix)) {
|
||||||
awsConf.setUserAgentSuffix(userAgentSuffix);
|
awsConf.setUserAgentSuffix(userAgentSuffix);
|
||||||
}
|
}
|
||||||
return configureAmazonS3Client(
|
// optional metrics
|
||||||
newAmazonS3Client(credentials, awsConf), conf);
|
RequestMetricCollector metrics = statisticsFromAwsSdk != null
|
||||||
|
? new AwsStatisticsCollector(statisticsFromAwsSdk)
|
||||||
|
: null;
|
||||||
|
|
||||||
|
return newAmazonS3Client(
|
||||||
|
credentials,
|
||||||
|
awsConf,
|
||||||
|
metrics,
|
||||||
|
conf.getTrimmed(ENDPOINT, ""),
|
||||||
|
conf.getBoolean(PATH_STYLE_ACCESS, false));
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create an {@link AmazonS3} client.
|
||||||
|
* Override this to provide an extended version of the client
|
||||||
|
* @param credentials credentials to use
|
||||||
|
* @param awsConf AWS configuration
|
||||||
|
* @param metrics metrics collector or null
|
||||||
|
* @param endpoint endpoint string; may be ""
|
||||||
|
* @param pathStyleAccess enable path style access?
|
||||||
|
* @return new AmazonS3 client
|
||||||
|
*/
|
||||||
|
protected AmazonS3 newAmazonS3Client(
|
||||||
|
final AWSCredentialsProvider credentials,
|
||||||
|
final ClientConfiguration awsConf,
|
||||||
|
final RequestMetricCollector metrics,
|
||||||
|
final String endpoint,
|
||||||
|
final boolean pathStyleAccess) {
|
||||||
|
if (metrics != null) {
|
||||||
|
LOG.debug("Building S3 client using the SDK builder API");
|
||||||
|
return buildAmazonS3Client(credentials, awsConf, metrics, endpoint,
|
||||||
|
pathStyleAccess);
|
||||||
|
} else {
|
||||||
|
LOG.debug("Building S3 client using the SDK builder API");
|
||||||
|
return classicAmazonS3Client(credentials, awsConf, endpoint,
|
||||||
|
pathStyleAccess);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Use the (newer) Builder SDK to create a an AWS S3 client.
|
||||||
|
* <p>
|
||||||
|
* This has a more complex endpoint configuration in a
|
||||||
|
* way which does not yet work in this code in a way
|
||||||
|
* which doesn't trigger regressions. So it is only used
|
||||||
|
* when SDK metrics are supplied.
|
||||||
|
* @param credentials credentials to use
|
||||||
|
* @param awsConf AWS configuration
|
||||||
|
* @param metrics metrics collector or null
|
||||||
|
* @param endpoint endpoint string; may be ""
|
||||||
|
* @param pathStyleAccess enable path style access?
|
||||||
|
* @return new AmazonS3 client
|
||||||
|
*/
|
||||||
|
private AmazonS3 buildAmazonS3Client(
|
||||||
|
final AWSCredentialsProvider credentials,
|
||||||
|
final ClientConfiguration awsConf,
|
||||||
|
final RequestMetricCollector metrics,
|
||||||
|
final String endpoint,
|
||||||
|
final boolean pathStyleAccess) {
|
||||||
|
AmazonS3ClientBuilder b = AmazonS3Client.builder();
|
||||||
|
b.withCredentials(credentials);
|
||||||
|
b.withClientConfiguration(awsConf);
|
||||||
|
b.withPathStyleAccessEnabled(pathStyleAccess);
|
||||||
|
if (metrics != null) {
|
||||||
|
b.withMetricsCollector(metrics);
|
||||||
|
}
|
||||||
|
|
||||||
|
// endpoint set up is a PITA
|
||||||
|
// client.setEndpoint("") is no longer available
|
||||||
|
AwsClientBuilder.EndpointConfiguration epr
|
||||||
|
= createEndpointConfiguration(endpoint, awsConf);
|
||||||
|
if (epr != null) {
|
||||||
|
// an endpoint binding was constructed: use it.
|
||||||
|
b.withEndpointConfiguration(epr);
|
||||||
|
}
|
||||||
|
final AmazonS3 client = b.build();
|
||||||
|
return client;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Wrapper around constructor for {@link AmazonS3} client.
|
* Wrapper around constructor for {@link AmazonS3} client.
|
||||||
* Override this to provide an extended version of the client
|
* Override this to provide an extended version of the client.
|
||||||
|
* <p>
|
||||||
|
* This uses a deprecated constructor -it is currently
|
||||||
|
* the only one which works for us.
|
||||||
* @param credentials credentials to use
|
* @param credentials credentials to use
|
||||||
* @param awsConf AWS configuration
|
* @param awsConf AWS configuration
|
||||||
* @return new AmazonS3 client
|
* @param endpoint endpoint string; may be ""
|
||||||
|
* @param pathStyleAccess enable path style access?
|
||||||
|
* @return new AmazonS3 client
|
||||||
*/
|
*/
|
||||||
protected AmazonS3 newAmazonS3Client(
|
@SuppressWarnings("deprecation")
|
||||||
AWSCredentialsProvider credentials, ClientConfiguration awsConf) {
|
private AmazonS3 classicAmazonS3Client(
|
||||||
return new AmazonS3Client(credentials, awsConf);
|
AWSCredentialsProvider credentials,
|
||||||
|
ClientConfiguration awsConf,
|
||||||
|
final String endpoint,
|
||||||
|
final boolean pathStyleAccess) {
|
||||||
|
final AmazonS3 client = new AmazonS3Client(credentials, awsConf);
|
||||||
|
return configureAmazonS3Client(client, endpoint, pathStyleAccess);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Configure S3 client from the Hadoop configuration.
|
* Configure classic S3 client.
|
||||||
*
|
* <p>
|
||||||
* This includes: endpoint, Path Access and possibly other
|
* This includes: endpoint, Path Access and possibly other
|
||||||
* options.
|
* options.
|
||||||
*
|
*
|
||||||
* @param conf Hadoop configuration
|
* @param s3 S3 Client.
|
||||||
|
* @param endPoint s3 endpoint, may be empty
|
||||||
|
* @param pathStyleAccess enable path style access?
|
||||||
* @return S3 client
|
* @return S3 client
|
||||||
* @throws IllegalArgumentException if misconfigured
|
* @throws IllegalArgumentException if misconfigured
|
||||||
*/
|
*/
|
||||||
private static AmazonS3 configureAmazonS3Client(AmazonS3 s3,
|
protected static AmazonS3 configureAmazonS3Client(AmazonS3 s3,
|
||||||
Configuration conf)
|
final String endPoint,
|
||||||
|
final boolean pathStyleAccess)
|
||||||
throws IllegalArgumentException {
|
throws IllegalArgumentException {
|
||||||
String endPoint = conf.getTrimmed(ENDPOINT, "");
|
|
||||||
if (!endPoint.isEmpty()) {
|
if (!endPoint.isEmpty()) {
|
||||||
try {
|
try {
|
||||||
s3.setEndpoint(endPoint);
|
s3.setEndpoint(endPoint);
|
||||||
|
@ -111,7 +226,7 @@ public class DefaultS3ClientFactory extends Configured
|
||||||
throw new IllegalArgumentException(msg, e);
|
throw new IllegalArgumentException(msg, e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return applyS3ClientOptions(s3, conf);
|
return applyS3ClientOptions(s3, pathStyleAccess);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -119,23 +234,23 @@ public class DefaultS3ClientFactory extends Configured
|
||||||
* the Hadoop configuration.
|
* the Hadoop configuration.
|
||||||
* This is different from the general AWS configuration creation as
|
* This is different from the general AWS configuration creation as
|
||||||
* it is unique to S3 connections.
|
* it is unique to S3 connections.
|
||||||
*
|
* <p>
|
||||||
* The {@link Constants#PATH_STYLE_ACCESS} option enables path-style access
|
* The {@link Constants#PATH_STYLE_ACCESS} option enables path-style access
|
||||||
* to S3 buckets if configured. By default, the
|
* to S3 buckets if configured. By default, the
|
||||||
* behavior is to use virtual hosted-style access with URIs of the form
|
* behavior is to use virtual hosted-style access with URIs of the form
|
||||||
* {@code http://bucketname.s3.amazonaws.com}
|
* {@code http://bucketname.s3.amazonaws.com}
|
||||||
|
* <p>
|
||||||
* Enabling path-style access and a
|
* Enabling path-style access and a
|
||||||
* region-specific endpoint switches the behavior to use URIs of the form
|
* region-specific endpoint switches the behavior to use URIs of the form
|
||||||
* {@code http://s3-eu-west-1.amazonaws.com/bucketname}.
|
* {@code http://s3-eu-west-1.amazonaws.com/bucketname}.
|
||||||
* It is common to use this when connecting to private S3 servers, as it
|
* It is common to use this when connecting to private S3 servers, as it
|
||||||
* avoids the need to play with DNS entries.
|
* avoids the need to play with DNS entries.
|
||||||
* @param s3 S3 client
|
* @param s3 S3 client
|
||||||
* @param conf Hadoop configuration
|
* @param pathStyleAccess enable path style access?
|
||||||
* @return the S3 client
|
* @return the S3 client
|
||||||
*/
|
*/
|
||||||
private static AmazonS3 applyS3ClientOptions(AmazonS3 s3,
|
protected static AmazonS3 applyS3ClientOptions(AmazonS3 s3,
|
||||||
Configuration conf) {
|
final boolean pathStyleAccess) {
|
||||||
final boolean pathStyleAccess = conf.getBoolean(PATH_STYLE_ACCESS, false);
|
|
||||||
if (pathStyleAccess) {
|
if (pathStyleAccess) {
|
||||||
LOG.debug("Enabling path style access!");
|
LOG.debug("Enabling path style access!");
|
||||||
s3.setS3ClientOptions(S3ClientOptions.builder()
|
s3.setS3ClientOptions(S3ClientOptions.builder()
|
||||||
|
@ -144,4 +259,54 @@ public class DefaultS3ClientFactory extends Configured
|
||||||
}
|
}
|
||||||
return s3;
|
return s3;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Given an endpoint string, return an endpoint config, or null, if none
|
||||||
|
* is needed.
|
||||||
|
* <p>
|
||||||
|
* This is a pretty painful piece of code. It is trying to replicate
|
||||||
|
* what AwsClient.setEndpoint() does, because you can't
|
||||||
|
* call that setter on an AwsClient constructed via
|
||||||
|
* the builder, and you can't pass a metrics collector
|
||||||
|
* down except through the builder.
|
||||||
|
* <p>
|
||||||
|
* Note also that AWS signing is a mystery which nobody fully
|
||||||
|
* understands, especially given all problems surface in a
|
||||||
|
* "400 bad request" response, which, like all security systems,
|
||||||
|
* provides minimal diagnostics out of fear of leaking
|
||||||
|
* secrets.
|
||||||
|
*
|
||||||
|
* @param endpoint possibly null endpoint.
|
||||||
|
* @param awsConf config to build the URI from.
|
||||||
|
* @return a configuration for the S3 client builder.
|
||||||
|
*/
|
||||||
|
@VisibleForTesting
|
||||||
|
public static AwsClientBuilder.EndpointConfiguration
|
||||||
|
createEndpointConfiguration(
|
||||||
|
final String endpoint, final ClientConfiguration awsConf) {
|
||||||
|
LOG.debug("Creating endpoint configuration for {}", endpoint);
|
||||||
|
if (endpoint == null || endpoint.isEmpty()) {
|
||||||
|
// the default endpoint...we should be using null at this point.
|
||||||
|
LOG.debug("Using default endpoint -no need to generate a configuration");
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
final URI epr = RuntimeHttpUtils.toUri(endpoint, awsConf);
|
||||||
|
LOG.debug("Endpoint URI = {}", epr);
|
||||||
|
|
||||||
|
String region;
|
||||||
|
if (!ServiceUtils.isS3USStandardEndpoint(endpoint)) {
|
||||||
|
LOG.debug("Endpoint {} is not the default; parsing", epr);
|
||||||
|
region = AwsHostNameUtils.parseRegion(
|
||||||
|
epr.getHost(),
|
||||||
|
S3_SERVICE_NAME);
|
||||||
|
} else {
|
||||||
|
// US-east, set region == null.
|
||||||
|
LOG.debug("Endpoint {} is the standard one; declare region as null", epr);
|
||||||
|
region = null;
|
||||||
|
}
|
||||||
|
LOG.debug("Region for endpoint {}, URI {} is determined as {}",
|
||||||
|
endpoint, epr, region);
|
||||||
|
return new AwsClientBuilder.EndpointConfiguration(endpoint, region);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,6 +20,7 @@ package org.apache.hadoop.fs.s3a;
|
||||||
|
|
||||||
import com.amazonaws.ClientConfiguration;
|
import com.amazonaws.ClientConfiguration;
|
||||||
import com.amazonaws.auth.AWSCredentialsProvider;
|
import com.amazonaws.auth.AWSCredentialsProvider;
|
||||||
|
import com.amazonaws.metrics.RequestMetricCollector;
|
||||||
import com.amazonaws.services.s3.AmazonS3;
|
import com.amazonaws.services.s3.AmazonS3;
|
||||||
|
|
||||||
import org.apache.hadoop.classification.InterfaceAudience;
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
|
@ -40,12 +41,21 @@ public class InconsistentS3ClientFactory extends DefaultS3ClientFactory {
|
||||||
* Logs a warning that this is being done.
|
* Logs a warning that this is being done.
|
||||||
* @param credentials credentials to use
|
* @param credentials credentials to use
|
||||||
* @param awsConf AWS configuration
|
* @param awsConf AWS configuration
|
||||||
|
* @param metrics metric collector
|
||||||
|
* @param endpoint AWS endpoint
|
||||||
|
* @param pathStyleAccess should path style access be supported?
|
||||||
* @return an inconsistent client.
|
* @return an inconsistent client.
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
protected AmazonS3 newAmazonS3Client(AWSCredentialsProvider credentials,
|
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! **");
|
LOG.warn("** FAILURE INJECTION ENABLED. Do not run in production! **");
|
||||||
return new InconsistentAmazonS3Client(credentials, awsConf, getConf());
|
InconsistentAmazonS3Client s3
|
||||||
|
= new InconsistentAmazonS3Client(credentials, awsConf, getConf());
|
||||||
|
configureAmazonS3Client(s3, endpoint, pathStyleAccess);
|
||||||
|
return s3;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -30,17 +30,19 @@ import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import org.apache.commons.lang3.StringUtils;
|
import org.apache.commons.lang3.StringUtils;
|
||||||
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.io.retry.RetryPolicy;
|
import org.apache.hadoop.io.retry.RetryPolicy;
|
||||||
import org.apache.hadoop.util.DurationInfo;
|
import org.apache.hadoop.util.DurationInfo;
|
||||||
|
import org.apache.hadoop.util.functional.CallableRaisingIOE;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Class to provide lambda expression invocation of AWS operations.
|
* Class to provide lambda expression invocation of AWS operations.
|
||||||
*
|
*
|
||||||
* The core retry logic is in
|
* The core retry logic is in
|
||||||
* {@link #retryUntranslated(String, boolean, Retried, Operation)};
|
* {@link #retryUntranslated(String, boolean, Retried, CallableRaisingIOE)};
|
||||||
* the other {@code retry() and retryUntranslated()} calls are wrappers.
|
* the other {@code retry() and retryUntranslated()} calls are wrappers.
|
||||||
*
|
*
|
||||||
* The static {@link #once(String, String, Operation)} and
|
* The static {@link #once(String, String, CallableRaisingIOE)} and
|
||||||
* {@link #once(String, String, VoidOperation)} calls take an operation and
|
* {@link #once(String, String, VoidOperation)} calls take an operation and
|
||||||
* return it with AWS exceptions translated to IOEs of some form.
|
* return it with AWS exceptions translated to IOEs of some form.
|
||||||
*
|
*
|
||||||
|
@ -56,11 +58,13 @@ import org.apache.hadoop.util.DurationInfo;
|
||||||
* These callbacks can be used for reporting and incrementing statistics.
|
* These callbacks can be used for reporting and incrementing statistics.
|
||||||
*
|
*
|
||||||
* The static {@link #quietly(String, String, VoidOperation)} and
|
* The static {@link #quietly(String, String, VoidOperation)} and
|
||||||
* {@link #quietlyEval(String, String, Operation)} calls exist to take any
|
* {@link #quietlyEval(String, String, CallableRaisingIOE)} calls exist to
|
||||||
* operation and quietly catch and log at debug. The return value of
|
* take any operation and quietly catch and log at debug.
|
||||||
* {@link #quietlyEval(String, String, Operation)} is a java 8 optional,
|
* The return value of {@link #quietlyEval(String, String, CallableRaisingIOE)}
|
||||||
|
* is a java 8 optional,
|
||||||
* which can then be used in java8-expressions.
|
* which can then be used in java8-expressions.
|
||||||
*/
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
public class Invoker {
|
public class Invoker {
|
||||||
private static final Logger LOG = LoggerFactory.getLogger(Invoker.class);
|
private static final Logger LOG = LoggerFactory.getLogger(Invoker.class);
|
||||||
|
|
||||||
|
@ -104,10 +108,11 @@ public class Invoker {
|
||||||
* @throws IOException any IOE raised, or translated exception
|
* @throws IOException any IOE raised, or translated exception
|
||||||
*/
|
*/
|
||||||
@Retries.OnceTranslated
|
@Retries.OnceTranslated
|
||||||
public static <T> T once(String action, String path, Operation<T> operation)
|
public static <T> T once(String action, String path,
|
||||||
|
CallableRaisingIOE<T> operation)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
try (DurationInfo ignored = new DurationInfo(LOG, false, "%s", action)) {
|
try (DurationInfo ignored = new DurationInfo(LOG, false, "%s", action)) {
|
||||||
return operation.execute();
|
return operation.apply();
|
||||||
} catch (AmazonClientException e) {
|
} catch (AmazonClientException e) {
|
||||||
throw S3AUtils.translateException(action, path, e);
|
throw S3AUtils.translateException(action, path, e);
|
||||||
}
|
}
|
||||||
|
@ -143,7 +148,7 @@ public class Invoker {
|
||||||
Logger log,
|
Logger log,
|
||||||
String action,
|
String action,
|
||||||
String path,
|
String path,
|
||||||
Operation<T> operation) {
|
CallableRaisingIOE<T> operation) {
|
||||||
try {
|
try {
|
||||||
once(action, path, operation);
|
once(action, path, operation);
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
|
@ -280,7 +285,7 @@ public class Invoker {
|
||||||
public <T> T retry(String action,
|
public <T> T retry(String action,
|
||||||
@Nullable String path,
|
@Nullable String path,
|
||||||
boolean idempotent,
|
boolean idempotent,
|
||||||
Operation<T> operation)
|
CallableRaisingIOE<T> operation)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
|
|
||||||
return retry(action, path, idempotent, retryCallback, operation);
|
return retry(action, path, idempotent, retryCallback, operation);
|
||||||
|
@ -288,7 +293,7 @@ public class Invoker {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Execute a function with retry processing.
|
* Execute a function with retry processing.
|
||||||
* Uses {@link #once(String, String, Operation)} as the inner
|
* Uses {@link #once(String, String, CallableRaisingIOE)} as the inner
|
||||||
* invocation mechanism before retry logic is performed.
|
* invocation mechanism before retry logic is performed.
|
||||||
* @param <T> type of return value
|
* @param <T> type of return value
|
||||||
* @param action action to execute (used in error messages)
|
* @param action action to execute (used in error messages)
|
||||||
|
@ -306,7 +311,7 @@ public class Invoker {
|
||||||
@Nullable String path,
|
@Nullable String path,
|
||||||
boolean idempotent,
|
boolean idempotent,
|
||||||
Retried retrying,
|
Retried retrying,
|
||||||
Operation<T> operation)
|
CallableRaisingIOE<T> operation)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
return retryUntranslated(
|
return retryUntranslated(
|
||||||
toDescription(action, path),
|
toDescription(action, path),
|
||||||
|
@ -317,7 +322,7 @@ public class Invoker {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Execute a function with retry processing when doRetry=true, else just once.
|
* Execute a function with retry processing when doRetry=true, else just once.
|
||||||
* Uses {@link #once(String, String, Operation)} as the inner
|
* Uses {@link #once(String, String, CallableRaisingIOE)} as the inner
|
||||||
* invocation mechanism before retry logic is performed.
|
* invocation mechanism before retry logic is performed.
|
||||||
* @param <T> type of return value
|
* @param <T> type of return value
|
||||||
* @param doRetry true if retries should be performed
|
* @param doRetry true if retries should be performed
|
||||||
|
@ -337,7 +342,7 @@ public class Invoker {
|
||||||
@Nullable String path,
|
@Nullable String path,
|
||||||
boolean idempotent,
|
boolean idempotent,
|
||||||
Retried retrying,
|
Retried retrying,
|
||||||
Operation<T> operation)
|
CallableRaisingIOE<T> operation)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
if (doRetry) {
|
if (doRetry) {
|
||||||
return retryUntranslated(
|
return retryUntranslated(
|
||||||
|
@ -366,7 +371,7 @@ public class Invoker {
|
||||||
public <T> T retryUntranslated(
|
public <T> T retryUntranslated(
|
||||||
String text,
|
String text,
|
||||||
boolean idempotent,
|
boolean idempotent,
|
||||||
Operation<T> operation) throws IOException {
|
CallableRaisingIOE<T> operation) throws IOException {
|
||||||
return retryUntranslated(text, idempotent,
|
return retryUntranslated(text, idempotent,
|
||||||
retryCallback, operation);
|
retryCallback, operation);
|
||||||
}
|
}
|
||||||
|
@ -391,7 +396,7 @@ public class Invoker {
|
||||||
String text,
|
String text,
|
||||||
boolean idempotent,
|
boolean idempotent,
|
||||||
Retried retrying,
|
Retried retrying,
|
||||||
Operation<T> operation) throws IOException {
|
CallableRaisingIOE<T> operation) throws IOException {
|
||||||
|
|
||||||
Preconditions.checkArgument(retrying != null, "null retrying argument");
|
Preconditions.checkArgument(retrying != null, "null retrying argument");
|
||||||
int retryCount = 0;
|
int retryCount = 0;
|
||||||
|
@ -404,7 +409,7 @@ public class Invoker {
|
||||||
LOG.debug("retry #{}", retryCount);
|
LOG.debug("retry #{}", retryCount);
|
||||||
}
|
}
|
||||||
// execute the operation, returning if successful
|
// execute the operation, returning if successful
|
||||||
return operation.execute();
|
return operation.apply();
|
||||||
} catch (IOException | SdkBaseException e) {
|
} catch (IOException | SdkBaseException e) {
|
||||||
caught = e;
|
caught = e;
|
||||||
}
|
}
|
||||||
|
@ -490,7 +495,7 @@ public class Invoker {
|
||||||
*/
|
*/
|
||||||
public static <T> Optional<T> quietlyEval(String action,
|
public static <T> Optional<T> quietlyEval(String action,
|
||||||
String path,
|
String path,
|
||||||
Operation<T> operation) {
|
CallableRaisingIOE<T> operation) {
|
||||||
try {
|
try {
|
||||||
return Optional.of(once(action, path, operation));
|
return Optional.of(once(action, path, operation));
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
|
@ -510,15 +515,6 @@ public class Invoker {
|
||||||
(StringUtils.isNotEmpty(path) ? (" on " + path) : "");
|
(StringUtils.isNotEmpty(path) ? (" on " + path) : "");
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Arbitrary operation throwing an IOException.
|
|
||||||
* @param <T> return type
|
|
||||||
*/
|
|
||||||
@FunctionalInterface
|
|
||||||
public interface Operation<T> {
|
|
||||||
T execute() throws IOException;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Void operation which may raise an IOException.
|
* Void operation which may raise an IOException.
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -38,8 +38,11 @@ import org.apache.hadoop.fs.s3a.s3guard.DirListingMetadata;
|
||||||
import org.apache.hadoop.fs.s3a.s3guard.MetadataStoreListFilesIterator;
|
import org.apache.hadoop.fs.s3a.s3guard.MetadataStoreListFilesIterator;
|
||||||
import org.apache.hadoop.fs.s3a.s3guard.PathMetadata;
|
import org.apache.hadoop.fs.s3a.s3guard.PathMetadata;
|
||||||
import org.apache.hadoop.fs.s3a.s3guard.S3Guard;
|
import org.apache.hadoop.fs.s3a.s3guard.S3Guard;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatistics;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
|
||||||
|
import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
|
||||||
|
import org.apache.hadoop.util.functional.RemoteIterators;
|
||||||
|
|
||||||
import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
|
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
|
|
||||||
import java.io.FileNotFoundException;
|
import java.io.FileNotFoundException;
|
||||||
|
@ -48,7 +51,6 @@ import java.time.Instant;
|
||||||
import java.time.OffsetDateTime;
|
import java.time.OffsetDateTime;
|
||||||
import java.time.ZoneOffset;
|
import java.time.ZoneOffset;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collections;
|
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
@ -57,6 +59,7 @@ import java.util.Map;
|
||||||
import java.util.NoSuchElementException;
|
import java.util.NoSuchElementException;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.CompletableFuture;
|
import java.util.concurrent.CompletableFuture;
|
||||||
|
import java.util.StringJoiner;
|
||||||
|
|
||||||
import static org.apache.hadoop.fs.impl.FutureIOSupport.awaitFuture;
|
import static org.apache.hadoop.fs.impl.FutureIOSupport.awaitFuture;
|
||||||
import static org.apache.hadoop.fs.s3a.Constants.S3N_FOLDER_SUFFIX;
|
import static org.apache.hadoop.fs.s3a.Constants.S3N_FOLDER_SUFFIX;
|
||||||
|
@ -67,6 +70,12 @@ import static org.apache.hadoop.fs.s3a.S3AUtils.objectRepresentsDirectory;
|
||||||
import static org.apache.hadoop.fs.s3a.S3AUtils.stringify;
|
import static org.apache.hadoop.fs.s3a.S3AUtils.stringify;
|
||||||
import static org.apache.hadoop.fs.s3a.S3AUtils.translateException;
|
import static org.apache.hadoop.fs.s3a.S3AUtils.translateException;
|
||||||
import static org.apache.hadoop.fs.s3a.auth.RoleModel.pathToKey;
|
import static org.apache.hadoop.fs.s3a.auth.RoleModel.pathToKey;
|
||||||
|
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_CONTINUE_LIST_REQUEST;
|
||||||
|
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_LIST_REQUEST;
|
||||||
|
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore;
|
||||||
|
import static org.apache.hadoop.util.functional.RemoteIterators.filteringRemoteIterator;
|
||||||
|
import static org.apache.hadoop.util.functional.RemoteIterators.remoteIteratorFromArray;
|
||||||
|
import static org.apache.hadoop.util.functional.RemoteIterators.remoteIteratorFromSingleton;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Place for the S3A listing classes; keeps all the small classes under control.
|
* Place for the S3A listing classes; keeps all the small classes under control.
|
||||||
|
@ -96,11 +105,14 @@ public class Listing extends AbstractStoreOperation {
|
||||||
* @param acceptor the file status acceptor
|
* @param acceptor the file status acceptor
|
||||||
* @return the file status iterator
|
* @return the file status iterator
|
||||||
*/
|
*/
|
||||||
ProvidedFileStatusIterator createProvidedFileStatusIterator(
|
RemoteIterator<S3AFileStatus> createProvidedFileStatusIterator(
|
||||||
S3AFileStatus[] fileStatuses,
|
S3AFileStatus[] fileStatuses,
|
||||||
PathFilter filter,
|
PathFilter filter,
|
||||||
FileStatusAcceptor acceptor) {
|
FileStatusAcceptor acceptor) {
|
||||||
return new ProvidedFileStatusIterator(fileStatuses, filter, acceptor);
|
return filteringRemoteIterator(
|
||||||
|
remoteIteratorFromArray(fileStatuses),
|
||||||
|
status ->
|
||||||
|
filter.accept(status.getPath()) && acceptor.accept(status));
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -109,11 +121,11 @@ public class Listing extends AbstractStoreOperation {
|
||||||
* @return the file status iterator.
|
* @return the file status iterator.
|
||||||
*/
|
*/
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
public static ProvidedFileStatusIterator toProvidedFileStatusIterator(
|
public static RemoteIterator<S3AFileStatus> toProvidedFileStatusIterator(
|
||||||
S3AFileStatus[] fileStatuses) {
|
S3AFileStatus[] fileStatuses) {
|
||||||
return new ProvidedFileStatusIterator(fileStatuses,
|
return filteringRemoteIterator(
|
||||||
ACCEPT_ALL,
|
remoteIteratorFromArray(fileStatuses),
|
||||||
Listing.ACCEPT_ALL_BUT_S3N);
|
Listing.ACCEPT_ALL_BUT_S3N::accept);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -185,9 +197,11 @@ public class Listing extends AbstractStoreOperation {
|
||||||
* @return a new remote iterator
|
* @return a new remote iterator
|
||||||
*/
|
*/
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
public LocatedFileStatusIterator createLocatedFileStatusIterator(
|
public RemoteIterator<S3ALocatedFileStatus> createLocatedFileStatusIterator(
|
||||||
RemoteIterator<S3AFileStatus> statusIterator) {
|
RemoteIterator<S3AFileStatus> statusIterator) {
|
||||||
return new LocatedFileStatusIterator(statusIterator);
|
return RemoteIterators.mappingRemoteIterator(
|
||||||
|
statusIterator,
|
||||||
|
listingOperationCallbacks::toLocatedFileStatus);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -199,11 +213,28 @@ public class Listing extends AbstractStoreOperation {
|
||||||
* @return a new remote iterator.
|
* @return a new remote iterator.
|
||||||
*/
|
*/
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
TombstoneReconcilingIterator createTombstoneReconcilingIterator(
|
RemoteIterator<S3ALocatedFileStatus> createTombstoneReconcilingIterator(
|
||||||
RemoteIterator<S3ALocatedFileStatus> iterator, Set<Path> tombstones) {
|
RemoteIterator<S3ALocatedFileStatus> iterator,
|
||||||
return new TombstoneReconcilingIterator(iterator, tombstones);
|
@Nullable Set<Path> tombstones) {
|
||||||
|
if (tombstones == null || tombstones.isEmpty()) {
|
||||||
|
// no need to filter.
|
||||||
|
return iterator;
|
||||||
|
} else {
|
||||||
|
return filteringRemoteIterator(
|
||||||
|
iterator,
|
||||||
|
candidate -> !tombstones.contains(candidate.getPath()));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a remote iterator from a single status entry.
|
||||||
|
* @param status status
|
||||||
|
* @return iterator.
|
||||||
|
*/
|
||||||
|
public RemoteIterator<S3ALocatedFileStatus> createSingleStatusIterator(
|
||||||
|
S3ALocatedFileStatus status) {
|
||||||
|
return remoteIteratorFromSingleton(status);
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* List files under a path assuming the path to be a directory.
|
* List files under a path assuming the path to be a directory.
|
||||||
|
@ -369,7 +400,7 @@ public class Listing extends AbstractStoreOperation {
|
||||||
allowAuthoritative);
|
allowAuthoritative);
|
||||||
// In auth mode return directly with auth flag.
|
// In auth mode return directly with auth flag.
|
||||||
if (allowAuthoritative && dirMeta != null && dirMeta.isAuthoritative()) {
|
if (allowAuthoritative && dirMeta != null && dirMeta.isAuthoritative()) {
|
||||||
ProvidedFileStatusIterator mfsItr = createProvidedFileStatusIterator(
|
RemoteIterator<S3AFileStatus> mfsItr = createProvidedFileStatusIterator(
|
||||||
S3Guard.dirMetaToStatuses(dirMeta),
|
S3Guard.dirMetaToStatuses(dirMeta),
|
||||||
ACCEPT_ALL,
|
ACCEPT_ALL,
|
||||||
Listing.ACCEPT_ALL_BUT_S3N);
|
Listing.ACCEPT_ALL_BUT_S3N);
|
||||||
|
@ -429,105 +460,6 @@ public class Listing extends AbstractStoreOperation {
|
||||||
boolean accept(FileStatus status);
|
boolean accept(FileStatus status);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* A remote iterator which only iterates over a single `LocatedFileStatus`
|
|
||||||
* value.
|
|
||||||
*
|
|
||||||
* If the status value is null, the iterator declares that it has no data.
|
|
||||||
* This iterator is used to handle {@link S3AFileSystem#listStatus(Path)}
|
|
||||||
* calls where the path handed in refers to a file, not a directory:
|
|
||||||
* this is the iterator returned.
|
|
||||||
*/
|
|
||||||
static final class SingleStatusRemoteIterator
|
|
||||||
implements RemoteIterator<S3ALocatedFileStatus> {
|
|
||||||
|
|
||||||
/**
|
|
||||||
* The status to return; set to null after the first iteration.
|
|
||||||
*/
|
|
||||||
private S3ALocatedFileStatus status;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Constructor.
|
|
||||||
* @param status status value: may be null, in which case
|
|
||||||
* the iterator is empty.
|
|
||||||
*/
|
|
||||||
SingleStatusRemoteIterator(S3ALocatedFileStatus status) {
|
|
||||||
this.status = status;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* {@inheritDoc}
|
|
||||||
* @return true if there is a file status to return: this is always false
|
|
||||||
* for the second iteration, and may be false for the first.
|
|
||||||
* @throws IOException never
|
|
||||||
*/
|
|
||||||
@Override
|
|
||||||
public boolean hasNext() throws IOException {
|
|
||||||
return status != null;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* {@inheritDoc}
|
|
||||||
* @return the non-null status element passed in when the instance was
|
|
||||||
* constructed, if it ha not already been retrieved.
|
|
||||||
* @throws IOException never
|
|
||||||
* @throws NoSuchElementException if this is the second call, or it is
|
|
||||||
* the first call and a null {@link LocatedFileStatus} entry was passed
|
|
||||||
* to the constructor.
|
|
||||||
*/
|
|
||||||
@Override
|
|
||||||
public S3ALocatedFileStatus next() throws IOException {
|
|
||||||
if (hasNext()) {
|
|
||||||
S3ALocatedFileStatus s = this.status;
|
|
||||||
status = null;
|
|
||||||
return s;
|
|
||||||
} else {
|
|
||||||
throw new NoSuchElementException();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* This wraps up a provided non-null list of file status as a remote iterator.
|
|
||||||
*
|
|
||||||
* It firstly filters the provided list and later {@link #next} call will get
|
|
||||||
* from the filtered list. This suffers from scalability issues if the
|
|
||||||
* provided list is too large.
|
|
||||||
*
|
|
||||||
* There is no remote data to fetch.
|
|
||||||
*/
|
|
||||||
static class ProvidedFileStatusIterator
|
|
||||||
implements RemoteIterator<S3AFileStatus> {
|
|
||||||
private final ArrayList<S3AFileStatus> filteredStatusList;
|
|
||||||
private int index = 0;
|
|
||||||
|
|
||||||
ProvidedFileStatusIterator(S3AFileStatus[] fileStatuses, PathFilter filter,
|
|
||||||
FileStatusAcceptor acceptor) {
|
|
||||||
Preconditions.checkArgument(fileStatuses != null, "Null status list!");
|
|
||||||
|
|
||||||
filteredStatusList = new ArrayList<>(fileStatuses.length);
|
|
||||||
for (S3AFileStatus status : fileStatuses) {
|
|
||||||
if (filter.accept(status.getPath()) && acceptor.accept(status)) {
|
|
||||||
filteredStatusList.add(status);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
filteredStatusList.trimToSize();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean hasNext() throws IOException {
|
|
||||||
return index < filteredStatusList.size();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public S3AFileStatus next() throws IOException {
|
|
||||||
if (!hasNext()) {
|
|
||||||
throw new NoSuchElementException();
|
|
||||||
}
|
|
||||||
return filteredStatusList.get(index++);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Wraps up object listing into a remote iterator which will ask for more
|
* Wraps up object listing into a remote iterator which will ask for more
|
||||||
* listing data if needed.
|
* listing data if needed.
|
||||||
|
@ -555,7 +487,7 @@ public class Listing extends AbstractStoreOperation {
|
||||||
* Thread safety: None.
|
* Thread safety: None.
|
||||||
*/
|
*/
|
||||||
class FileStatusListingIterator
|
class FileStatusListingIterator
|
||||||
implements RemoteIterator<S3AFileStatus> {
|
implements RemoteIterator<S3AFileStatus>, IOStatisticsSource {
|
||||||
|
|
||||||
/** Source of objects. */
|
/** Source of objects. */
|
||||||
private final ObjectListingIterator source;
|
private final ObjectListingIterator source;
|
||||||
|
@ -758,6 +690,23 @@ public class Listing extends AbstractStoreOperation {
|
||||||
public int getBatchSize() {
|
public int getBatchSize() {
|
||||||
return batchSize;
|
return batchSize;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return any IOStatistics provided by the underlying stream.
|
||||||
|
* @return IO stats from the inner stream.
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public IOStatistics getIOStatistics() {
|
||||||
|
return source.getIOStatistics();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return new StringJoiner(", ",
|
||||||
|
FileStatusListingIterator.class.getSimpleName() + "[", "]")
|
||||||
|
.add(source.toString())
|
||||||
|
.toString();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -780,7 +729,8 @@ public class Listing extends AbstractStoreOperation {
|
||||||
*
|
*
|
||||||
* Thread safety: none.
|
* Thread safety: none.
|
||||||
*/
|
*/
|
||||||
class ObjectListingIterator implements RemoteIterator<S3ListResult> {
|
class ObjectListingIterator implements RemoteIterator<S3ListResult>,
|
||||||
|
IOStatisticsSource {
|
||||||
|
|
||||||
/** The path listed. */
|
/** The path listed. */
|
||||||
private final Path listPath;
|
private final Path listPath;
|
||||||
|
@ -805,6 +755,8 @@ public class Listing extends AbstractStoreOperation {
|
||||||
*/
|
*/
|
||||||
private int maxKeys;
|
private int maxKeys;
|
||||||
|
|
||||||
|
private final IOStatisticsStore iostats;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Future to store current batch listing result.
|
* Future to store current batch listing result.
|
||||||
*/
|
*/
|
||||||
|
@ -828,10 +780,14 @@ public class Listing extends AbstractStoreOperation {
|
||||||
S3ListRequest request) throws IOException {
|
S3ListRequest request) throws IOException {
|
||||||
this.listPath = listPath;
|
this.listPath = listPath;
|
||||||
this.maxKeys = listingOperationCallbacks.getMaxKeys();
|
this.maxKeys = listingOperationCallbacks.getMaxKeys();
|
||||||
this.s3ListResultFuture = listingOperationCallbacks
|
|
||||||
.listObjectsAsync(request);
|
|
||||||
this.request = request;
|
this.request = request;
|
||||||
this.objectsPrev = null;
|
this.objectsPrev = null;
|
||||||
|
this.iostats = iostatisticsStore()
|
||||||
|
.withDurationTracking(OBJECT_LIST_REQUEST)
|
||||||
|
.withDurationTracking(OBJECT_CONTINUE_LIST_REQUEST)
|
||||||
|
.build();
|
||||||
|
this.s3ListResultFuture = listingOperationCallbacks
|
||||||
|
.listObjectsAsync(request, iostats);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -895,7 +851,7 @@ public class Listing extends AbstractStoreOperation {
|
||||||
LOG.debug("[{}], Requesting next {} objects under {}",
|
LOG.debug("[{}], Requesting next {} objects under {}",
|
||||||
listingCount, maxKeys, listPath);
|
listingCount, maxKeys, listPath);
|
||||||
s3ListResultFuture = listingOperationCallbacks
|
s3ListResultFuture = listingOperationCallbacks
|
||||||
.continueListObjectsAsync(request, objects);
|
.continueListObjectsAsync(request, objects, iostats);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -903,7 +859,13 @@ public class Listing extends AbstractStoreOperation {
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return "Object listing iterator against " + listPath
|
return "Object listing iterator against " + listPath
|
||||||
+ "; listing count "+ listingCount
|
+ "; listing count "+ listingCount
|
||||||
+ "; isTruncated=" + objects.isTruncated();
|
+ "; isTruncated=" + objects.isTruncated()
|
||||||
|
+ "; " + iostats;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public IOStatistics getIOStatistics() {
|
||||||
|
return iostats;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -966,89 +928,6 @@ public class Listing extends AbstractStoreOperation {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Take a remote iterator over a set of {@link FileStatus} instances and
|
|
||||||
* return a remote iterator of {@link LocatedFileStatus} instances.
|
|
||||||
*/
|
|
||||||
class LocatedFileStatusIterator
|
|
||||||
implements RemoteIterator<S3ALocatedFileStatus> {
|
|
||||||
private final RemoteIterator<S3AFileStatus> statusIterator;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Constructor.
|
|
||||||
* @param statusIterator an iterator over the remote status entries
|
|
||||||
*/
|
|
||||||
LocatedFileStatusIterator(RemoteIterator<S3AFileStatus> statusIterator) {
|
|
||||||
this.statusIterator = statusIterator;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean hasNext() throws IOException {
|
|
||||||
return statusIterator.hasNext();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public S3ALocatedFileStatus next() throws IOException {
|
|
||||||
return listingOperationCallbacks
|
|
||||||
.toLocatedFileStatus(statusIterator.next());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Wraps another iterator and filters out files that appear in the provided
|
|
||||||
* set of tombstones. Will read ahead in the iterator when necessary to
|
|
||||||
* ensure that emptiness is detected early enough if only deleted objects
|
|
||||||
* remain in the source iterator.
|
|
||||||
*/
|
|
||||||
static class TombstoneReconcilingIterator implements
|
|
||||||
RemoteIterator<S3ALocatedFileStatus> {
|
|
||||||
private S3ALocatedFileStatus next = null;
|
|
||||||
private final RemoteIterator<S3ALocatedFileStatus> iterator;
|
|
||||||
private final Set<Path> tombstones;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @param iterator Source iterator to filter
|
|
||||||
* @param tombstones set of tombstone markers to filter out of results
|
|
||||||
*/
|
|
||||||
TombstoneReconcilingIterator(RemoteIterator<S3ALocatedFileStatus>
|
|
||||||
iterator, Set<Path> tombstones) {
|
|
||||||
this.iterator = iterator;
|
|
||||||
if (tombstones != null) {
|
|
||||||
this.tombstones = tombstones;
|
|
||||||
} else {
|
|
||||||
this.tombstones = Collections.emptySet();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private boolean fetch() throws IOException {
|
|
||||||
while (next == null && iterator.hasNext()) {
|
|
||||||
S3ALocatedFileStatus candidate = iterator.next();
|
|
||||||
if (!tombstones.contains(candidate.getPath())) {
|
|
||||||
next = candidate;
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
public boolean hasNext() throws IOException {
|
|
||||||
if (next != null) {
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
return fetch();
|
|
||||||
}
|
|
||||||
|
|
||||||
public S3ALocatedFileStatus next() throws IOException {
|
|
||||||
if (hasNext()) {
|
|
||||||
S3ALocatedFileStatus result = next;
|
|
||||||
next = null;
|
|
||||||
fetch();
|
|
||||||
return result;
|
|
||||||
}
|
|
||||||
throw new NoSuchElementException();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Accept all entries except those which map to S3N pseudo directory markers.
|
* Accept all entries except those which map to S3N pseudo directory markers.
|
||||||
*/
|
*/
|
||||||
|
@ -1117,4 +996,9 @@ public class Listing extends AbstractStoreOperation {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static RemoteIterator<LocatedFileStatus> toLocatedFileStatusIterator(
|
||||||
|
RemoteIterator<? extends LocatedFileStatus> iterator) {
|
||||||
|
return (RemoteIterator < LocatedFileStatus >) iterator;
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,6 +20,8 @@ package org.apache.hadoop.fs.s3a;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.OutputStream;
|
import java.io.OutputStream;
|
||||||
|
import java.time.Duration;
|
||||||
|
import java.time.Instant;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Locale;
|
import java.util.Locale;
|
||||||
|
@ -49,10 +51,15 @@ import org.apache.hadoop.fs.PathIOException;
|
||||||
import org.apache.hadoop.fs.StreamCapabilities;
|
import org.apache.hadoop.fs.StreamCapabilities;
|
||||||
import org.apache.hadoop.fs.s3a.commit.CommitConstants;
|
import org.apache.hadoop.fs.s3a.commit.CommitConstants;
|
||||||
import org.apache.hadoop.fs.s3a.commit.PutTracker;
|
import org.apache.hadoop.fs.s3a.commit.PutTracker;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatistics;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatisticsLogging;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
|
||||||
import org.apache.hadoop.util.Progressable;
|
import org.apache.hadoop.util.Progressable;
|
||||||
|
|
||||||
import static org.apache.hadoop.fs.s3a.S3AUtils.*;
|
import static org.apache.hadoop.fs.s3a.S3AUtils.*;
|
||||||
import static org.apache.hadoop.fs.s3a.Statistic.*;
|
import static org.apache.hadoop.fs.s3a.Statistic.*;
|
||||||
|
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.emptyStatistics;
|
||||||
import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
|
import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -67,7 +74,7 @@ import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
@InterfaceStability.Unstable
|
@InterfaceStability.Unstable
|
||||||
class S3ABlockOutputStream extends OutputStream implements
|
class S3ABlockOutputStream extends OutputStream implements
|
||||||
StreamCapabilities {
|
StreamCapabilities, IOStatisticsSource {
|
||||||
|
|
||||||
private static final Logger LOG =
|
private static final Logger LOG =
|
||||||
LoggerFactory.getLogger(S3ABlockOutputStream.class);
|
LoggerFactory.getLogger(S3ABlockOutputStream.class);
|
||||||
|
@ -81,6 +88,9 @@ class S3ABlockOutputStream extends OutputStream implements
|
||||||
/** Size of all blocks. */
|
/** Size of all blocks. */
|
||||||
private final int blockSize;
|
private final int blockSize;
|
||||||
|
|
||||||
|
/** IO Statistics. */
|
||||||
|
private final IOStatistics iostatistics;
|
||||||
|
|
||||||
/** Total bytes for uploads submitted so far. */
|
/** Total bytes for uploads submitted so far. */
|
||||||
private long bytesSubmitted;
|
private long bytesSubmitted;
|
||||||
|
|
||||||
|
@ -109,7 +119,7 @@ class S3ABlockOutputStream extends OutputStream implements
|
||||||
private long blockCount = 0;
|
private long blockCount = 0;
|
||||||
|
|
||||||
/** Statistics to build up. */
|
/** Statistics to build up. */
|
||||||
private final S3AInstrumentation.OutputStreamStatistics statistics;
|
private final BlockOutputStreamStatistics statistics;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Write operation helper; encapsulation of the filesystem operations.
|
* Write operation helper; encapsulation of the filesystem operations.
|
||||||
|
@ -146,7 +156,7 @@ class S3ABlockOutputStream extends OutputStream implements
|
||||||
Progressable progress,
|
Progressable progress,
|
||||||
long blockSize,
|
long blockSize,
|
||||||
S3ADataBlocks.BlockFactory blockFactory,
|
S3ADataBlocks.BlockFactory blockFactory,
|
||||||
S3AInstrumentation.OutputStreamStatistics statistics,
|
BlockOutputStreamStatistics statistics,
|
||||||
WriteOperationHelper writeOperationHelper,
|
WriteOperationHelper writeOperationHelper,
|
||||||
PutTracker putTracker)
|
PutTracker putTracker)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
|
@ -155,6 +165,10 @@ class S3ABlockOutputStream extends OutputStream implements
|
||||||
this.blockFactory = blockFactory;
|
this.blockFactory = blockFactory;
|
||||||
this.blockSize = (int) blockSize;
|
this.blockSize = (int) blockSize;
|
||||||
this.statistics = statistics;
|
this.statistics = statistics;
|
||||||
|
// test instantiations may not provide statistics;
|
||||||
|
this.iostatistics = statistics != null
|
||||||
|
? statistics.getIOStatistics()
|
||||||
|
: emptyStatistics();
|
||||||
this.writeOperationHelper = writeOperationHelper;
|
this.writeOperationHelper = writeOperationHelper;
|
||||||
this.putTracker = putTracker;
|
this.putTracker = putTracker;
|
||||||
Preconditions.checkArgument(blockSize >= Constants.MULTIPART_MIN_SIZE,
|
Preconditions.checkArgument(blockSize >= Constants.MULTIPART_MIN_SIZE,
|
||||||
|
@ -282,6 +296,7 @@ class S3ABlockOutputStream extends OutputStream implements
|
||||||
if (len == 0) {
|
if (len == 0) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
statistics.writeBytes(len);
|
||||||
S3ADataBlocks.DataBlock block = createBlockIfNeeded();
|
S3ADataBlocks.DataBlock block = createBlockIfNeeded();
|
||||||
int written = block.write(source, offset, len);
|
int written = block.write(source, offset, len);
|
||||||
int remainingCapacity = block.remainingCapacity();
|
int remainingCapacity = block.remainingCapacity();
|
||||||
|
@ -382,7 +397,8 @@ class S3ABlockOutputStream extends OutputStream implements
|
||||||
// then complete the operation
|
// then complete the operation
|
||||||
if (putTracker.aboutToComplete(multiPartUpload.getUploadId(),
|
if (putTracker.aboutToComplete(multiPartUpload.getUploadId(),
|
||||||
partETags,
|
partETags,
|
||||||
bytes)) {
|
bytes,
|
||||||
|
iostatistics)) {
|
||||||
multiPartUpload.complete(partETags);
|
multiPartUpload.complete(partETags);
|
||||||
} else {
|
} else {
|
||||||
LOG.info("File {} will be visible when the job is committed", key);
|
LOG.info("File {} will be visible when the job is committed", key);
|
||||||
|
@ -432,10 +448,9 @@ class S3ABlockOutputStream extends OutputStream implements
|
||||||
writeOperationHelper.createPutObjectRequest(key, uploadData.getFile())
|
writeOperationHelper.createPutObjectRequest(key, uploadData.getFile())
|
||||||
: writeOperationHelper.createPutObjectRequest(key,
|
: writeOperationHelper.createPutObjectRequest(key,
|
||||||
uploadData.getUploadStream(), size);
|
uploadData.getUploadStream(), size);
|
||||||
long transferQueueTime = now();
|
|
||||||
BlockUploadProgress callback =
|
BlockUploadProgress callback =
|
||||||
new BlockUploadProgress(
|
new BlockUploadProgress(
|
||||||
block, progressListener, transferQueueTime);
|
block, progressListener, now());
|
||||||
putObjectRequest.setGeneralProgressListener(callback);
|
putObjectRequest.setGeneralProgressListener(callback);
|
||||||
statistics.blockUploadQueued(size);
|
statistics.blockUploadQueued(size);
|
||||||
ListenableFuture<PutObjectResult> putObjectResult =
|
ListenableFuture<PutObjectResult> putObjectResult =
|
||||||
|
@ -473,6 +488,8 @@ class S3ABlockOutputStream extends OutputStream implements
|
||||||
if (block != null) {
|
if (block != null) {
|
||||||
sb.append(", activeBlock=").append(block);
|
sb.append(", activeBlock=").append(block);
|
||||||
}
|
}
|
||||||
|
sb.append(" Statistics=")
|
||||||
|
.append(IOStatisticsLogging.ioStatisticsSourceToString(this));
|
||||||
sb.append('}');
|
sb.append('}');
|
||||||
return sb.toString();
|
return sb.toString();
|
||||||
}
|
}
|
||||||
|
@ -485,15 +502,15 @@ class S3ABlockOutputStream extends OutputStream implements
|
||||||
* Current time in milliseconds.
|
* Current time in milliseconds.
|
||||||
* @return time
|
* @return time
|
||||||
*/
|
*/
|
||||||
private long now() {
|
private Instant now() {
|
||||||
return System.currentTimeMillis();
|
return Instant.now();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get the statistics for this stream.
|
* Get the statistics for this stream.
|
||||||
* @return stream statistics
|
* @return stream statistics
|
||||||
*/
|
*/
|
||||||
S3AInstrumentation.OutputStreamStatistics getStatistics() {
|
BlockOutputStreamStatistics getStatistics() {
|
||||||
return statistics;
|
return statistics;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -520,11 +537,20 @@ class S3ABlockOutputStream extends OutputStream implements
|
||||||
case StreamCapabilities.HSYNC:
|
case StreamCapabilities.HSYNC:
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
|
// yes, we do statistics.
|
||||||
|
case StreamCapabilities.IOSTATISTICS:
|
||||||
|
return true;
|
||||||
|
|
||||||
default:
|
default:
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public IOStatistics getIOStatistics() {
|
||||||
|
return iostatistics;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Multiple partition upload.
|
* Multiple partition upload.
|
||||||
*/
|
*/
|
||||||
|
@ -636,10 +662,9 @@ class S3ABlockOutputStream extends OutputStream implements
|
||||||
noteUploadFailure(e);
|
noteUploadFailure(e);
|
||||||
throw e;
|
throw e;
|
||||||
}
|
}
|
||||||
long transferQueueTime = now();
|
|
||||||
BlockUploadProgress callback =
|
BlockUploadProgress callback =
|
||||||
new BlockUploadProgress(
|
new BlockUploadProgress(
|
||||||
block, progressListener, transferQueueTime);
|
block, progressListener, now());
|
||||||
request.setGeneralProgressListener(callback);
|
request.setGeneralProgressListener(callback);
|
||||||
statistics.blockUploadQueued(block.dataSize());
|
statistics.blockUploadQueued(block.dataSize());
|
||||||
ListenableFuture<PartETag> partETagFuture =
|
ListenableFuture<PartETag> partETagFuture =
|
||||||
|
@ -754,8 +779,8 @@ class S3ABlockOutputStream extends OutputStream implements
|
||||||
private final class BlockUploadProgress implements ProgressListener {
|
private final class BlockUploadProgress implements ProgressListener {
|
||||||
private final S3ADataBlocks.DataBlock block;
|
private final S3ADataBlocks.DataBlock block;
|
||||||
private final ProgressListener nextListener;
|
private final ProgressListener nextListener;
|
||||||
private final long transferQueueTime;
|
private final Instant transferQueueTime;
|
||||||
private long transferStartTime;
|
private Instant transferStartTime;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Track the progress of a single block upload.
|
* Track the progress of a single block upload.
|
||||||
|
@ -766,7 +791,7 @@ class S3ABlockOutputStream extends OutputStream implements
|
||||||
*/
|
*/
|
||||||
private BlockUploadProgress(S3ADataBlocks.DataBlock block,
|
private BlockUploadProgress(S3ADataBlocks.DataBlock block,
|
||||||
ProgressListener nextListener,
|
ProgressListener nextListener,
|
||||||
long transferQueueTime) {
|
Instant transferQueueTime) {
|
||||||
this.block = block;
|
this.block = block;
|
||||||
this.transferQueueTime = transferQueueTime;
|
this.transferQueueTime = transferQueueTime;
|
||||||
this.nextListener = nextListener;
|
this.nextListener = nextListener;
|
||||||
|
@ -787,17 +812,22 @@ class S3ABlockOutputStream extends OutputStream implements
|
||||||
|
|
||||||
case TRANSFER_PART_STARTED_EVENT:
|
case TRANSFER_PART_STARTED_EVENT:
|
||||||
transferStartTime = now();
|
transferStartTime = now();
|
||||||
statistics.blockUploadStarted(transferStartTime - transferQueueTime,
|
statistics.blockUploadStarted(
|
||||||
|
Duration.between(transferQueueTime, transferStartTime),
|
||||||
size);
|
size);
|
||||||
incrementWriteOperations();
|
incrementWriteOperations();
|
||||||
break;
|
break;
|
||||||
|
|
||||||
case TRANSFER_PART_COMPLETED_EVENT:
|
case TRANSFER_PART_COMPLETED_EVENT:
|
||||||
statistics.blockUploadCompleted(now() - transferStartTime, size);
|
statistics.blockUploadCompleted(
|
||||||
|
Duration.between(transferStartTime, now()),
|
||||||
|
size);
|
||||||
break;
|
break;
|
||||||
|
|
||||||
case TRANSFER_PART_FAILED_EVENT:
|
case TRANSFER_PART_FAILED_EVENT:
|
||||||
statistics.blockUploadFailed(now() - transferStartTime, size);
|
statistics.blockUploadFailed(
|
||||||
|
Duration.between(transferStartTime, now()),
|
||||||
|
size);
|
||||||
LOG.warn("Transfer failure of block {}", block);
|
LOG.warn("Transfer failure of block {}", block);
|
||||||
break;
|
break;
|
||||||
|
|
||||||
|
|
|
@ -37,6 +37,7 @@ import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import org.apache.hadoop.fs.FSExceptionMessages;
|
import org.apache.hadoop.fs.FSExceptionMessages;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics;
|
||||||
import org.apache.hadoop.util.DirectBufferPool;
|
import org.apache.hadoop.util.DirectBufferPool;
|
||||||
|
|
||||||
import static org.apache.hadoop.fs.s3a.S3ADataBlocks.DataBlock.DestState.*;
|
import static org.apache.hadoop.fs.s3a.S3ADataBlocks.DataBlock.DestState.*;
|
||||||
|
@ -180,7 +181,7 @@ final class S3ADataBlocks {
|
||||||
* @return a new block.
|
* @return a new block.
|
||||||
*/
|
*/
|
||||||
abstract DataBlock create(long index, int limit,
|
abstract DataBlock create(long index, int limit,
|
||||||
S3AInstrumentation.OutputStreamStatistics statistics)
|
BlockOutputStreamStatistics statistics)
|
||||||
throws IOException;
|
throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -210,10 +211,10 @@ final class S3ADataBlocks {
|
||||||
|
|
||||||
private volatile DestState state = Writing;
|
private volatile DestState state = Writing;
|
||||||
protected final long index;
|
protected final long index;
|
||||||
protected final S3AInstrumentation.OutputStreamStatistics statistics;
|
private final BlockOutputStreamStatistics statistics;
|
||||||
|
|
||||||
protected DataBlock(long index,
|
protected DataBlock(long index,
|
||||||
S3AInstrumentation.OutputStreamStatistics statistics) {
|
BlockOutputStreamStatistics statistics) {
|
||||||
this.index = index;
|
this.index = index;
|
||||||
this.statistics = statistics;
|
this.statistics = statistics;
|
||||||
}
|
}
|
||||||
|
@ -372,6 +373,10 @@ final class S3ADataBlocks {
|
||||||
statistics.blockReleased();
|
statistics.blockReleased();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
protected BlockOutputStreamStatistics getStatistics() {
|
||||||
|
return statistics;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// ====================================================================
|
// ====================================================================
|
||||||
|
@ -387,7 +392,7 @@ final class S3ADataBlocks {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
DataBlock create(long index, int limit,
|
DataBlock create(long index, int limit,
|
||||||
S3AInstrumentation.OutputStreamStatistics statistics)
|
BlockOutputStreamStatistics statistics)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
return new ByteArrayBlock(0, limit, statistics);
|
return new ByteArrayBlock(0, limit, statistics);
|
||||||
}
|
}
|
||||||
|
@ -432,7 +437,7 @@ final class S3ADataBlocks {
|
||||||
|
|
||||||
ByteArrayBlock(long index,
|
ByteArrayBlock(long index,
|
||||||
int limit,
|
int limit,
|
||||||
S3AInstrumentation.OutputStreamStatistics statistics) {
|
BlockOutputStreamStatistics statistics) {
|
||||||
super(index, statistics);
|
super(index, statistics);
|
||||||
this.limit = limit;
|
this.limit = limit;
|
||||||
buffer = new S3AByteArrayOutputStream(limit);
|
buffer = new S3AByteArrayOutputStream(limit);
|
||||||
|
@ -510,7 +515,7 @@ final class S3ADataBlocks {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
ByteBufferBlock create(long index, int limit,
|
ByteBufferBlock create(long index, int limit,
|
||||||
S3AInstrumentation.OutputStreamStatistics statistics)
|
BlockOutputStreamStatistics statistics)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
return new ByteBufferBlock(index, limit, statistics);
|
return new ByteBufferBlock(index, limit, statistics);
|
||||||
}
|
}
|
||||||
|
@ -560,7 +565,7 @@ final class S3ADataBlocks {
|
||||||
*/
|
*/
|
||||||
ByteBufferBlock(long index,
|
ByteBufferBlock(long index,
|
||||||
int bufferSize,
|
int bufferSize,
|
||||||
S3AInstrumentation.OutputStreamStatistics statistics) {
|
BlockOutputStreamStatistics statistics) {
|
||||||
super(index, statistics);
|
super(index, statistics);
|
||||||
this.bufferSize = bufferSize;
|
this.bufferSize = bufferSize;
|
||||||
blockBuffer = requestBuffer(bufferSize);
|
blockBuffer = requestBuffer(bufferSize);
|
||||||
|
@ -805,7 +810,7 @@ final class S3ADataBlocks {
|
||||||
@Override
|
@Override
|
||||||
DataBlock create(long index,
|
DataBlock create(long index,
|
||||||
int limit,
|
int limit,
|
||||||
S3AInstrumentation.OutputStreamStatistics statistics)
|
BlockOutputStreamStatistics statistics)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
File destFile = getOwner()
|
File destFile = getOwner()
|
||||||
.createTmpFileForWrite(String.format("s3ablock-%04d-", index),
|
.createTmpFileForWrite(String.format("s3ablock-%04d-", index),
|
||||||
|
@ -829,7 +834,7 @@ final class S3ADataBlocks {
|
||||||
DiskBlock(File bufferFile,
|
DiskBlock(File bufferFile,
|
||||||
int limit,
|
int limit,
|
||||||
long index,
|
long index,
|
||||||
S3AInstrumentation.OutputStreamStatistics statistics)
|
BlockOutputStreamStatistics statistics)
|
||||||
throws FileNotFoundException {
|
throws FileNotFoundException {
|
||||||
super(index, statistics);
|
super(index, statistics);
|
||||||
this.limit = limit;
|
this.limit = limit;
|
||||||
|
|
|
@ -116,11 +116,14 @@ import org.apache.hadoop.fs.s3a.impl.S3AMultipartUploaderBuilder;
|
||||||
import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum;
|
import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum;
|
||||||
import org.apache.hadoop.fs.s3a.impl.StoreContext;
|
import org.apache.hadoop.fs.s3a.impl.StoreContext;
|
||||||
import org.apache.hadoop.fs.s3a.impl.StoreContextBuilder;
|
import org.apache.hadoop.fs.s3a.impl.StoreContextBuilder;
|
||||||
import org.apache.hadoop.fs.s3a.impl.statistics.S3AMultipartUploaderStatisticsImpl;
|
|
||||||
import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
|
import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
|
||||||
import org.apache.hadoop.fs.s3a.select.InternalSelectConstants;
|
import org.apache.hadoop.fs.s3a.select.InternalSelectConstants;
|
||||||
import org.apache.hadoop.fs.s3a.tools.MarkerToolOperations;
|
import org.apache.hadoop.fs.s3a.tools.MarkerToolOperations;
|
||||||
import org.apache.hadoop.fs.s3a.tools.MarkerToolOperationsImpl;
|
import org.apache.hadoop.fs.s3a.tools.MarkerToolOperationsImpl;
|
||||||
|
import org.apache.hadoop.fs.statistics.DurationTracker;
|
||||||
|
import org.apache.hadoop.fs.statistics.DurationTrackerFactory;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatistics;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
|
||||||
import org.apache.hadoop.io.IOUtils;
|
import org.apache.hadoop.io.IOUtils;
|
||||||
import org.apache.hadoop.io.Text;
|
import org.apache.hadoop.io.Text;
|
||||||
import org.apache.hadoop.security.token.DelegationTokenIssuer;
|
import org.apache.hadoop.security.token.DelegationTokenIssuer;
|
||||||
|
@ -158,6 +161,11 @@ import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
|
||||||
import org.apache.hadoop.fs.s3a.s3guard.PathMetadata;
|
import org.apache.hadoop.fs.s3a.s3guard.PathMetadata;
|
||||||
import org.apache.hadoop.fs.s3a.s3guard.S3Guard;
|
import org.apache.hadoop.fs.s3a.s3guard.S3Guard;
|
||||||
import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider;
|
import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.StatisticsFromAwsSdk;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.impl.BondedS3AStatisticsContext;
|
||||||
import org.apache.hadoop.fs.s3native.S3xLoginHelper;
|
import org.apache.hadoop.fs.s3native.S3xLoginHelper;
|
||||||
import org.apache.hadoop.io.retry.RetryPolicies;
|
import org.apache.hadoop.io.retry.RetryPolicies;
|
||||||
import org.apache.hadoop.fs.store.EtagChecksum;
|
import org.apache.hadoop.fs.store.EtagChecksum;
|
||||||
|
@ -169,10 +177,12 @@ import org.apache.hadoop.util.ReflectionUtils;
|
||||||
import org.apache.hadoop.util.SemaphoredDelegatingExecutor;
|
import org.apache.hadoop.util.SemaphoredDelegatingExecutor;
|
||||||
import org.apache.hadoop.util.concurrent.HadoopExecutors;
|
import org.apache.hadoop.util.concurrent.HadoopExecutors;
|
||||||
|
|
||||||
|
import static java.util.Objects.requireNonNull;
|
||||||
import static org.apache.hadoop.fs.impl.AbstractFSBuilderImpl.rejectUnknownMandatoryKeys;
|
import static org.apache.hadoop.fs.impl.AbstractFSBuilderImpl.rejectUnknownMandatoryKeys;
|
||||||
import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs;
|
import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs;
|
||||||
import static org.apache.hadoop.fs.s3a.Constants.*;
|
import static org.apache.hadoop.fs.s3a.Constants.*;
|
||||||
import static org.apache.hadoop.fs.s3a.Invoker.*;
|
import static org.apache.hadoop.fs.s3a.Invoker.*;
|
||||||
|
import static org.apache.hadoop.fs.s3a.Listing.toLocatedFileStatusIterator;
|
||||||
import static org.apache.hadoop.fs.s3a.S3AUtils.*;
|
import static org.apache.hadoop.fs.s3a.S3AUtils.*;
|
||||||
import static org.apache.hadoop.fs.s3a.Statistic.*;
|
import static org.apache.hadoop.fs.s3a.Statistic.*;
|
||||||
import static org.apache.commons.lang3.StringUtils.isNotEmpty;
|
import static org.apache.commons.lang3.StringUtils.isNotEmpty;
|
||||||
|
@ -184,12 +194,20 @@ import static org.apache.hadoop.fs.s3a.commit.CommitConstants.FS_S3A_COMMITTER_A
|
||||||
import static org.apache.hadoop.fs.s3a.commit.CommitConstants.FS_S3A_COMMITTER_STAGING_ABORT_PENDING_UPLOADS;
|
import static org.apache.hadoop.fs.s3a.commit.CommitConstants.FS_S3A_COMMITTER_STAGING_ABORT_PENDING_UPLOADS;
|
||||||
import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit;
|
import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit;
|
||||||
import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletionIgnoringExceptions;
|
import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletionIgnoringExceptions;
|
||||||
|
import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isObjectNotFound;
|
||||||
import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isUnknownBucket;
|
import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isUnknownBucket;
|
||||||
|
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.AWS_SDK_METRICS_ENABLED;
|
||||||
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_404;
|
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_404;
|
||||||
import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.fixBucketRegion;
|
import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.fixBucketRegion;
|
||||||
import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.logDnsLookup;
|
import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.logDnsLookup;
|
||||||
import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.dirMetaToStatuses;
|
import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.dirMetaToStatuses;
|
||||||
|
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_CONTINUE_LIST_REQUEST;
|
||||||
|
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_LIST_REQUEST;
|
||||||
|
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.pairedTrackerFactory;
|
||||||
|
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfInvocation;
|
||||||
|
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfOperation;
|
||||||
import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
|
import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
|
||||||
|
import static org.apache.hadoop.util.functional.RemoteIterators.typeCastingRemoteIterator;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The core S3A Filesystem implementation.
|
* The core S3A Filesystem implementation.
|
||||||
|
@ -207,7 +225,7 @@ import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
@InterfaceStability.Evolving
|
@InterfaceStability.Evolving
|
||||||
public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
||||||
AWSPolicyProvider, DelegationTokenProvider {
|
AWSPolicyProvider, DelegationTokenProvider, IOStatisticsSource {
|
||||||
/**
|
/**
|
||||||
* Default blocksize as used in blocksize and FS status queries.
|
* Default blocksize as used in blocksize and FS status queries.
|
||||||
*/
|
*/
|
||||||
|
@ -259,9 +277,13 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
||||||
* is no encryption.
|
* is no encryption.
|
||||||
*/
|
*/
|
||||||
private EncryptionSecrets encryptionSecrets = new EncryptionSecrets();
|
private EncryptionSecrets encryptionSecrets = new EncryptionSecrets();
|
||||||
|
/** The core instrumentation. */
|
||||||
private S3AInstrumentation instrumentation;
|
private S3AInstrumentation instrumentation;
|
||||||
private final S3AStorageStatistics storageStatistics =
|
/** Accessors to statistics for this FS. */
|
||||||
createStorageStatistics();
|
private S3AStatisticsContext statisticsContext;
|
||||||
|
/** Storage Statistics Bonded to the instrumentation. */
|
||||||
|
private S3AStorageStatistics storageStatistics;
|
||||||
|
|
||||||
private long readAhead;
|
private long readAhead;
|
||||||
private S3AInputPolicy inputPolicy;
|
private S3AInputPolicy inputPolicy;
|
||||||
private ChangeDetectionPolicy changeDetectionPolicy;
|
private ChangeDetectionPolicy changeDetectionPolicy;
|
||||||
|
@ -368,6 +390,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
||||||
|
|
||||||
invoker = new Invoker(new S3ARetryPolicy(getConf()), onRetry);
|
invoker = new Invoker(new S3ARetryPolicy(getConf()), onRetry);
|
||||||
instrumentation = new S3AInstrumentation(uri);
|
instrumentation = new S3AInstrumentation(uri);
|
||||||
|
initializeStatisticsBinding();
|
||||||
|
|
||||||
// Username is the current user at the time the FS was instantiated.
|
// Username is the current user at the time the FS was instantiated.
|
||||||
owner = UserGroupInformation.getCurrentUser();
|
owner = UserGroupInformation.getCurrentUser();
|
||||||
|
@ -377,7 +400,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
||||||
|
|
||||||
s3guardInvoker = new Invoker(new S3GuardExistsRetryPolicy(getConf()),
|
s3guardInvoker = new Invoker(new S3GuardExistsRetryPolicy(getConf()),
|
||||||
onRetry);
|
onRetry);
|
||||||
writeHelper = new WriteOperationHelper(this, getConf());
|
writeHelper = new WriteOperationHelper(this, getConf(),
|
||||||
|
statisticsContext);
|
||||||
|
|
||||||
failOnMetadataWriteError = conf.getBoolean(FAIL_ON_METADATA_WRITE_ERROR,
|
failOnMetadataWriteError = conf.getBoolean(FAIL_ON_METADATA_WRITE_ERROR,
|
||||||
FAIL_ON_METADATA_WRITE_ERROR_DEFAULT);
|
FAIL_ON_METADATA_WRITE_ERROR_DEFAULT);
|
||||||
|
@ -531,6 +555,33 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Initialize the statistics binding.
|
||||||
|
* This is done by creating an {@code IntegratedS3AStatisticsContext}
|
||||||
|
* with callbacks to get the FS's instrumentation and FileSystem.statistics
|
||||||
|
* field; the latter may change after {@link #initialize(URI, Configuration)},
|
||||||
|
* so needs to be dynamically adapted.
|
||||||
|
* Protected so that (mock) subclasses can replace it with a
|
||||||
|
* different statistics binding, if desired.
|
||||||
|
*/
|
||||||
|
protected void initializeStatisticsBinding() {
|
||||||
|
storageStatistics = createStorageStatistics(
|
||||||
|
requireNonNull(getIOStatistics()));
|
||||||
|
statisticsContext = new BondedS3AStatisticsContext(
|
||||||
|
new BondedS3AStatisticsContext.S3AFSStatisticsSource() {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public S3AInstrumentation getInstrumentation() {
|
||||||
|
return S3AFileSystem.this.getInstrumentation();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Statistics getInstanceStatistics() {
|
||||||
|
return S3AFileSystem.this.statistics;
|
||||||
|
}
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Initialize the thread pool.
|
* Initialize the thread pool.
|
||||||
* This must be re-invoked after replacing the S3Client during test
|
* This must be re-invoked after replacing the S3Client during test
|
||||||
|
@ -565,13 +616,15 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create the storage statistics or bind to an existing one.
|
* Create the storage statistics or bind to an existing one.
|
||||||
* @return a storage statistics instance.
|
* @param ioStatistics IOStatistics to build the storage statistics from.
|
||||||
|
* @return a storage statistics instance; expected to be that of the FS.
|
||||||
*/
|
*/
|
||||||
protected static S3AStorageStatistics createStorageStatistics() {
|
protected static S3AStorageStatistics createStorageStatistics(
|
||||||
|
final IOStatistics ioStatistics) {
|
||||||
return (S3AStorageStatistics)
|
return (S3AStorageStatistics)
|
||||||
GlobalStorageStatistics.INSTANCE
|
GlobalStorageStatistics.INSTANCE
|
||||||
.put(S3AStorageStatistics.NAME,
|
.put(S3AStorageStatistics.NAME,
|
||||||
() -> new S3AStorageStatistics());
|
() -> new S3AStorageStatistics(ioStatistics));
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -610,6 +663,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
||||||
* Get S3A Instrumentation. For test purposes.
|
* Get S3A Instrumentation. For test purposes.
|
||||||
* @return this instance's instrumentation.
|
* @return this instance's instrumentation.
|
||||||
*/
|
*/
|
||||||
|
@VisibleForTesting
|
||||||
public S3AInstrumentation getInstrumentation() {
|
public S3AInstrumentation getInstrumentation() {
|
||||||
return instrumentation;
|
return instrumentation;
|
||||||
}
|
}
|
||||||
|
@ -677,8 +731,16 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
||||||
S3_CLIENT_FACTORY_IMPL, DEFAULT_S3_CLIENT_FACTORY_IMPL,
|
S3_CLIENT_FACTORY_IMPL, DEFAULT_S3_CLIENT_FACTORY_IMPL,
|
||||||
S3ClientFactory.class);
|
S3ClientFactory.class);
|
||||||
|
|
||||||
|
StatisticsFromAwsSdk awsStats = null;
|
||||||
|
// TODO: HADOOP-16830 when the S3 client building code works
|
||||||
|
// with different regions,
|
||||||
|
// then non-null stats can be passed in here.
|
||||||
|
if (AWS_SDK_METRICS_ENABLED) {
|
||||||
|
awsStats = statisticsContext.newStatisticsFromAwsSdk();
|
||||||
|
}
|
||||||
|
|
||||||
s3 = ReflectionUtils.newInstance(s3ClientFactoryClass, conf)
|
s3 = ReflectionUtils.newInstance(s3ClientFactoryClass, conf)
|
||||||
.createS3Client(getUri(), bucket, credentials, uaSuffix);
|
.createS3Client(getUri(), bucket, credentials, uaSuffix, awsStats);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -1176,7 +1238,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
||||||
invoker,
|
invoker,
|
||||||
s3guardInvoker,
|
s3guardInvoker,
|
||||||
statistics,
|
statistics,
|
||||||
instrumentation,
|
statisticsContext,
|
||||||
fileStatus,
|
fileStatus,
|
||||||
seekPolicy,
|
seekPolicy,
|
||||||
changePolicy,
|
changePolicy,
|
||||||
|
@ -1273,15 +1335,20 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
||||||
PutTracker putTracker =
|
PutTracker putTracker =
|
||||||
committerIntegration.createTracker(path, key);
|
committerIntegration.createTracker(path, key);
|
||||||
String destKey = putTracker.getDestKey();
|
String destKey = putTracker.getDestKey();
|
||||||
|
final BlockOutputStreamStatistics outputStreamStatistics
|
||||||
|
= statisticsContext.newOutputStreamStatistics();
|
||||||
return new FSDataOutputStream(
|
return new FSDataOutputStream(
|
||||||
new S3ABlockOutputStream(this,
|
new S3ABlockOutputStream(this,
|
||||||
destKey,
|
destKey,
|
||||||
new SemaphoredDelegatingExecutor(boundedThreadPool,
|
new SemaphoredDelegatingExecutor(
|
||||||
blockOutputActiveBlocks, true),
|
boundedThreadPool,
|
||||||
|
blockOutputActiveBlocks,
|
||||||
|
true,
|
||||||
|
outputStreamStatistics),
|
||||||
progress,
|
progress,
|
||||||
partSize,
|
partSize,
|
||||||
blockFactory,
|
blockFactory,
|
||||||
instrumentation.newOutputStreamStatistics(statistics),
|
outputStreamStatistics,
|
||||||
getWriteOperationHelper(),
|
getWriteOperationHelper(),
|
||||||
putTracker),
|
putTracker),
|
||||||
null);
|
null);
|
||||||
|
@ -1655,20 +1722,26 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
||||||
@Override
|
@Override
|
||||||
@Retries.RetryRaw
|
@Retries.RetryRaw
|
||||||
public CompletableFuture<S3ListResult> listObjectsAsync(
|
public CompletableFuture<S3ListResult> listObjectsAsync(
|
||||||
S3ListRequest request)
|
S3ListRequest request,
|
||||||
|
DurationTrackerFactory trackerFactory)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
return submit(unboundedThreadPool,
|
return submit(unboundedThreadPool, () ->
|
||||||
() -> listObjects(request));
|
listObjects(request,
|
||||||
|
pairedTrackerFactory(trackerFactory,
|
||||||
|
getDurationTrackerFactory())));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@Retries.RetryRaw
|
@Retries.RetryRaw
|
||||||
public CompletableFuture<S3ListResult> continueListObjectsAsync(
|
public CompletableFuture<S3ListResult> continueListObjectsAsync(
|
||||||
S3ListRequest request,
|
S3ListRequest request,
|
||||||
S3ListResult prevResult)
|
S3ListResult prevResult,
|
||||||
|
DurationTrackerFactory trackerFactory)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
return submit(unboundedThreadPool,
|
return submit(unboundedThreadPool,
|
||||||
() -> continueListObjects(request, prevResult));
|
() -> continueListObjects(request, prevResult,
|
||||||
|
pairedTrackerFactory(trackerFactory,
|
||||||
|
getDurationTrackerFactory())));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -1817,8 +1890,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
||||||
* @param count the count to increment
|
* @param count the count to increment
|
||||||
*/
|
*/
|
||||||
protected void incrementStatistic(Statistic statistic, long count) {
|
protected void incrementStatistic(Statistic statistic, long count) {
|
||||||
instrumentation.incrementCounter(statistic, count);
|
statisticsContext.incrementCounter(statistic, count);
|
||||||
storageStatistics.incrementCounter(statistic, count);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -1827,7 +1899,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
||||||
* @param count the count to decrement
|
* @param count the count to decrement
|
||||||
*/
|
*/
|
||||||
protected void decrementGauge(Statistic statistic, long count) {
|
protected void decrementGauge(Statistic statistic, long count) {
|
||||||
instrumentation.decrementGauge(statistic, count);
|
statisticsContext.decrementGauge(statistic, count);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -1836,7 +1908,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
||||||
* @param count the count to increment
|
* @param count the count to increment
|
||||||
*/
|
*/
|
||||||
protected void incrementGauge(Statistic statistic, long count) {
|
protected void incrementGauge(Statistic statistic, long count) {
|
||||||
instrumentation.incrementGauge(statistic, count);
|
statisticsContext.incrementGauge(statistic, count);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -1849,6 +1921,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
||||||
if (isThrottleException(ex)) {
|
if (isThrottleException(ex)) {
|
||||||
operationThrottled(false);
|
operationThrottled(false);
|
||||||
} else {
|
} else {
|
||||||
|
incrementStatistic(STORE_IO_RETRY);
|
||||||
incrementStatistic(IGNORED_ERRORS);
|
incrementStatistic(IGNORED_ERRORS);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -1900,11 +1973,11 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
||||||
LOG.debug("Request throttled on {}", metastore ? "S3": "DynamoDB");
|
LOG.debug("Request throttled on {}", metastore ? "S3": "DynamoDB");
|
||||||
if (metastore) {
|
if (metastore) {
|
||||||
incrementStatistic(S3GUARD_METADATASTORE_THROTTLED);
|
incrementStatistic(S3GUARD_METADATASTORE_THROTTLED);
|
||||||
instrumentation.addValueToQuantiles(S3GUARD_METADATASTORE_THROTTLE_RATE,
|
statisticsContext.addValueToQuantiles(S3GUARD_METADATASTORE_THROTTLE_RATE,
|
||||||
1);
|
1);
|
||||||
} else {
|
} else {
|
||||||
incrementStatistic(STORE_IO_THROTTLED);
|
incrementStatistic(STORE_IO_THROTTLED);
|
||||||
instrumentation.addValueToQuantiles(STORE_IO_THROTTLE_RATE, 1);
|
statisticsContext.addValueToQuantiles(STORE_IO_THROTTLE_RATE, 1);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1917,6 +1990,27 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
||||||
return storageStatistics;
|
return storageStatistics;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the instrumentation's IOStatistics.
|
||||||
|
* @return statistics
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public IOStatistics getIOStatistics() {
|
||||||
|
return instrumentation != null
|
||||||
|
? instrumentation.getIOStatistics()
|
||||||
|
: null;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the factory for duration tracking.
|
||||||
|
* @return a factory from the instrumentation.
|
||||||
|
*/
|
||||||
|
protected DurationTrackerFactory getDurationTrackerFactory() {
|
||||||
|
return instrumentation != null ?
|
||||||
|
instrumentation.getDurationTrackerFactory()
|
||||||
|
: null;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Request object metadata; increments counters in the process.
|
* Request object metadata; increments counters in the process.
|
||||||
* Retry policy: retry untranslated.
|
* Retry policy: retry untranslated.
|
||||||
|
@ -1954,15 +2048,30 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
||||||
ObjectMetadata meta = changeInvoker.retryUntranslated("GET " + key, true,
|
ObjectMetadata meta = changeInvoker.retryUntranslated("GET " + key, true,
|
||||||
() -> {
|
() -> {
|
||||||
incrementStatistic(OBJECT_METADATA_REQUESTS);
|
incrementStatistic(OBJECT_METADATA_REQUESTS);
|
||||||
LOG.debug("HEAD {} with change tracker {}", key, changeTracker);
|
DurationTracker duration = getDurationTrackerFactory()
|
||||||
if (changeTracker != null) {
|
.trackDuration(ACTION_HTTP_HEAD_REQUEST.getSymbol());
|
||||||
changeTracker.maybeApplyConstraint(request);
|
try {
|
||||||
|
LOG.debug("HEAD {} with change tracker {}", key, changeTracker);
|
||||||
|
if (changeTracker != null) {
|
||||||
|
changeTracker.maybeApplyConstraint(request);
|
||||||
|
}
|
||||||
|
ObjectMetadata objectMetadata = s3.getObjectMetadata(request);
|
||||||
|
if (changeTracker != null) {
|
||||||
|
changeTracker.processMetadata(objectMetadata, operation);
|
||||||
|
}
|
||||||
|
return objectMetadata;
|
||||||
|
} catch(AmazonServiceException ase) {
|
||||||
|
if (!isObjectNotFound(ase)) {
|
||||||
|
// file not found is not considered a failure of the call,
|
||||||
|
// so only switch the duration tracker to update failure
|
||||||
|
// metrics on other exception outcomes.
|
||||||
|
duration.failed();
|
||||||
|
}
|
||||||
|
throw ase;
|
||||||
|
} finally {
|
||||||
|
// update the tracker.
|
||||||
|
duration.close();
|
||||||
}
|
}
|
||||||
ObjectMetadata objectMetadata = s3.getObjectMetadata(request);
|
|
||||||
if (changeTracker != null) {
|
|
||||||
changeTracker.processMetadata(objectMetadata, operation);
|
|
||||||
}
|
|
||||||
return objectMetadata;
|
|
||||||
});
|
});
|
||||||
incrementReadOperations();
|
incrementReadOperations();
|
||||||
return meta;
|
return meta;
|
||||||
|
@ -1974,13 +2083,15 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
||||||
*
|
*
|
||||||
* Retry policy: retry untranslated.
|
* Retry policy: retry untranslated.
|
||||||
* @param request request to initiate
|
* @param request request to initiate
|
||||||
|
* @param trackerFactory duration tracking
|
||||||
* @return the results
|
* @return the results
|
||||||
* @throws IOException if the retry invocation raises one (it shouldn't).
|
* @throws IOException if the retry invocation raises one (it shouldn't).
|
||||||
*/
|
*/
|
||||||
@Retries.RetryRaw
|
@Retries.RetryRaw
|
||||||
protected S3ListResult listObjects(S3ListRequest request) throws IOException {
|
protected S3ListResult listObjects(S3ListRequest request,
|
||||||
|
@Nullable final DurationTrackerFactory trackerFactory)
|
||||||
|
throws IOException {
|
||||||
incrementReadOperations();
|
incrementReadOperations();
|
||||||
incrementStatistic(OBJECT_LIST_REQUESTS);
|
|
||||||
LOG.debug("LIST {}", request);
|
LOG.debug("LIST {}", request);
|
||||||
validateListArguments(request);
|
validateListArguments(request);
|
||||||
try(DurationInfo ignored =
|
try(DurationInfo ignored =
|
||||||
|
@ -1988,13 +2099,15 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
||||||
return invoker.retryUntranslated(
|
return invoker.retryUntranslated(
|
||||||
request.toString(),
|
request.toString(),
|
||||||
true,
|
true,
|
||||||
() -> {
|
trackDurationOfOperation(trackerFactory,
|
||||||
if (useListV1) {
|
OBJECT_LIST_REQUEST,
|
||||||
return S3ListResult.v1(s3.listObjects(request.getV1()));
|
() -> {
|
||||||
} else {
|
if (useListV1) {
|
||||||
return S3ListResult.v2(s3.listObjectsV2(request.getV2()));
|
return S3ListResult.v1(s3.listObjects(request.getV1()));
|
||||||
}
|
} else {
|
||||||
});
|
return S3ListResult.v2(s3.listObjectsV2(request.getV2()));
|
||||||
|
}
|
||||||
|
}));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -2015,12 +2128,14 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
||||||
* Retry policy: retry untranslated.
|
* Retry policy: retry untranslated.
|
||||||
* @param request last list objects request to continue
|
* @param request last list objects request to continue
|
||||||
* @param prevResult last paged result to continue from
|
* @param prevResult last paged result to continue from
|
||||||
|
* @param trackerFactory duration tracking
|
||||||
* @return the next result object
|
* @return the next result object
|
||||||
* @throws IOException none, just there for retryUntranslated.
|
* @throws IOException none, just there for retryUntranslated.
|
||||||
*/
|
*/
|
||||||
@Retries.RetryRaw
|
@Retries.RetryRaw
|
||||||
protected S3ListResult continueListObjects(S3ListRequest request,
|
protected S3ListResult continueListObjects(S3ListRequest request,
|
||||||
S3ListResult prevResult) throws IOException {
|
S3ListResult prevResult,
|
||||||
|
final DurationTrackerFactory trackerFactory) throws IOException {
|
||||||
incrementReadOperations();
|
incrementReadOperations();
|
||||||
validateListArguments(request);
|
validateListArguments(request);
|
||||||
try(DurationInfo ignored =
|
try(DurationInfo ignored =
|
||||||
|
@ -2028,17 +2143,19 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
||||||
return invoker.retryUntranslated(
|
return invoker.retryUntranslated(
|
||||||
request.toString(),
|
request.toString(),
|
||||||
true,
|
true,
|
||||||
() -> {
|
trackDurationOfOperation(
|
||||||
incrementStatistic(OBJECT_CONTINUE_LIST_REQUESTS);
|
trackerFactory,
|
||||||
if (useListV1) {
|
OBJECT_CONTINUE_LIST_REQUEST,
|
||||||
return S3ListResult.v1(
|
() -> {
|
||||||
s3.listNextBatchOfObjects(prevResult.getV1()));
|
if (useListV1) {
|
||||||
} else {
|
return S3ListResult.v1(
|
||||||
request.getV2().setContinuationToken(prevResult.getV2()
|
s3.listNextBatchOfObjects(prevResult.getV1()));
|
||||||
.getNextContinuationToken());
|
} else {
|
||||||
return S3ListResult.v2(s3.listObjectsV2(request.getV2()));
|
request.getV2().setContinuationToken(prevResult.getV2()
|
||||||
}
|
.getNextContinuationToken());
|
||||||
});
|
return S3ListResult.v2(s3.listObjectsV2(request.getV2()));
|
||||||
|
}
|
||||||
|
}));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -2083,9 +2200,10 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
||||||
invoker.retryUntranslated(String.format("Delete %s:/%s", bucket, key),
|
invoker.retryUntranslated(String.format("Delete %s:/%s", bucket, key),
|
||||||
DELETE_CONSIDERED_IDEMPOTENT,
|
DELETE_CONSIDERED_IDEMPOTENT,
|
||||||
()-> {
|
()-> {
|
||||||
incrementStatistic(OBJECT_DELETE_REQUESTS);
|
|
||||||
incrementStatistic(OBJECT_DELETE_OBJECTS);
|
incrementStatistic(OBJECT_DELETE_OBJECTS);
|
||||||
s3.deleteObject(bucket, key);
|
trackDurationOfInvocation(getDurationTrackerFactory(),
|
||||||
|
OBJECT_DELETE_REQUEST.getSymbol(),
|
||||||
|
() -> s3.deleteObject(bucket, key));
|
||||||
return null;
|
return null;
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
@ -2169,11 +2287,12 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
||||||
// handle the failure
|
// handle the failure
|
||||||
retryHandler.bulkDeleteRetried(deleteRequest, e);
|
retryHandler.bulkDeleteRetried(deleteRequest, e);
|
||||||
},
|
},
|
||||||
() -> {
|
// duration is tracked in the bulk delete counters
|
||||||
incrementStatistic(OBJECT_DELETE_REQUESTS, 1);
|
trackDurationOfOperation(getDurationTrackerFactory(),
|
||||||
incrementStatistic(OBJECT_DELETE_OBJECTS, keyCount);
|
OBJECT_BULK_DELETE_REQUEST.getSymbol(), () -> {
|
||||||
return s3.deleteObjects(deleteRequest);
|
incrementStatistic(OBJECT_DELETE_OBJECTS, keyCount);
|
||||||
});
|
return s3.deleteObjects(deleteRequest);
|
||||||
|
}));
|
||||||
} catch (MultiObjectDeleteException e) {
|
} catch (MultiObjectDeleteException e) {
|
||||||
// one or more of the keys could not be deleted.
|
// one or more of the keys could not be deleted.
|
||||||
// log and rethrow
|
// log and rethrow
|
||||||
|
@ -2665,17 +2784,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
||||||
throws FileNotFoundException, IOException {
|
throws FileNotFoundException, IOException {
|
||||||
RemoteIterator<S3AFileStatus> listStatusItr = once("listStatus",
|
RemoteIterator<S3AFileStatus> listStatusItr = once("listStatus",
|
||||||
p.toString(), () -> innerListStatus(p));
|
p.toString(), () -> innerListStatus(p));
|
||||||
return new RemoteIterator<FileStatus>() {
|
return typeCastingRemoteIterator(listStatusItr);
|
||||||
@Override
|
|
||||||
public boolean hasNext() throws IOException {
|
|
||||||
return listStatusItr.hasNext();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public FileStatus next() throws IOException {
|
|
||||||
return listStatusItr.next();
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -3201,8 +3310,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
||||||
S3ListRequest request = createListObjectsRequest(dirKey, "/",
|
S3ListRequest request = createListObjectsRequest(dirKey, "/",
|
||||||
listSize);
|
listSize);
|
||||||
// execute the request
|
// execute the request
|
||||||
S3ListResult listResult = listObjects(request);
|
S3ListResult listResult = listObjects(request,
|
||||||
|
getDurationTrackerFactory());
|
||||||
|
|
||||||
if (listResult.hasPrefixesOrObjects(contextAccessors, tombstones)) {
|
if (listResult.hasPrefixesOrObjects(contextAccessors, tombstones)) {
|
||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
|
@ -3607,8 +3716,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
||||||
ChangeTracker changeTracker = new ChangeTracker(
|
ChangeTracker changeTracker = new ChangeTracker(
|
||||||
keyToQualifiedPath(srcKey).toString(),
|
keyToQualifiedPath(srcKey).toString(),
|
||||||
changeDetectionPolicy,
|
changeDetectionPolicy,
|
||||||
readContext.instrumentation.newInputStreamStatistics()
|
readContext.getS3AStatisticsContext()
|
||||||
.getVersionMismatchCounter(),
|
.newInputStreamStatistics()
|
||||||
|
.getChangeTrackerStatistics(),
|
||||||
srcAttributes);
|
srcAttributes);
|
||||||
|
|
||||||
String action = "copyFile(" + srcKey + ", " + dstKey + ")";
|
String action = "copyFile(" + srcKey + ", " + dstKey + ")";
|
||||||
|
@ -4102,12 +4212,10 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
||||||
sb.append(", delegation tokens=")
|
sb.append(", delegation tokens=")
|
||||||
.append(delegationTokens.map(Objects::toString).orElse("disabled"));
|
.append(delegationTokens.map(Objects::toString).orElse("disabled"));
|
||||||
sb.append(", ").append(directoryPolicy);
|
sb.append(", ").append(directoryPolicy);
|
||||||
sb.append(", statistics {")
|
// if logging at debug, toString returns the entire IOStatistics set.
|
||||||
.append(statistics)
|
if (getInstrumentation() != null) {
|
||||||
.append("}");
|
sb.append(", instrumentation {")
|
||||||
if (instrumentation != null) {
|
.append(getInstrumentation().toString())
|
||||||
sb.append(", metrics {")
|
|
||||||
.append(instrumentation.dump("{", "=", "} ", true))
|
|
||||||
.append("}");
|
.append("}");
|
||||||
}
|
}
|
||||||
sb.append('}');
|
sb.append('}');
|
||||||
|
@ -4307,21 +4415,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
||||||
new Listing.AcceptFilesOnly(qualify(f)), null, true, false));
|
new Listing.AcceptFilesOnly(qualify(f)), null, true, false));
|
||||||
}
|
}
|
||||||
|
|
||||||
private static RemoteIterator<LocatedFileStatus> toLocatedFileStatusIterator(
|
|
||||||
RemoteIterator<? extends LocatedFileStatus> iterator) {
|
|
||||||
return new RemoteIterator<LocatedFileStatus>() {
|
|
||||||
@Override
|
|
||||||
public boolean hasNext() throws IOException {
|
|
||||||
return iterator.hasNext();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public LocatedFileStatus next() throws IOException {
|
|
||||||
return iterator.next();
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Recursive List of files and empty directories.
|
* Recursive List of files and empty directories.
|
||||||
* @param f path to list from
|
* @param f path to list from
|
||||||
|
@ -4404,7 +4497,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
||||||
if (status != null && status.isFile()) {
|
if (status != null && status.isFile()) {
|
||||||
// simple case: File
|
// simple case: File
|
||||||
LOG.debug("Path is a file: {}", path);
|
LOG.debug("Path is a file: {}", path);
|
||||||
return new Listing.SingleStatusRemoteIterator(
|
return listing.createSingleStatusIterator(
|
||||||
toLocatedFileStatus(status));
|
toLocatedFileStatus(status));
|
||||||
}
|
}
|
||||||
// Assuming the path to be a directory
|
// Assuming the path to be a directory
|
||||||
|
@ -4424,7 +4517,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
||||||
? status
|
? status
|
||||||
: (S3AFileStatus) getFileStatus(path);
|
: (S3AFileStatus) getFileStatus(path);
|
||||||
if (fileStatus.isFile()) {
|
if (fileStatus.isFile()) {
|
||||||
return new Listing.SingleStatusRemoteIterator(
|
return listing.createSingleStatusIterator(
|
||||||
toLocatedFileStatus(fileStatus));
|
toLocatedFileStatus(fileStatus));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -4483,7 +4576,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
||||||
if (fileStatus.isFile()) {
|
if (fileStatus.isFile()) {
|
||||||
// simple case: File
|
// simple case: File
|
||||||
LOG.debug("Path is a file");
|
LOG.debug("Path is a file");
|
||||||
return new Listing.SingleStatusRemoteIterator(
|
return listing.createSingleStatusIterator(
|
||||||
filter.accept(path)
|
filter.accept(path)
|
||||||
? toLocatedFileStatus(fileStatus)
|
? toLocatedFileStatus(fileStatus)
|
||||||
: null);
|
: null);
|
||||||
|
@ -4630,8 +4723,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
||||||
* Create a new instance of the committer statistics.
|
* Create a new instance of the committer statistics.
|
||||||
* @return a new committer statistics instance
|
* @return a new committer statistics instance
|
||||||
*/
|
*/
|
||||||
public S3AInstrumentation.CommitterStatistics newCommitterStatistics() {
|
public CommitterStatistics newCommitterStatistics() {
|
||||||
return instrumentation.newCommitterStatistics();
|
return statisticsContext.newCommitterStatistics();
|
||||||
}
|
}
|
||||||
|
|
||||||
@SuppressWarnings("deprecation")
|
@SuppressWarnings("deprecation")
|
||||||
|
@ -4765,8 +4858,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
||||||
ChangeTracker changeTracker =
|
ChangeTracker changeTracker =
|
||||||
new ChangeTracker(uri.toString(),
|
new ChangeTracker(uri.toString(),
|
||||||
changeDetectionPolicy,
|
changeDetectionPolicy,
|
||||||
readContext.instrumentation.newInputStreamStatistics()
|
readContext.getS3AStatisticsContext()
|
||||||
.getVersionMismatchCounter(),
|
.newInputStreamStatistics()
|
||||||
|
.getChangeTrackerStatistics(),
|
||||||
objectAttributes);
|
objectAttributes);
|
||||||
|
|
||||||
// will retry internally if wrong version detected
|
// will retry internally if wrong version detected
|
||||||
|
@ -4913,7 +5007,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
||||||
getWriteOperationHelper(),
|
getWriteOperationHelper(),
|
||||||
ctx,
|
ctx,
|
||||||
basePath,
|
basePath,
|
||||||
new S3AMultipartUploaderStatisticsImpl(ctx::incrementStatistic));
|
statisticsContext.createMultipartUploaderStatistics());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -4933,7 +5027,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
||||||
.setExecutor(boundedThreadPool)
|
.setExecutor(boundedThreadPool)
|
||||||
.setExecutorCapacity(executorCapacity)
|
.setExecutorCapacity(executorCapacity)
|
||||||
.setInvoker(invoker)
|
.setInvoker(invoker)
|
||||||
.setInstrumentation(getInstrumentation())
|
.setInstrumentation(statisticsContext)
|
||||||
.setStorageStatistics(getStorageStatistics())
|
.setStorageStatistics(getStorageStatistics())
|
||||||
.setInputPolicy(getInputPolicy())
|
.setInputPolicy(getInputPolicy())
|
||||||
.setChangeDetectionPolicy(changeDetectionPolicy)
|
.setChangeDetectionPolicy(changeDetectionPolicy)
|
||||||
|
|
|
@ -33,10 +33,14 @@ import org.apache.hadoop.classification.InterfaceStability;
|
||||||
import org.apache.hadoop.fs.CanSetReadahead;
|
import org.apache.hadoop.fs.CanSetReadahead;
|
||||||
import org.apache.hadoop.fs.CanUnbuffer;
|
import org.apache.hadoop.fs.CanUnbuffer;
|
||||||
import org.apache.hadoop.fs.FSExceptionMessages;
|
import org.apache.hadoop.fs.FSExceptionMessages;
|
||||||
import org.apache.hadoop.fs.FSInputStream;
|
import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
|
||||||
|
import org.apache.hadoop.fs.s3a.impl.ChangeTracker;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatistics;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
|
||||||
import org.apache.hadoop.fs.PathIOException;
|
import org.apache.hadoop.fs.PathIOException;
|
||||||
import org.apache.hadoop.fs.StreamCapabilities;
|
import org.apache.hadoop.fs.StreamCapabilities;
|
||||||
import org.apache.hadoop.fs.s3a.impl.ChangeTracker;
|
import org.apache.hadoop.fs.FSInputStream;
|
||||||
|
import org.apache.hadoop.fs.statistics.DurationTracker;
|
||||||
|
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
@ -68,7 +72,7 @@ import static org.apache.hadoop.util.StringUtils.toLowerCase;
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
@InterfaceStability.Evolving
|
@InterfaceStability.Evolving
|
||||||
public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
||||||
CanUnbuffer, StreamCapabilities {
|
CanUnbuffer, StreamCapabilities, IOStatisticsSource {
|
||||||
|
|
||||||
public static final String E_NEGATIVE_READAHEAD_VALUE
|
public static final String E_NEGATIVE_READAHEAD_VALUE
|
||||||
= "Negative readahead value";
|
= "Negative readahead value";
|
||||||
|
@ -105,7 +109,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
||||||
private final String uri;
|
private final String uri;
|
||||||
private static final Logger LOG =
|
private static final Logger LOG =
|
||||||
LoggerFactory.getLogger(S3AInputStream.class);
|
LoggerFactory.getLogger(S3AInputStream.class);
|
||||||
private final S3AInstrumentation.InputStreamStatistics streamStatistics;
|
private final S3AInputStreamStatistics streamStatistics;
|
||||||
private S3AEncryptionMethods serverSideEncryptionAlgorithm;
|
private S3AEncryptionMethods serverSideEncryptionAlgorithm;
|
||||||
private String serverSideEncryptionKey;
|
private String serverSideEncryptionKey;
|
||||||
private S3AInputPolicy inputPolicy;
|
private S3AInputPolicy inputPolicy;
|
||||||
|
@ -131,6 +135,11 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
||||||
/** change tracker. */
|
/** change tracker. */
|
||||||
private final ChangeTracker changeTracker;
|
private final ChangeTracker changeTracker;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* IOStatistics report.
|
||||||
|
*/
|
||||||
|
private final IOStatistics ioStatistics;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create the stream.
|
* Create the stream.
|
||||||
* This does not attempt to open it; that is only done on the first
|
* This does not attempt to open it; that is only done on the first
|
||||||
|
@ -154,13 +163,15 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
||||||
this.contentLength = l;
|
this.contentLength = l;
|
||||||
this.client = client;
|
this.client = client;
|
||||||
this.uri = "s3a://" + this.bucket + "/" + this.key;
|
this.uri = "s3a://" + this.bucket + "/" + this.key;
|
||||||
this.streamStatistics = ctx.instrumentation.newInputStreamStatistics();
|
this.streamStatistics = ctx.getS3AStatisticsContext()
|
||||||
|
.newInputStreamStatistics();
|
||||||
|
this.ioStatistics = streamStatistics.getIOStatistics();
|
||||||
this.serverSideEncryptionAlgorithm =
|
this.serverSideEncryptionAlgorithm =
|
||||||
s3Attributes.getServerSideEncryptionAlgorithm();
|
s3Attributes.getServerSideEncryptionAlgorithm();
|
||||||
this.serverSideEncryptionKey = s3Attributes.getServerSideEncryptionKey();
|
this.serverSideEncryptionKey = s3Attributes.getServerSideEncryptionKey();
|
||||||
this.changeTracker = new ChangeTracker(uri,
|
this.changeTracker = new ChangeTracker(uri,
|
||||||
ctx.getChangeDetectionPolicy(),
|
ctx.getChangeDetectionPolicy(),
|
||||||
streamStatistics.getVersionMismatchCounter(),
|
streamStatistics.getChangeTrackerStatistics(),
|
||||||
s3Attributes);
|
s3Attributes);
|
||||||
setInputPolicy(ctx.getInputPolicy());
|
setInputPolicy(ctx.getInputPolicy());
|
||||||
setReadahead(ctx.getReadahead());
|
setReadahead(ctx.getReadahead());
|
||||||
|
@ -210,8 +221,21 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
||||||
String text = String.format("%s %s at %d",
|
String text = String.format("%s %s at %d",
|
||||||
operation, uri, targetPos);
|
operation, uri, targetPos);
|
||||||
changeTracker.maybeApplyConstraint(request);
|
changeTracker.maybeApplyConstraint(request);
|
||||||
object = Invoker.once(text, uri,
|
DurationTracker tracker = streamStatistics.initiateGetRequest();
|
||||||
() -> client.getObject(request));
|
try {
|
||||||
|
object = Invoker.once(text, uri,
|
||||||
|
() -> client.getObject(request));
|
||||||
|
} catch(IOException e) {
|
||||||
|
// input function failed: note it
|
||||||
|
tracker.failed();
|
||||||
|
// and rethrow
|
||||||
|
throw e;
|
||||||
|
} finally {
|
||||||
|
// update the tracker.
|
||||||
|
// this is called after any catch() call will have
|
||||||
|
// set the failed flag.
|
||||||
|
tracker.close();
|
||||||
|
}
|
||||||
|
|
||||||
changeTracker.processResponse(object, operation,
|
changeTracker.processResponse(object, operation,
|
||||||
targetPos);
|
targetPos);
|
||||||
|
@ -294,13 +318,11 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
||||||
if (skipForward) {
|
if (skipForward) {
|
||||||
// the forward seek range is within the limits
|
// the forward seek range is within the limits
|
||||||
LOG.debug("Forward seek on {}, of {} bytes", uri, diff);
|
LOG.debug("Forward seek on {}, of {} bytes", uri, diff);
|
||||||
streamStatistics.seekForwards(diff);
|
|
||||||
long skipped = wrappedStream.skip(diff);
|
long skipped = wrappedStream.skip(diff);
|
||||||
if (skipped > 0) {
|
if (skipped > 0) {
|
||||||
pos += skipped;
|
pos += skipped;
|
||||||
// as these bytes have been read, they are included in the counter
|
|
||||||
incrementBytesRead(diff);
|
|
||||||
}
|
}
|
||||||
|
streamStatistics.seekForwards(diff, skipped);
|
||||||
|
|
||||||
if (pos == targetPos) {
|
if (pos == targetPos) {
|
||||||
// all is well
|
// all is well
|
||||||
|
@ -312,6 +334,9 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
||||||
LOG.warn("Failed to seek on {} to {}. Current position {}",
|
LOG.warn("Failed to seek on {} to {}. Current position {}",
|
||||||
uri, targetPos, pos);
|
uri, targetPos, pos);
|
||||||
}
|
}
|
||||||
|
} else {
|
||||||
|
// not attempting to read any bytes from the stream
|
||||||
|
streamStatistics.seekForwards(diff, 0);
|
||||||
}
|
}
|
||||||
} else if (diff < 0) {
|
} else if (diff < 0) {
|
||||||
// backwards seek
|
// backwards seek
|
||||||
|
@ -356,7 +381,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
||||||
// open. After that, an exception generally means the file has changed
|
// open. After that, an exception generally means the file has changed
|
||||||
// and there is no point retrying anymore.
|
// and there is no point retrying anymore.
|
||||||
Invoker invoker = context.getReadInvoker();
|
Invoker invoker = context.getReadInvoker();
|
||||||
invoker.maybeRetry(streamStatistics.openOperations == 0,
|
invoker.maybeRetry(streamStatistics.getOpenOperations() == 0,
|
||||||
"lazySeek", pathStr, true,
|
"lazySeek", pathStr, true,
|
||||||
() -> {
|
() -> {
|
||||||
//For lazy seek
|
//For lazy seek
|
||||||
|
@ -720,7 +745,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
||||||
sb.append(" contentRangeFinish=").append(contentRangeFinish);
|
sb.append(" contentRangeFinish=").append(contentRangeFinish);
|
||||||
sb.append(" remainingInCurrentRequest=")
|
sb.append(" remainingInCurrentRequest=")
|
||||||
.append(remainingInCurrentRequest());
|
.append(remainingInCurrentRequest());
|
||||||
sb.append(changeTracker);
|
sb.append(" ").append(changeTracker);
|
||||||
sb.append('\n').append(s);
|
sb.append('\n').append(s);
|
||||||
sb.append('}');
|
sb.append('}');
|
||||||
return sb.toString();
|
return sb.toString();
|
||||||
|
@ -774,7 +799,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
@InterfaceStability.Unstable
|
@InterfaceStability.Unstable
|
||||||
public S3AInstrumentation.InputStreamStatistics getS3AStreamStatistics() {
|
public S3AInputStreamStatistics getS3AStreamStatistics() {
|
||||||
return streamStatistics;
|
return streamStatistics;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -858,13 +883,14 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
||||||
try {
|
try {
|
||||||
closeStream("unbuffer()", contentRangeFinish, false);
|
closeStream("unbuffer()", contentRangeFinish, false);
|
||||||
} finally {
|
} finally {
|
||||||
streamStatistics.merge(false);
|
streamStatistics.unbuffered();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean hasCapability(String capability) {
|
public boolean hasCapability(String capability) {
|
||||||
switch (toLowerCase(capability)) {
|
switch (toLowerCase(capability)) {
|
||||||
|
case StreamCapabilities.IOSTATISTICS:
|
||||||
case StreamCapabilities.READAHEAD:
|
case StreamCapabilities.READAHEAD:
|
||||||
case StreamCapabilities.UNBUFFER:
|
case StreamCapabilities.UNBUFFER:
|
||||||
return true;
|
return true;
|
||||||
|
@ -877,4 +903,9 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
||||||
boolean isObjectStreamOpen() {
|
boolean isObjectStreamOpen() {
|
||||||
return wrappedStream != null;
|
return wrappedStream != null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public IOStatistics getIOStatistics() {
|
||||||
|
return ioStatistics;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -24,19 +24,23 @@ import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
|
||||||
|
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.s3a.impl.ActiveOperationContext;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Base class for operation context struct passed through codepaths for main
|
* Class for operation context struct passed through codepaths for main
|
||||||
* S3AFileSystem operations.
|
* S3AFileSystem operations.
|
||||||
* Anything op-specific should be moved to a subclass of this.
|
* 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.
|
@SuppressWarnings("visibilitymodifier")
|
||||||
public class S3AOpContext {
|
public class S3AOpContext extends ActiveOperationContext {
|
||||||
|
|
||||||
final boolean isS3GuardEnabled;
|
final boolean isS3GuardEnabled;
|
||||||
final Invoker invoker;
|
final Invoker invoker;
|
||||||
@Nullable final FileSystem.Statistics stats;
|
@Nullable final FileSystem.Statistics stats;
|
||||||
final S3AInstrumentation instrumentation;
|
|
||||||
@Nullable final Invoker s3guardInvoker;
|
@Nullable final Invoker s3guardInvoker;
|
||||||
|
|
||||||
/** FileStatus for "destination" path being operated on. */
|
/** FileStatus for "destination" path being operated on. */
|
||||||
|
@ -53,9 +57,14 @@ public class S3AOpContext {
|
||||||
* @param dstFileStatus file status from existence check
|
* @param dstFileStatus file status from existence check
|
||||||
*/
|
*/
|
||||||
public S3AOpContext(boolean isS3GuardEnabled, Invoker invoker,
|
public S3AOpContext(boolean isS3GuardEnabled, Invoker invoker,
|
||||||
Invoker s3guardInvoker, @Nullable FileSystem.Statistics stats,
|
@Nullable Invoker s3guardInvoker,
|
||||||
S3AInstrumentation instrumentation, FileStatus dstFileStatus) {
|
@Nullable FileSystem.Statistics stats,
|
||||||
|
S3AStatisticsContext instrumentation,
|
||||||
|
FileStatus dstFileStatus) {
|
||||||
|
|
||||||
|
super(newOperationId(),
|
||||||
|
instrumentation,
|
||||||
|
null);
|
||||||
Preconditions.checkNotNull(invoker, "Null invoker arg");
|
Preconditions.checkNotNull(invoker, "Null invoker arg");
|
||||||
Preconditions.checkNotNull(instrumentation, "Null instrumentation arg");
|
Preconditions.checkNotNull(instrumentation, "Null instrumentation arg");
|
||||||
Preconditions.checkNotNull(dstFileStatus, "Null dstFileStatus arg");
|
Preconditions.checkNotNull(dstFileStatus, "Null dstFileStatus arg");
|
||||||
|
@ -65,7 +74,6 @@ public class S3AOpContext {
|
||||||
this.invoker = invoker;
|
this.invoker = invoker;
|
||||||
this.s3guardInvoker = s3guardInvoker;
|
this.s3guardInvoker = s3guardInvoker;
|
||||||
this.stats = stats;
|
this.stats = stats;
|
||||||
this.instrumentation = instrumentation;
|
|
||||||
this.dstFileStatus = dstFileStatus;
|
this.dstFileStatus = dstFileStatus;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -77,8 +85,10 @@ public class S3AOpContext {
|
||||||
* @param instrumentation instrumentation to use
|
* @param instrumentation instrumentation to use
|
||||||
* @param dstFileStatus file status from existence check
|
* @param dstFileStatus file status from existence check
|
||||||
*/
|
*/
|
||||||
public S3AOpContext(boolean isS3GuardEnabled, Invoker invoker,
|
public S3AOpContext(boolean isS3GuardEnabled,
|
||||||
@Nullable FileSystem.Statistics stats, S3AInstrumentation instrumentation,
|
Invoker invoker,
|
||||||
|
@Nullable FileSystem.Statistics stats,
|
||||||
|
S3AStatisticsContext instrumentation,
|
||||||
FileStatus dstFileStatus) {
|
FileStatus dstFileStatus) {
|
||||||
this(isS3GuardEnabled, invoker, null, stats, instrumentation,
|
this(isS3GuardEnabled, invoker, null, stats, instrumentation,
|
||||||
dstFileStatus);
|
dstFileStatus);
|
||||||
|
@ -97,10 +107,6 @@ public class S3AOpContext {
|
||||||
return stats;
|
return stats;
|
||||||
}
|
}
|
||||||
|
|
||||||
public S3AInstrumentation getInstrumentation() {
|
|
||||||
return instrumentation;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
public Invoker getS3guardInvoker() {
|
public Invoker getS3guardInvoker() {
|
||||||
return s3guardInvoker;
|
return s3guardInvoker;
|
||||||
|
|
|
@ -22,6 +22,7 @@ import org.apache.hadoop.fs.FileStatus;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy;
|
import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
|
|
||||||
|
@ -60,8 +61,8 @@ public class S3AReadOpContext extends S3AOpContext {
|
||||||
* @param isS3GuardEnabled true iff S3Guard is enabled.
|
* @param isS3GuardEnabled true iff S3Guard is enabled.
|
||||||
* @param invoker invoker for normal retries.
|
* @param invoker invoker for normal retries.
|
||||||
* @param s3guardInvoker S3Guard-specific retry invoker.
|
* @param s3guardInvoker S3Guard-specific retry invoker.
|
||||||
* @param stats statistics (may be null)
|
* @param stats Fileystem statistics (may be null)
|
||||||
* @param instrumentation FS instrumentation
|
* @param instrumentation statistics context
|
||||||
* @param dstFileStatus target file status
|
* @param dstFileStatus target file status
|
||||||
* @param inputPolicy the input policy
|
* @param inputPolicy the input policy
|
||||||
* @param readahead readahead for GET operations/skip, etc.
|
* @param readahead readahead for GET operations/skip, etc.
|
||||||
|
@ -71,13 +72,14 @@ public class S3AReadOpContext extends S3AOpContext {
|
||||||
final Path path,
|
final Path path,
|
||||||
boolean isS3GuardEnabled,
|
boolean isS3GuardEnabled,
|
||||||
Invoker invoker,
|
Invoker invoker,
|
||||||
Invoker s3guardInvoker,
|
@Nullable Invoker s3guardInvoker,
|
||||||
@Nullable FileSystem.Statistics stats,
|
@Nullable FileSystem.Statistics stats,
|
||||||
S3AInstrumentation instrumentation,
|
S3AStatisticsContext instrumentation,
|
||||||
FileStatus dstFileStatus,
|
FileStatus dstFileStatus,
|
||||||
S3AInputPolicy inputPolicy,
|
S3AInputPolicy inputPolicy,
|
||||||
ChangeDetectionPolicy changeDetectionPolicy,
|
ChangeDetectionPolicy changeDetectionPolicy,
|
||||||
final long readahead) {
|
final long readahead) {
|
||||||
|
|
||||||
super(isS3GuardEnabled, invoker, s3guardInvoker, stats, instrumentation,
|
super(isS3GuardEnabled, invoker, s3guardInvoker, stats, instrumentation,
|
||||||
dstFileStatus);
|
dstFileStatus);
|
||||||
this.path = checkNotNull(path);
|
this.path = checkNotNull(path);
|
||||||
|
|
|
@ -20,106 +20,27 @@ package org.apache.hadoop.fs.s3a;
|
||||||
|
|
||||||
import org.apache.hadoop.classification.InterfaceAudience;
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.classification.InterfaceStability;
|
import org.apache.hadoop.classification.InterfaceStability;
|
||||||
import org.apache.hadoop.fs.StorageStatistics;
|
import org.apache.hadoop.fs.statistics.IOStatistics;
|
||||||
import org.slf4j.Logger;
|
import org.apache.hadoop.fs.statistics.impl.StorageStatisticsFromIOStatistics;
|
||||||
import org.slf4j.LoggerFactory;
|
|
||||||
|
|
||||||
import java.util.Collections;
|
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.emptyStatistics;
|
||||||
import java.util.EnumMap;
|
|
||||||
import java.util.Iterator;
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.NoSuchElementException;
|
|
||||||
import java.util.concurrent.atomic.AtomicLong;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Storage statistics for S3A.
|
* Storage statistics for S3A, dynamically generated from the IOStatistics.
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
@InterfaceStability.Evolving
|
@InterfaceStability.Evolving
|
||||||
public class S3AStorageStatistics extends StorageStatistics
|
public class S3AStorageStatistics
|
||||||
implements Iterable<StorageStatistics.LongStatistic> {
|
extends StorageStatisticsFromIOStatistics {
|
||||||
private static final Logger LOG =
|
|
||||||
LoggerFactory.getLogger(S3AStorageStatistics.class);
|
|
||||||
|
|
||||||
public static final String NAME = "S3AStorageStatistics";
|
public static final String NAME = "S3AStorageStatistics";
|
||||||
private final Map<Statistic, AtomicLong> opsCount =
|
|
||||||
new EnumMap<>(Statistic.class);
|
public S3AStorageStatistics(final IOStatistics ioStatistics) {
|
||||||
|
super(NAME, "s3a", ioStatistics);
|
||||||
|
}
|
||||||
|
|
||||||
public S3AStorageStatistics() {
|
public S3AStorageStatistics() {
|
||||||
super(NAME);
|
super(NAME, "s3a", emptyStatistics());
|
||||||
for (Statistic opType : Statistic.values()) {
|
|
||||||
opsCount.put(opType, new AtomicLong(0));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Increment a specific counter.
|
|
||||||
* @param op operation
|
|
||||||
* @param count increment value
|
|
||||||
* @return the new value
|
|
||||||
*/
|
|
||||||
public long incrementCounter(Statistic op, long count) {
|
|
||||||
long updated = opsCount.get(op).addAndGet(count);
|
|
||||||
LOG.debug("{} += {} -> {}", op, count, updated);
|
|
||||||
return updated;
|
|
||||||
}
|
|
||||||
|
|
||||||
private class LongIterator implements Iterator<LongStatistic> {
|
|
||||||
private Iterator<Map.Entry<Statistic, AtomicLong>> iterator =
|
|
||||||
Collections.unmodifiableSet(opsCount.entrySet()).iterator();
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean hasNext() {
|
|
||||||
return iterator.hasNext();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public LongStatistic next() {
|
|
||||||
if (!iterator.hasNext()) {
|
|
||||||
throw new NoSuchElementException();
|
|
||||||
}
|
|
||||||
final Map.Entry<Statistic, AtomicLong> entry = iterator.next();
|
|
||||||
return new LongStatistic(entry.getKey().getSymbol(),
|
|
||||||
entry.getValue().get());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void remove() {
|
|
||||||
throw new UnsupportedOperationException();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public String getScheme() {
|
|
||||||
return "s3a";
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Iterator<LongStatistic> getLongStatistics() {
|
|
||||||
return new LongIterator();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Iterator<LongStatistic> iterator() {
|
|
||||||
return getLongStatistics();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Long getLong(String key) {
|
|
||||||
final Statistic type = Statistic.fromSymbol(key);
|
|
||||||
return type == null ? null : opsCount.get(type).get();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean isTracked(String key) {
|
|
||||||
return Statistic.fromSymbol(key) != null;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void reset() {
|
|
||||||
for (AtomicLong value : opsCount.values()) {
|
|
||||||
value.set(0);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -48,6 +48,7 @@ import org.apache.hadoop.fs.LocatedFileStatus;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.fs.PathFilter;
|
import org.apache.hadoop.fs.PathFilter;
|
||||||
import org.apache.hadoop.fs.RemoteIterator;
|
import org.apache.hadoop.fs.RemoteIterator;
|
||||||
|
import org.apache.hadoop.util.functional.RemoteIterators;
|
||||||
import org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider;
|
import org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider;
|
||||||
import org.apache.hadoop.fs.s3a.impl.NetworkBinding;
|
import org.apache.hadoop.fs.s3a.impl.NetworkBinding;
|
||||||
import org.apache.hadoop.fs.s3native.S3xLoginHelper;
|
import org.apache.hadoop.fs.s3native.S3xLoginHelper;
|
||||||
|
@ -1470,12 +1471,7 @@ public final class S3AUtils {
|
||||||
public static long applyLocatedFiles(
|
public static long applyLocatedFiles(
|
||||||
RemoteIterator<? extends LocatedFileStatus> iterator,
|
RemoteIterator<? extends LocatedFileStatus> iterator,
|
||||||
CallOnLocatedFileStatus eval) throws IOException {
|
CallOnLocatedFileStatus eval) throws IOException {
|
||||||
long count = 0;
|
return RemoteIterators.foreach(iterator, eval::call);
|
||||||
while (iterator.hasNext()) {
|
|
||||||
count++;
|
|
||||||
eval.call(iterator.next());
|
|
||||||
}
|
|
||||||
return count;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -26,6 +26,7 @@ import com.amazonaws.services.s3.AmazonS3;
|
||||||
|
|
||||||
import org.apache.hadoop.classification.InterfaceAudience;
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.classification.InterfaceStability;
|
import org.apache.hadoop.classification.InterfaceStability;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.StatisticsFromAwsSdk;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Factory for creation of {@link AmazonS3} client instances.
|
* Factory for creation of {@link AmazonS3} client instances.
|
||||||
|
@ -41,12 +42,14 @@ public interface S3ClientFactory {
|
||||||
* @param bucket Optional bucket to use to look up per-bucket proxy secrets
|
* @param bucket Optional bucket to use to look up per-bucket proxy secrets
|
||||||
* @param credentialSet credentials to use
|
* @param credentialSet credentials to use
|
||||||
* @param userAgentSuffix optional suffix for the UA field.
|
* @param userAgentSuffix optional suffix for the UA field.
|
||||||
|
* @param statisticsFromAwsSdk binding for AWS stats - may be null
|
||||||
* @return S3 client
|
* @return S3 client
|
||||||
* @throws IOException IO problem
|
* @throws IOException IO problem
|
||||||
*/
|
*/
|
||||||
AmazonS3 createS3Client(URI name,
|
AmazonS3 createS3Client(URI name,
|
||||||
String bucket,
|
String bucket,
|
||||||
AWSCredentialsProvider credentialSet,
|
AWSCredentialsProvider credentialSet,
|
||||||
String userAgentSuffix) throws IOException;
|
String userAgentSuffix,
|
||||||
|
StatisticsFromAwsSdk statisticsFromAwsSdk) throws IOException;
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -18,248 +18,504 @@
|
||||||
|
|
||||||
package org.apache.hadoop.fs.s3a;
|
package org.apache.hadoop.fs.s3a;
|
||||||
|
|
||||||
import org.apache.hadoop.fs.StorageStatistics.CommonStatisticNames;
|
|
||||||
|
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
|
import org.apache.hadoop.classification.InterfaceStability;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.StatisticTypeEnum;
|
||||||
|
import org.apache.hadoop.fs.statistics.StoreStatisticNames;
|
||||||
|
import org.apache.hadoop.fs.statistics.StreamStatisticNames;
|
||||||
|
|
||||||
|
import static org.apache.hadoop.fs.s3a.statistics.StatisticTypeEnum.TYPE_COUNTER;
|
||||||
|
import static org.apache.hadoop.fs.s3a.statistics.StatisticTypeEnum.TYPE_DURATION;
|
||||||
|
import static org.apache.hadoop.fs.s3a.statistics.StatisticTypeEnum.TYPE_GAUGE;
|
||||||
|
import static org.apache.hadoop.fs.s3a.statistics.StatisticTypeEnum.TYPE_QUANTILE;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Statistic which are collected in S3A.
|
* Statistic which are collected in S3A.
|
||||||
* These statistics are available at a low level in {@link S3AStorageStatistics}
|
* Counter and duration statistics are published in
|
||||||
* and as metrics in {@link S3AInstrumentation}
|
* {@link S3AFileSystem#getStorageStatistics()}.
|
||||||
|
* and as metrics in {@link S3AInstrumentation}.
|
||||||
|
* <p>
|
||||||
|
* Where possible, stream names come from {@link StreamStatisticNames}
|
||||||
|
* and {@link StoreStatisticNames}
|
||||||
|
* </p>
|
||||||
*/
|
*/
|
||||||
|
@InterfaceStability.Unstable
|
||||||
public enum Statistic {
|
public enum Statistic {
|
||||||
|
|
||||||
|
/* Low-level duration counters */
|
||||||
|
ACTION_EXECUTOR_ACQUIRED(
|
||||||
|
StoreStatisticNames.ACTION_EXECUTOR_ACQUIRED,
|
||||||
|
"Executor acquired.",
|
||||||
|
TYPE_DURATION),
|
||||||
|
ACTION_HTTP_HEAD_REQUEST(
|
||||||
|
StoreStatisticNames.ACTION_HTTP_HEAD_REQUEST,
|
||||||
|
"HEAD request.",
|
||||||
|
TYPE_DURATION),
|
||||||
|
ACTION_HTTP_GET_REQUEST(
|
||||||
|
StoreStatisticNames.ACTION_HTTP_GET_REQUEST,
|
||||||
|
"GET request.",
|
||||||
|
TYPE_DURATION),
|
||||||
|
|
||||||
|
/* FileSystem Level statistics */
|
||||||
DIRECTORIES_CREATED("directories_created",
|
DIRECTORIES_CREATED("directories_created",
|
||||||
"Total number of directories created through the object store."),
|
"Total number of directories created through the object store.",
|
||||||
|
TYPE_COUNTER),
|
||||||
DIRECTORIES_DELETED("directories_deleted",
|
DIRECTORIES_DELETED("directories_deleted",
|
||||||
"Total number of directories deleted through the object store."),
|
"Total number of directories deleted through the object store.",
|
||||||
|
TYPE_COUNTER),
|
||||||
FILES_COPIED("files_copied",
|
FILES_COPIED("files_copied",
|
||||||
"Total number of files copied within the object store."),
|
"Total number of files copied within the object store.",
|
||||||
|
TYPE_COUNTER),
|
||||||
FILES_COPIED_BYTES("files_copied_bytes",
|
FILES_COPIED_BYTES("files_copied_bytes",
|
||||||
"Total number of bytes copied within the object store."),
|
"Total number of bytes copied within the object store.",
|
||||||
|
TYPE_COUNTER),
|
||||||
FILES_CREATED("files_created",
|
FILES_CREATED("files_created",
|
||||||
"Total number of files created through the object store."),
|
"Total number of files created through the object store.",
|
||||||
|
TYPE_COUNTER),
|
||||||
FILES_DELETED("files_deleted",
|
FILES_DELETED("files_deleted",
|
||||||
"Total number of files deleted from the object store."),
|
"Total number of files deleted from the object store.",
|
||||||
|
TYPE_COUNTER),
|
||||||
FILES_DELETE_REJECTED("files_delete_rejected",
|
FILES_DELETE_REJECTED("files_delete_rejected",
|
||||||
"Total number of files whose delete request was rejected"),
|
"Total number of files whose delete request was rejected",
|
||||||
|
TYPE_COUNTER),
|
||||||
FAKE_DIRECTORIES_CREATED("fake_directories_created",
|
FAKE_DIRECTORIES_CREATED("fake_directories_created",
|
||||||
"Total number of fake directory entries created in the object store."),
|
"Total number of fake directory entries created in the object store.",
|
||||||
|
TYPE_COUNTER),
|
||||||
FAKE_DIRECTORIES_DELETED("fake_directories_deleted",
|
FAKE_DIRECTORIES_DELETED("fake_directories_deleted",
|
||||||
"Total number of fake directory deletes submitted to object store."),
|
"Total number of fake directory deletes submitted to object store.",
|
||||||
IGNORED_ERRORS("ignored_errors", "Errors caught and ignored"),
|
TYPE_COUNTER),
|
||||||
INVOCATION_COPY_FROM_LOCAL_FILE(CommonStatisticNames.OP_COPY_FROM_LOCAL_FILE,
|
IGNORED_ERRORS("ignored_errors", "Errors caught and ignored",
|
||||||
"Calls of copyFromLocalFile()"),
|
TYPE_COUNTER),
|
||||||
INVOCATION_CREATE(CommonStatisticNames.OP_CREATE,
|
INVOCATION_COPY_FROM_LOCAL_FILE(
|
||||||
"Calls of create()"),
|
StoreStatisticNames.OP_COPY_FROM_LOCAL_FILE,
|
||||||
INVOCATION_CREATE_NON_RECURSIVE(CommonStatisticNames.OP_CREATE_NON_RECURSIVE,
|
"Calls of copyFromLocalFile()",
|
||||||
"Calls of createNonRecursive()"),
|
TYPE_COUNTER),
|
||||||
INVOCATION_DELETE(CommonStatisticNames.OP_DELETE,
|
INVOCATION_CREATE(
|
||||||
"Calls of delete()"),
|
StoreStatisticNames.OP_CREATE,
|
||||||
INVOCATION_EXISTS(CommonStatisticNames.OP_EXISTS,
|
"Calls of create()",
|
||||||
"Calls of exists()"),
|
TYPE_COUNTER),
|
||||||
INVOCATION_GET_DELEGATION_TOKEN(CommonStatisticNames.OP_GET_DELEGATION_TOKEN,
|
INVOCATION_CREATE_NON_RECURSIVE(
|
||||||
"Calls of getDelegationToken()"),
|
StoreStatisticNames.OP_CREATE_NON_RECURSIVE,
|
||||||
INVOCATION_GET_FILE_CHECKSUM(CommonStatisticNames.OP_GET_FILE_CHECKSUM,
|
"Calls of createNonRecursive()",
|
||||||
"Calls of getFileChecksum()"),
|
TYPE_COUNTER),
|
||||||
INVOCATION_GET_FILE_STATUS(CommonStatisticNames.OP_GET_FILE_STATUS,
|
INVOCATION_DELETE(
|
||||||
"Calls of getFileStatus()"),
|
StoreStatisticNames.OP_DELETE,
|
||||||
INVOCATION_GLOB_STATUS(CommonStatisticNames.OP_GLOB_STATUS,
|
"Calls of delete()",
|
||||||
"Calls of globStatus()"),
|
TYPE_COUNTER),
|
||||||
INVOCATION_IS_DIRECTORY(CommonStatisticNames.OP_IS_DIRECTORY,
|
INVOCATION_EXISTS(
|
||||||
"Calls of isDirectory()"),
|
StoreStatisticNames.OP_EXISTS,
|
||||||
INVOCATION_IS_FILE(CommonStatisticNames.OP_IS_FILE,
|
"Calls of exists()",
|
||||||
"Calls of isFile()"),
|
TYPE_COUNTER),
|
||||||
INVOCATION_LIST_FILES(CommonStatisticNames.OP_LIST_FILES,
|
INVOCATION_GET_DELEGATION_TOKEN(
|
||||||
"Calls of listFiles()"),
|
StoreStatisticNames.OP_GET_DELEGATION_TOKEN,
|
||||||
INVOCATION_LIST_LOCATED_STATUS(CommonStatisticNames.OP_LIST_LOCATED_STATUS,
|
"Calls of getDelegationToken()",
|
||||||
"Calls of listLocatedStatus()"),
|
TYPE_COUNTER),
|
||||||
INVOCATION_LIST_STATUS(CommonStatisticNames.OP_LIST_STATUS,
|
INVOCATION_GET_FILE_CHECKSUM(
|
||||||
"Calls of listStatus()"),
|
StoreStatisticNames.OP_GET_FILE_CHECKSUM,
|
||||||
INVOCATION_MKDIRS(CommonStatisticNames.OP_MKDIRS,
|
"Calls of getFileChecksum()",
|
||||||
"Calls of mkdirs()"),
|
TYPE_COUNTER),
|
||||||
INVOCATION_OPEN(CommonStatisticNames.OP_OPEN,
|
INVOCATION_GET_FILE_STATUS(
|
||||||
"Calls of open()"),
|
StoreStatisticNames.OP_GET_FILE_STATUS,
|
||||||
INVOCATION_RENAME(CommonStatisticNames.OP_RENAME,
|
"Calls of getFileStatus()",
|
||||||
"Calls of rename()"),
|
TYPE_COUNTER),
|
||||||
OBJECT_COPY_REQUESTS("object_copy_requests", "Object copy requests"),
|
INVOCATION_GLOB_STATUS(
|
||||||
OBJECT_DELETE_REQUESTS("object_delete_requests", "Object delete requests"),
|
StoreStatisticNames.OP_GLOB_STATUS,
|
||||||
OBJECT_DELETE_OBJECTS("object_delete_objects",
|
"Calls of globStatus()",
|
||||||
"Objects deleted in delete requests"),
|
TYPE_COUNTER),
|
||||||
OBJECT_LIST_REQUESTS("object_list_requests",
|
INVOCATION_IS_DIRECTORY(
|
||||||
"Number of object listings made"),
|
StoreStatisticNames.OP_IS_DIRECTORY,
|
||||||
OBJECT_CONTINUE_LIST_REQUESTS("object_continue_list_requests",
|
"Calls of isDirectory()",
|
||||||
"Number of continued object listings made"),
|
TYPE_COUNTER),
|
||||||
OBJECT_METADATA_REQUESTS("object_metadata_requests",
|
INVOCATION_IS_FILE(
|
||||||
"Number of requests for object metadata"),
|
StoreStatisticNames.OP_IS_FILE,
|
||||||
OBJECT_MULTIPART_UPLOAD_INITIATED("object_multipart_initiated",
|
"Calls of isFile()",
|
||||||
"Object multipart upload initiated"),
|
TYPE_COUNTER),
|
||||||
OBJECT_MULTIPART_UPLOAD_ABORTED("object_multipart_aborted",
|
INVOCATION_LIST_FILES(
|
||||||
"Object multipart upload aborted"),
|
StoreStatisticNames.OP_LIST_FILES,
|
||||||
OBJECT_PUT_REQUESTS("object_put_requests",
|
"Calls of listFiles()",
|
||||||
"Object put/multipart upload count"),
|
TYPE_COUNTER),
|
||||||
OBJECT_PUT_REQUESTS_COMPLETED("object_put_requests_completed",
|
INVOCATION_LIST_LOCATED_STATUS(
|
||||||
"Object put/multipart upload completed count"),
|
StoreStatisticNames.OP_LIST_LOCATED_STATUS,
|
||||||
OBJECT_PUT_REQUESTS_ACTIVE("object_put_requests_active",
|
"Calls of listLocatedStatus()",
|
||||||
"Current number of active put requests"),
|
TYPE_COUNTER),
|
||||||
OBJECT_PUT_BYTES("object_put_bytes", "number of bytes uploaded"),
|
INVOCATION_LIST_STATUS(
|
||||||
OBJECT_PUT_BYTES_PENDING("object_put_bytes_pending",
|
StoreStatisticNames.OP_LIST_STATUS,
|
||||||
"number of bytes queued for upload/being actively uploaded"),
|
"Calls of listStatus()",
|
||||||
OBJECT_SELECT_REQUESTS("object_select_requests",
|
TYPE_COUNTER),
|
||||||
"Count of S3 Select requests issued"),
|
INVOCATION_MKDIRS(
|
||||||
STREAM_ABORTED("stream_aborted",
|
StoreStatisticNames.OP_MKDIRS,
|
||||||
"Count of times the TCP stream was aborted"),
|
"Calls of mkdirs()",
|
||||||
STREAM_BACKWARD_SEEK_OPERATIONS("stream_backward_seek_operations",
|
TYPE_COUNTER),
|
||||||
"Number of executed seek operations which went backwards in a stream"),
|
INVOCATION_OPEN(
|
||||||
STREAM_CLOSED("stream_closed", "Count of times the TCP stream was closed"),
|
StoreStatisticNames.OP_OPEN,
|
||||||
STREAM_CLOSE_OPERATIONS("stream_close_operations",
|
"Calls of open()",
|
||||||
"Total count of times an attempt to close a data stream was made"),
|
TYPE_COUNTER),
|
||||||
STREAM_FORWARD_SEEK_OPERATIONS("stream_forward_seek_operations",
|
INVOCATION_RENAME(
|
||||||
"Number of executed seek operations which went forward in a stream"),
|
StoreStatisticNames.OP_RENAME,
|
||||||
STREAM_OPENED("stream_opened",
|
"Calls of rename()",
|
||||||
"Total count of times an input stream to object store was opened"),
|
TYPE_COUNTER),
|
||||||
STREAM_READ_EXCEPTIONS("stream_read_exceptions",
|
|
||||||
"Number of exceptions invoked on input streams"),
|
|
||||||
STREAM_READ_FULLY_OPERATIONS("stream_read_fully_operations",
|
|
||||||
"Count of readFully() operations in streams"),
|
|
||||||
STREAM_READ_OPERATIONS("stream_read_operations",
|
|
||||||
"Count of read() operations in streams"),
|
|
||||||
STREAM_READ_OPERATIONS_INCOMPLETE("stream_read_operations_incomplete",
|
|
||||||
"Count of incomplete read() operations in streams"),
|
|
||||||
STREAM_READ_VERSION_MISMATCHES("stream_read_version_mismatches",
|
|
||||||
"Count of version mismatches encountered while reading streams"),
|
|
||||||
STREAM_SEEK_BYTES_BACKWARDS("stream_bytes_backwards_on_seek",
|
|
||||||
"Count of bytes moved backwards during seek operations"),
|
|
||||||
STREAM_SEEK_BYTES_READ("stream_bytes_read",
|
|
||||||
"Count of bytes read during seek() in stream operations"),
|
|
||||||
STREAM_SEEK_BYTES_SKIPPED("stream_bytes_skipped_on_seek",
|
|
||||||
"Count of bytes skipped during forward seek operation"),
|
|
||||||
STREAM_SEEK_OPERATIONS("stream_seek_operations",
|
|
||||||
"Number of seek operations during stream IO."),
|
|
||||||
STREAM_CLOSE_BYTES_READ("stream_bytes_read_in_close",
|
|
||||||
"Count of bytes read when closing streams during seek operations."),
|
|
||||||
STREAM_ABORT_BYTES_DISCARDED("stream_bytes_discarded_in_abort",
|
|
||||||
"Count of bytes discarded by aborting the stream"),
|
|
||||||
STREAM_WRITE_FAILURES("stream_write_failures",
|
|
||||||
"Count of stream write failures reported"),
|
|
||||||
STREAM_WRITE_BLOCK_UPLOADS("stream_write_block_uploads",
|
|
||||||
"Count of block/partition uploads completed"),
|
|
||||||
STREAM_WRITE_BLOCK_UPLOADS_ACTIVE("stream_write_block_uploads_active",
|
|
||||||
"Count of block/partition uploads completed"),
|
|
||||||
STREAM_WRITE_BLOCK_UPLOADS_COMMITTED("stream_write_block_uploads_committed",
|
|
||||||
"Count of number of block uploads committed"),
|
|
||||||
STREAM_WRITE_BLOCK_UPLOADS_ABORTED("stream_write_block_uploads_aborted",
|
|
||||||
"Count of number of block uploads aborted"),
|
|
||||||
|
|
||||||
STREAM_WRITE_BLOCK_UPLOADS_PENDING("stream_write_block_uploads_pending",
|
/* Object IO */
|
||||||
"Gauge of block/partitions uploads queued to be written"),
|
OBJECT_COPY_REQUESTS(StoreStatisticNames.OBJECT_COPY_REQUESTS,
|
||||||
STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING(
|
"Object copy requests",
|
||||||
"stream_write_block_uploads_data_pending",
|
TYPE_COUNTER),
|
||||||
"Gauge of block/partitions data uploads queued to be written"),
|
OBJECT_DELETE_REQUEST(StoreStatisticNames.OBJECT_DELETE_REQUEST,
|
||||||
STREAM_WRITE_TOTAL_TIME("stream_write_total_time",
|
"Object delete requests",
|
||||||
"Count of total time taken for uploads to complete"),
|
TYPE_DURATION),
|
||||||
STREAM_WRITE_TOTAL_DATA("stream_write_total_data",
|
OBJECT_BULK_DELETE_REQUEST(StoreStatisticNames.OBJECT_BULK_DELETE_REQUEST,
|
||||||
"Count of total data uploaded in block output"),
|
"Object bulk delete requests",
|
||||||
STREAM_WRITE_QUEUE_DURATION("stream_write_queue_duration",
|
TYPE_DURATION),
|
||||||
"Total queue duration of all block uploads"),
|
OBJECT_DELETE_OBJECTS(StoreStatisticNames.OBJECT_DELETE_OBJECTS,
|
||||||
|
"Objects deleted in delete requests",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
OBJECT_LIST_REQUEST(StoreStatisticNames.OBJECT_LIST_REQUEST,
|
||||||
|
"Count of object listings made",
|
||||||
|
TYPE_DURATION),
|
||||||
|
OBJECT_CONTINUE_LIST_REQUESTS(
|
||||||
|
StoreStatisticNames.OBJECT_CONTINUE_LIST_REQUEST,
|
||||||
|
"Count of continued object listings made",
|
||||||
|
TYPE_DURATION),
|
||||||
|
OBJECT_METADATA_REQUESTS(
|
||||||
|
StoreStatisticNames.OBJECT_METADATA_REQUESTS,
|
||||||
|
"Count of requests for object metadata",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
OBJECT_MULTIPART_UPLOAD_INITIATED(
|
||||||
|
StoreStatisticNames.OBJECT_MULTIPART_UPLOAD_INITIATED,
|
||||||
|
"Object multipart upload initiated",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
OBJECT_MULTIPART_UPLOAD_ABORTED(
|
||||||
|
StoreStatisticNames.OBJECT_MULTIPART_UPLOAD_ABORTED,
|
||||||
|
"Object multipart upload aborted",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
OBJECT_PUT_REQUESTS(
|
||||||
|
StoreStatisticNames.OBJECT_PUT_REQUEST,
|
||||||
|
"Object put/multipart upload count",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
OBJECT_PUT_REQUESTS_COMPLETED(
|
||||||
|
StoreStatisticNames.OBJECT_PUT_REQUEST_COMPLETED,
|
||||||
|
"Object put/multipart upload completed count",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
OBJECT_PUT_REQUESTS_ACTIVE(
|
||||||
|
StoreStatisticNames.OBJECT_PUT_REQUEST_ACTIVE,
|
||||||
|
"Current number of active put requests",
|
||||||
|
TYPE_GAUGE),
|
||||||
|
OBJECT_PUT_BYTES(
|
||||||
|
StoreStatisticNames.OBJECT_PUT_BYTES,
|
||||||
|
"number of bytes uploaded",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
OBJECT_PUT_BYTES_PENDING(
|
||||||
|
StoreStatisticNames.OBJECT_PUT_BYTES_PENDING,
|
||||||
|
"number of bytes queued for upload/being actively uploaded",
|
||||||
|
TYPE_GAUGE),
|
||||||
|
OBJECT_SELECT_REQUESTS(
|
||||||
|
StoreStatisticNames.OBJECT_SELECT_REQUESTS,
|
||||||
|
"Count of S3 Select requests issued",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
STREAM_READ_ABORTED(
|
||||||
|
StreamStatisticNames.STREAM_READ_ABORTED,
|
||||||
|
"Count of times the TCP stream was aborted",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
|
||||||
// S3guard committer stats
|
/* Stream Reads */
|
||||||
|
STREAM_READ_BYTES(
|
||||||
|
StreamStatisticNames.STREAM_READ_BYTES,
|
||||||
|
"Bytes read from an input stream in read() calls",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
STREAM_READ_BYTES_DISCARDED_ABORT(
|
||||||
|
StreamStatisticNames.STREAM_READ_BYTES_DISCARDED_ABORT,
|
||||||
|
"Count of bytes discarded by aborting an input stream",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
STREAM_READ_BYTES_READ_CLOSE(
|
||||||
|
StreamStatisticNames.STREAM_READ_BYTES_DISCARDED_CLOSE,
|
||||||
|
"Count of bytes read and discarded when closing an input stream",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
STREAM_READ_CLOSED(
|
||||||
|
StreamStatisticNames.STREAM_READ_CLOSED,
|
||||||
|
"Count of times the TCP stream was closed",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
STREAM_READ_CLOSE_OPERATIONS(
|
||||||
|
StreamStatisticNames.STREAM_READ_CLOSE_OPERATIONS,
|
||||||
|
"Total count of times an attempt to close an input stream was made",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
STREAM_READ_EXCEPTIONS(
|
||||||
|
StreamStatisticNames.STREAM_READ_EXCEPTIONS,
|
||||||
|
"Count of exceptions raised during input stream reads",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
STREAM_READ_FULLY_OPERATIONS(
|
||||||
|
StreamStatisticNames.STREAM_READ_FULLY_OPERATIONS,
|
||||||
|
"Count of readFully() operations in an input stream",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
STREAM_READ_OPENED(
|
||||||
|
StreamStatisticNames.STREAM_READ_OPENED,
|
||||||
|
"Total count of times an input stream to object store data was opened",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
STREAM_READ_OPERATIONS(
|
||||||
|
StreamStatisticNames.STREAM_READ_OPERATIONS,
|
||||||
|
"Count of read() operations in an input stream",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
STREAM_READ_OPERATIONS_INCOMPLETE(
|
||||||
|
StreamStatisticNames.STREAM_READ_OPERATIONS_INCOMPLETE,
|
||||||
|
"Count of incomplete read() operations in an input stream",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
STREAM_READ_VERSION_MISMATCHES(
|
||||||
|
StreamStatisticNames.STREAM_READ_VERSION_MISMATCHES,
|
||||||
|
"Count of version mismatches encountered while reading an input stream",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
STREAM_READ_SEEK_BACKWARD_OPERATIONS(
|
||||||
|
StreamStatisticNames.STREAM_READ_SEEK_BACKWARD_OPERATIONS,
|
||||||
|
"Count of executed seek operations which went backwards in a stream",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
STREAM_READ_SEEK_BYTES_BACKWARDS(
|
||||||
|
StreamStatisticNames.STREAM_READ_SEEK_BYTES_BACKWARDS,
|
||||||
|
"Count of bytes moved backwards during seek operations"
|
||||||
|
+ " in an input stream",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
STREAM_READ_SEEK_BYTES_DISCARDED(
|
||||||
|
StreamStatisticNames.STREAM_READ_SEEK_BYTES_DISCARDED,
|
||||||
|
"Count of bytes read and discarded during seek() in an input stream",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
STREAM_READ_SEEK_BYTES_SKIPPED(
|
||||||
|
StreamStatisticNames.STREAM_READ_SEEK_BYTES_SKIPPED,
|
||||||
|
"Count of bytes skipped during forward seek operations"
|
||||||
|
+ " an input stream",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
STREAM_READ_SEEK_FORWARD_OPERATIONS(
|
||||||
|
StreamStatisticNames.STREAM_READ_SEEK_FORWARD_OPERATIONS,
|
||||||
|
"Count of executed seek operations which went forward in"
|
||||||
|
+ " an input stream",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
STREAM_READ_SEEK_OPERATIONS(
|
||||||
|
StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS,
|
||||||
|
"Count of seek operations in an input stream",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
STREAM_READ_SEEK_POLICY_CHANGED(
|
||||||
|
StreamStatisticNames.STREAM_READ_SEEK_POLICY_CHANGED,
|
||||||
|
"Count of times the seek policy was dynamically changed"
|
||||||
|
+ " in an input stream",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
STREAM_READ_TOTAL_BYTES(
|
||||||
|
StreamStatisticNames.STREAM_READ_TOTAL_BYTES,
|
||||||
|
"Total count of bytes read from an input stream",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
|
||||||
|
/* Stream Write statistics */
|
||||||
|
|
||||||
|
STREAM_WRITE_EXCEPTIONS(
|
||||||
|
StreamStatisticNames.STREAM_WRITE_EXCEPTIONS,
|
||||||
|
"Count of stream write failures reported",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
STREAM_WRITE_EXCEPTIONS_COMPLETING_UPLOADS(
|
||||||
|
StreamStatisticNames.STREAM_WRITE_EXCEPTIONS_COMPLETING_UPLOADS,
|
||||||
|
"Count of failures when finalizing a multipart upload",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
STREAM_WRITE_BLOCK_UPLOADS(
|
||||||
|
StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS,
|
||||||
|
"Count of block/partition uploads completed",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
STREAM_WRITE_BLOCK_UPLOADS_ACTIVE(
|
||||||
|
StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS_ACTIVE,
|
||||||
|
"Count of block/partition uploads active",
|
||||||
|
TYPE_GAUGE),
|
||||||
|
STREAM_WRITE_BLOCK_UPLOADS_COMMITTED(
|
||||||
|
StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS_COMMITTED,
|
||||||
|
"Count of number of block uploads committed",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
STREAM_WRITE_BLOCK_UPLOADS_ABORTED(
|
||||||
|
StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS_ABORTED,
|
||||||
|
"Count of number of block uploads aborted",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
|
||||||
|
STREAM_WRITE_BLOCK_UPLOADS_PENDING(
|
||||||
|
StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS_PENDING,
|
||||||
|
"Gauge of block/partitions uploads queued to be written",
|
||||||
|
TYPE_GAUGE),
|
||||||
|
STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING(
|
||||||
|
StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING,
|
||||||
|
"Gauge of data queued to be written",
|
||||||
|
TYPE_GAUGE),
|
||||||
|
STREAM_WRITE_TOTAL_TIME(
|
||||||
|
StreamStatisticNames.STREAM_WRITE_TOTAL_TIME,
|
||||||
|
"Count of total time taken for uploads to complete",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
STREAM_WRITE_TOTAL_DATA(StreamStatisticNames.STREAM_WRITE_TOTAL_DATA,
|
||||||
|
"Count of total data uploaded",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
STREAM_WRITE_BYTES(
|
||||||
|
StreamStatisticNames.STREAM_WRITE_BYTES,
|
||||||
|
"Count of bytes written to output stream"
|
||||||
|
+ " (including all not yet uploaded)",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
STREAM_WRITE_QUEUE_DURATION(
|
||||||
|
StreamStatisticNames.STREAM_WRITE_QUEUE_DURATION,
|
||||||
|
"Total queue duration of all block uploads",
|
||||||
|
TYPE_DURATION),
|
||||||
|
|
||||||
|
/* committer stats */
|
||||||
COMMITTER_COMMITS_CREATED(
|
COMMITTER_COMMITS_CREATED(
|
||||||
"committer_commits_created",
|
"committer_commits_created",
|
||||||
"Number of files to commit created"),
|
"Count of files to commit created",
|
||||||
|
TYPE_COUNTER),
|
||||||
COMMITTER_COMMITS_COMPLETED(
|
COMMITTER_COMMITS_COMPLETED(
|
||||||
"committer_commits_completed",
|
"committer_commits_completed",
|
||||||
"Number of files committed"),
|
"Count of files committed",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
COMMITTER_COMMIT_JOB(
|
||||||
|
"committer_commit_job",
|
||||||
|
"Duration Tracking of time to commit an entire job",
|
||||||
|
TYPE_DURATION),
|
||||||
COMMITTER_JOBS_SUCCEEDED(
|
COMMITTER_JOBS_SUCCEEDED(
|
||||||
"committer_jobs_completed",
|
"committer_jobs_completed",
|
||||||
"Number of successful jobs"),
|
"Count of successful jobs",
|
||||||
|
TYPE_COUNTER),
|
||||||
COMMITTER_JOBS_FAILED(
|
COMMITTER_JOBS_FAILED(
|
||||||
"committer_jobs_failed",
|
"committer_jobs_failed",
|
||||||
"Number of failed jobs"),
|
"Count of failed jobs",
|
||||||
|
TYPE_COUNTER),
|
||||||
COMMITTER_TASKS_SUCCEEDED(
|
COMMITTER_TASKS_SUCCEEDED(
|
||||||
"committer_tasks_completed",
|
"committer_tasks_completed",
|
||||||
"Number of successful tasks"),
|
"Count of successful tasks",
|
||||||
|
TYPE_COUNTER),
|
||||||
COMMITTER_TASKS_FAILED(
|
COMMITTER_TASKS_FAILED(
|
||||||
"committer_tasks_failed",
|
"committer_tasks_failed",
|
||||||
"Number of failed tasks"),
|
"Count of failed tasks",
|
||||||
|
TYPE_COUNTER),
|
||||||
COMMITTER_BYTES_COMMITTED(
|
COMMITTER_BYTES_COMMITTED(
|
||||||
"committer_bytes_committed",
|
"committer_bytes_committed",
|
||||||
"Amount of data committed"),
|
"Amount of data committed",
|
||||||
|
TYPE_COUNTER),
|
||||||
COMMITTER_BYTES_UPLOADED(
|
COMMITTER_BYTES_UPLOADED(
|
||||||
"committer_bytes_uploaded",
|
"committer_bytes_uploaded",
|
||||||
"Number of bytes uploaded duing commit operations"),
|
"Count of bytes uploaded duing commit operations",
|
||||||
|
TYPE_COUNTER),
|
||||||
COMMITTER_COMMITS_FAILED(
|
COMMITTER_COMMITS_FAILED(
|
||||||
"committer_commits_failed",
|
"committer_commits"+ StoreStatisticNames.SUFFIX_FAILURES,
|
||||||
"Number of commits failed"),
|
"Count of commits failed",
|
||||||
|
TYPE_COUNTER),
|
||||||
COMMITTER_COMMITS_ABORTED(
|
COMMITTER_COMMITS_ABORTED(
|
||||||
"committer_commits_aborted",
|
"committer_commits_aborted",
|
||||||
"Number of commits aborted"),
|
"Count of commits aborted",
|
||||||
|
TYPE_COUNTER),
|
||||||
COMMITTER_COMMITS_REVERTED(
|
COMMITTER_COMMITS_REVERTED(
|
||||||
"committer_commits_reverted",
|
"committer_commits_reverted",
|
||||||
"Number of commits reverted"),
|
"Count of commits reverted",
|
||||||
|
TYPE_COUNTER),
|
||||||
COMMITTER_MAGIC_FILES_CREATED(
|
COMMITTER_MAGIC_FILES_CREATED(
|
||||||
"committer_magic_files_created",
|
"committer_magic_files_created",
|
||||||
"Number of files created under 'magic' paths"),
|
"Count of files created under 'magic' paths",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
COMMITTER_MATERIALIZE_FILE(
|
||||||
|
"committer_materialize_file",
|
||||||
|
"Duration Tracking of time to materialize a file in job commit",
|
||||||
|
TYPE_DURATION),
|
||||||
|
COMMITTER_STAGE_FILE_UPLOAD(
|
||||||
|
"committer_stage_file_upload",
|
||||||
|
"Duration Tracking of files uploaded from a local staging path",
|
||||||
|
TYPE_DURATION),
|
||||||
|
|
||||||
// S3guard stats
|
/* S3guard stats */
|
||||||
S3GUARD_METADATASTORE_PUT_PATH_REQUEST(
|
S3GUARD_METADATASTORE_PUT_PATH_REQUEST(
|
||||||
"s3guard_metadatastore_put_path_request",
|
"s3guard_metadatastore_put_path_request",
|
||||||
"S3Guard metadata store put one metadata path request"),
|
"S3Guard metadata store put one metadata path request",
|
||||||
|
TYPE_COUNTER),
|
||||||
S3GUARD_METADATASTORE_PUT_PATH_LATENCY(
|
S3GUARD_METADATASTORE_PUT_PATH_LATENCY(
|
||||||
"s3guard_metadatastore_put_path_latency",
|
"s3guard_metadatastore_put_path_latency",
|
||||||
"S3Guard metadata store put one metadata path latency"),
|
"S3Guard metadata store put one metadata path latency",
|
||||||
S3GUARD_METADATASTORE_INITIALIZATION("s3guard_metadatastore_initialization",
|
TYPE_QUANTILE),
|
||||||
"S3Guard metadata store initialization times"),
|
S3GUARD_METADATASTORE_INITIALIZATION(
|
||||||
|
"s3guard_metadatastore_initialization",
|
||||||
|
"S3Guard metadata store initialization times",
|
||||||
|
TYPE_COUNTER),
|
||||||
S3GUARD_METADATASTORE_RECORD_DELETES(
|
S3GUARD_METADATASTORE_RECORD_DELETES(
|
||||||
"s3guard_metadatastore_record_deletes",
|
"s3guard_metadatastore_record_deletes",
|
||||||
"S3Guard metadata store records deleted"),
|
"S3Guard metadata store records deleted",
|
||||||
|
TYPE_COUNTER),
|
||||||
S3GUARD_METADATASTORE_RECORD_READS(
|
S3GUARD_METADATASTORE_RECORD_READS(
|
||||||
"s3guard_metadatastore_record_reads",
|
"s3guard_metadatastore_record_reads",
|
||||||
"S3Guard metadata store records read"),
|
"S3Guard metadata store records read",
|
||||||
|
TYPE_COUNTER),
|
||||||
S3GUARD_METADATASTORE_RECORD_WRITES(
|
S3GUARD_METADATASTORE_RECORD_WRITES(
|
||||||
"s3guard_metadatastore_record_writes",
|
"s3guard_metadatastore_record_writes",
|
||||||
"S3Guard metadata store records written"),
|
"S3Guard metadata store records written",
|
||||||
|
TYPE_COUNTER),
|
||||||
S3GUARD_METADATASTORE_RETRY("s3guard_metadatastore_retry",
|
S3GUARD_METADATASTORE_RETRY("s3guard_metadatastore_retry",
|
||||||
"S3Guard metadata store retry events"),
|
"S3Guard metadata store retry events",
|
||||||
|
TYPE_COUNTER),
|
||||||
S3GUARD_METADATASTORE_THROTTLED("s3guard_metadatastore_throttled",
|
S3GUARD_METADATASTORE_THROTTLED("s3guard_metadatastore_throttled",
|
||||||
"S3Guard metadata store throttled events"),
|
"S3Guard metadata store throttled events",
|
||||||
|
TYPE_COUNTER),
|
||||||
S3GUARD_METADATASTORE_THROTTLE_RATE(
|
S3GUARD_METADATASTORE_THROTTLE_RATE(
|
||||||
"s3guard_metadatastore_throttle_rate",
|
"s3guard_metadatastore_throttle_rate",
|
||||||
"S3Guard metadata store throttle rate"),
|
"S3Guard metadata store throttle rate",
|
||||||
|
TYPE_QUANTILE),
|
||||||
S3GUARD_METADATASTORE_AUTHORITATIVE_DIRECTORIES_UPDATED(
|
S3GUARD_METADATASTORE_AUTHORITATIVE_DIRECTORIES_UPDATED(
|
||||||
"s3guard_metadatastore_authoritative_directories_updated",
|
"s3guard_metadatastore_authoritative_directories_updated",
|
||||||
"S3Guard metadata store authoritative directories updated from S3"),
|
"S3Guard metadata store authoritative directories updated from S3",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
|
||||||
STORE_IO_THROTTLED("store_io_throttled", "Requests throttled and retried"),
|
|
||||||
STORE_IO_THROTTLE_RATE("store_io_throttle_rate",
|
|
||||||
"Rate of S3 request throttling"),
|
|
||||||
|
|
||||||
DELEGATION_TOKENS_ISSUED("delegation_tokens_issued",
|
/* General Store operations */
|
||||||
"Number of delegation tokens issued"),
|
STORE_IO_REQUEST(StoreStatisticNames.STORE_IO_REQUEST,
|
||||||
|
"requests made of the remote store",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
|
||||||
MULTIPART_INSTANTIATED(
|
STORE_IO_RETRY(StoreStatisticNames.STORE_IO_RETRY,
|
||||||
"multipart_instantiated",
|
"retried requests made of the remote store",
|
||||||
"Multipart Uploader Instantiated"),
|
TYPE_COUNTER),
|
||||||
MULTIPART_PART_PUT(
|
|
||||||
"multipart_part_put",
|
STORE_IO_THROTTLED(
|
||||||
"Multipart Part Put Operation"),
|
StoreStatisticNames.STORE_IO_THROTTLED,
|
||||||
MULTIPART_PART_PUT_BYTES(
|
"Requests throttled and retried",
|
||||||
"multipart_part_put_bytes",
|
TYPE_COUNTER),
|
||||||
"Multipart Part Put Bytes"),
|
STORE_IO_THROTTLE_RATE(
|
||||||
|
StoreStatisticNames.STORE_IO_THROTTLE_RATE,
|
||||||
|
"Rate of S3 request throttling",
|
||||||
|
TYPE_QUANTILE),
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Delegation Token Operations.
|
||||||
|
*/
|
||||||
|
DELEGATION_TOKEN_ISSUED(
|
||||||
|
StoreStatisticNames.DELEGATION_TOKEN_ISSUED,
|
||||||
|
"Count of delegation tokens issued",
|
||||||
|
TYPE_DURATION),
|
||||||
|
|
||||||
|
/* Multipart Upload API */
|
||||||
|
|
||||||
|
MULTIPART_UPLOAD_INSTANTIATED(
|
||||||
|
StoreStatisticNames.MULTIPART_UPLOAD_INSTANTIATED,
|
||||||
|
"Multipart Uploader Instantiated",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
MULTIPART_UPLOAD_PART_PUT(
|
||||||
|
StoreStatisticNames.MULTIPART_UPLOAD_PART_PUT,
|
||||||
|
"Multipart Part Put Operation",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
MULTIPART_UPLOAD_PART_PUT_BYTES(
|
||||||
|
StoreStatisticNames.MULTIPART_UPLOAD_PART_PUT_BYTES,
|
||||||
|
"Multipart Part Put Bytes",
|
||||||
|
TYPE_COUNTER),
|
||||||
MULTIPART_UPLOAD_ABORTED(
|
MULTIPART_UPLOAD_ABORTED(
|
||||||
"multipart_upload_aborted",
|
StoreStatisticNames.MULTIPART_UPLOAD_ABORTED,
|
||||||
"Multipart Upload Aborted"),
|
"Multipart Upload Aborted",
|
||||||
|
TYPE_COUNTER),
|
||||||
MULTIPART_UPLOAD_ABORT_UNDER_PATH_INVOKED(
|
MULTIPART_UPLOAD_ABORT_UNDER_PATH_INVOKED(
|
||||||
"multipart_upload_abort_under_path_invoked",
|
StoreStatisticNames.MULTIPART_UPLOAD_ABORT_UNDER_PATH_INVOKED,
|
||||||
"Multipart Upload Abort Udner Path Invoked"),
|
"Multipart Upload Abort Unner Path Invoked",
|
||||||
|
TYPE_COUNTER),
|
||||||
MULTIPART_UPLOAD_COMPLETED(
|
MULTIPART_UPLOAD_COMPLETED(
|
||||||
"multipart_upload_completed",
|
StoreStatisticNames.MULTIPART_UPLOAD_COMPLETED,
|
||||||
"Multipart Upload Completed"),
|
"Multipart Upload Completed",
|
||||||
|
TYPE_COUNTER),
|
||||||
MULTIPART_UPLOAD_STARTED(
|
MULTIPART_UPLOAD_STARTED(
|
||||||
"multipart_upload_started",
|
StoreStatisticNames.MULTIPART_UPLOAD_STARTED,
|
||||||
"Multipart Upload Started");
|
"Multipart Upload Started",
|
||||||
|
TYPE_COUNTER);
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A map used to support the {@link #fromSymbol(String)} call.
|
||||||
|
*/
|
||||||
private static final Map<String, Statistic> SYMBOL_MAP =
|
private static final Map<String, Statistic> SYMBOL_MAP =
|
||||||
new HashMap<>(Statistic.values().length);
|
new HashMap<>(Statistic.values().length);
|
||||||
static {
|
static {
|
||||||
|
@ -268,14 +524,28 @@ public enum Statistic {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
Statistic(String symbol, String description) {
|
|
||||||
|
/**
|
||||||
|
* Statistic definition.
|
||||||
|
* @param symbol name
|
||||||
|
* @param description description.
|
||||||
|
* @param type type
|
||||||
|
*/
|
||||||
|
Statistic(String symbol, String description, StatisticTypeEnum type) {
|
||||||
this.symbol = symbol;
|
this.symbol = symbol;
|
||||||
this.description = description;
|
this.description = description;
|
||||||
|
this.type = type;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/** Statistic name. */
|
||||||
private final String symbol;
|
private final String symbol;
|
||||||
|
|
||||||
|
/** Statistic description. */
|
||||||
private final String description;
|
private final String description;
|
||||||
|
|
||||||
|
/** Statistic type. */
|
||||||
|
private final StatisticTypeEnum type;
|
||||||
|
|
||||||
public String getSymbol() {
|
public String getSymbol() {
|
||||||
return symbol;
|
return symbol;
|
||||||
}
|
}
|
||||||
|
@ -302,4 +572,12 @@ public enum Statistic {
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return symbol;
|
return symbol;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* What type is this statistic?
|
||||||
|
* @return the type.
|
||||||
|
*/
|
||||||
|
public StatisticTypeEnum getType() {
|
||||||
|
return type;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -50,10 +50,12 @@ import org.apache.hadoop.classification.InterfaceStability;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.fs.PathIOException;
|
import org.apache.hadoop.fs.PathIOException;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
|
||||||
import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
|
import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
|
||||||
import org.apache.hadoop.fs.s3a.s3guard.S3Guard;
|
import org.apache.hadoop.fs.s3a.s3guard.S3Guard;
|
||||||
import org.apache.hadoop.fs.s3a.select.SelectBinding;
|
import org.apache.hadoop.fs.s3a.select.SelectBinding;
|
||||||
import org.apache.hadoop.util.DurationInfo;
|
import org.apache.hadoop.util.DurationInfo;
|
||||||
|
import org.apache.hadoop.util.functional.CallableRaisingIOE;
|
||||||
|
|
||||||
import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkArgument;
|
import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkArgument;
|
||||||
import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull;
|
import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull;
|
||||||
|
@ -108,17 +110,26 @@ public class WriteOperationHelper implements WriteOperations {
|
||||||
/** Bucket of the owner FS. */
|
/** Bucket of the owner FS. */
|
||||||
private final String bucket;
|
private final String bucket;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* statistics context.
|
||||||
|
*/
|
||||||
|
private final S3AStatisticsContext statisticsContext;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Constructor.
|
* Constructor.
|
||||||
* @param owner owner FS creating the helper
|
* @param owner owner FS creating the helper
|
||||||
* @param conf Configuration object
|
* @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.owner = owner;
|
||||||
this.invoker = new Invoker(new S3ARetryPolicy(conf),
|
this.invoker = new Invoker(new S3ARetryPolicy(conf),
|
||||||
this::operationRetried);
|
this::operationRetried);
|
||||||
this.conf = conf;
|
this.conf = conf;
|
||||||
|
this.statisticsContext = statisticsContext;
|
||||||
bucket = owner.getBucket();
|
bucket = owner.getBucket();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -138,19 +149,19 @@ public class WriteOperationHelper implements WriteOperations {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Execute a function with retry processing.
|
* Execute a function with retry processing.
|
||||||
|
* @param <T> type of return value
|
||||||
* @param action action to execute (used in error messages)
|
* @param action action to execute (used in error messages)
|
||||||
* @param path path of work (used in error messages)
|
* @param path path of work (used in error messages)
|
||||||
* @param idempotent does the operation have semantics
|
* @param idempotent does the operation have semantics
|
||||||
* which mean that it can be retried even if was already executed?
|
* which mean that it can be retried even if was already executed?
|
||||||
* @param operation operation to execute
|
* @param operation operation to execute
|
||||||
* @param <T> type of return value
|
|
||||||
* @return the result of the call
|
* @return the result of the call
|
||||||
* @throws IOException any IOE raised, or translated exception
|
* @throws IOException any IOE raised, or translated exception
|
||||||
*/
|
*/
|
||||||
public <T> T retry(String action,
|
public <T> T retry(String action,
|
||||||
String path,
|
String path,
|
||||||
boolean idempotent,
|
boolean idempotent,
|
||||||
Invoker.Operation<T> operation)
|
CallableRaisingIOE<T> operation)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
|
|
||||||
return invoker.retry(action, path, idempotent, operation);
|
return invoker.retry(action, path, idempotent, operation);
|
||||||
|
|
|
@ -42,6 +42,7 @@ import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.fs.PathIOException;
|
import org.apache.hadoop.fs.PathIOException;
|
||||||
import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
|
import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
|
||||||
|
import org.apache.hadoop.util.functional.CallableRaisingIOE;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Operations to update the store.
|
* Operations to update the store.
|
||||||
|
@ -56,19 +57,19 @@ public interface WriteOperations {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Execute a function with retry processing.
|
* Execute a function with retry processing.
|
||||||
|
* @param <T> type of return value
|
||||||
* @param action action to execute (used in error messages)
|
* @param action action to execute (used in error messages)
|
||||||
* @param path path of work (used in error messages)
|
* @param path path of work (used in error messages)
|
||||||
* @param idempotent does the operation have semantics
|
* @param idempotent does the operation have semantics
|
||||||
* which mean that it can be retried even if was already executed?
|
* which mean that it can be retried even if was already executed?
|
||||||
* @param operation operation to execute
|
* @param operation operation to execute
|
||||||
* @param <T> type of return value
|
|
||||||
* @return the result of the call
|
* @return the result of the call
|
||||||
* @throws IOException any IOE raised, or translated exception
|
* @throws IOException any IOE raised, or translated exception
|
||||||
*/
|
*/
|
||||||
<T> T retry(String action,
|
<T> T retry(String action,
|
||||||
String path,
|
String path,
|
||||||
boolean idempotent,
|
boolean idempotent,
|
||||||
Invoker.Operation<T> operation)
|
CallableRaisingIOE<T> operation)
|
||||||
throws IOException;
|
throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -34,9 +34,9 @@ import org.apache.commons.lang3.StringUtils;
|
||||||
import org.apache.hadoop.classification.InterfaceAudience;
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.s3a.AWSCredentialProviderList;
|
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.auth.RoleModel;
|
||||||
import org.apache.hadoop.fs.s3a.impl.StoreContext;
|
import org.apache.hadoop.fs.s3a.impl.StoreContext;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.DelegationTokenStatistics;
|
||||||
import org.apache.hadoop.io.Text;
|
import org.apache.hadoop.io.Text;
|
||||||
import org.apache.hadoop.security.Credentials;
|
import org.apache.hadoop.security.Credentials;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
|
@ -45,6 +45,8 @@ import org.apache.hadoop.security.token.Token;
|
||||||
import org.apache.hadoop.service.ServiceOperations;
|
import org.apache.hadoop.service.ServiceOperations;
|
||||||
import org.apache.hadoop.util.DurationInfo;
|
import org.apache.hadoop.util.DurationInfo;
|
||||||
|
|
||||||
|
import static org.apache.hadoop.fs.s3a.Statistic.DELEGATION_TOKEN_ISSUED;
|
||||||
|
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration;
|
||||||
import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkArgument;
|
import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkArgument;
|
||||||
import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkState;
|
import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkState;
|
||||||
import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.DEFAULT_DELEGATION_TOKEN_BINDING;
|
import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.DEFAULT_DELEGATION_TOKEN_BINDING;
|
||||||
|
@ -135,9 +137,9 @@ public class S3ADelegationTokens extends AbstractDTService {
|
||||||
AWSPolicyProvider.AccessLevel.WRITE);
|
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
|
* Name of the token binding as extracted from token kind; used for
|
||||||
|
@ -428,8 +430,10 @@ public class S3ADelegationTokens extends AbstractDTService {
|
||||||
|
|
||||||
try(DurationInfo ignored = new DurationInfo(LOG, DURATION_LOG_AT_INFO,
|
try(DurationInfo ignored = new DurationInfo(LOG, DURATION_LOG_AT_INFO,
|
||||||
"Creating New Delegation Token", tokenBinding.getKind())) {
|
"Creating New Delegation Token", tokenBinding.getKind())) {
|
||||||
Token<AbstractS3ATokenIdentifier> token
|
Token<AbstractS3ATokenIdentifier> token = trackDuration(stats,
|
||||||
= tokenBinding.createDelegationToken(rolePolicy, encryptionSecrets, renewer);
|
DELEGATION_TOKEN_ISSUED.getSymbol(), () ->
|
||||||
|
tokenBinding.createDelegationToken(rolePolicy,
|
||||||
|
encryptionSecrets, renewer));
|
||||||
if (token != null) {
|
if (token != null) {
|
||||||
token.setService(service);
|
token.setService(service);
|
||||||
noteTokenCreated(token);
|
noteTokenCreated(token);
|
||||||
|
|
|
@ -47,6 +47,10 @@ import org.apache.hadoop.fs.s3a.S3AFileSystem;
|
||||||
import org.apache.hadoop.fs.s3a.commit.files.PendingSet;
|
import org.apache.hadoop.fs.s3a.commit.files.PendingSet;
|
||||||
import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit;
|
import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit;
|
||||||
import org.apache.hadoop.fs.s3a.commit.files.SuccessData;
|
import org.apache.hadoop.fs.s3a.commit.files.SuccessData;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatistics;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
|
||||||
import org.apache.hadoop.mapreduce.JobContext;
|
import org.apache.hadoop.mapreduce.JobContext;
|
||||||
import org.apache.hadoop.mapreduce.JobID;
|
import org.apache.hadoop.mapreduce.JobID;
|
||||||
import org.apache.hadoop.mapreduce.JobStatus;
|
import org.apache.hadoop.mapreduce.JobStatus;
|
||||||
|
@ -60,6 +64,7 @@ import org.apache.hadoop.util.concurrent.HadoopExecutors;
|
||||||
import static org.apache.hadoop.fs.s3a.Constants.THREAD_POOL_SHUTDOWN_DELAY_SECONDS;
|
import static org.apache.hadoop.fs.s3a.Constants.THREAD_POOL_SHUTDOWN_DELAY_SECONDS;
|
||||||
import static org.apache.hadoop.fs.s3a.Invoker.ignoreIOExceptions;
|
import static org.apache.hadoop.fs.s3a.Invoker.ignoreIOExceptions;
|
||||||
import static org.apache.hadoop.fs.s3a.S3AUtils.*;
|
import static org.apache.hadoop.fs.s3a.S3AUtils.*;
|
||||||
|
import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_COMMIT_JOB;
|
||||||
import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
|
import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
|
||||||
import static org.apache.hadoop.fs.s3a.commit.CommitUtils.*;
|
import static org.apache.hadoop.fs.s3a.commit.CommitUtils.*;
|
||||||
import static org.apache.hadoop.fs.s3a.commit.CommitUtilsWithMR.*;
|
import static org.apache.hadoop.fs.s3a.commit.CommitUtilsWithMR.*;
|
||||||
|
@ -67,6 +72,7 @@ import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.E_NO_SP
|
||||||
import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.FS_S3A_COMMITTER_UUID;
|
import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.FS_S3A_COMMITTER_UUID;
|
||||||
import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.FS_S3A_COMMITTER_UUID_SOURCE;
|
import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.FS_S3A_COMMITTER_UUID_SOURCE;
|
||||||
import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.SPARK_WRITE_UUID;
|
import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.SPARK_WRITE_UUID;
|
||||||
|
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfInvocation;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Abstract base class for S3A committers; allows for any commonality
|
* Abstract base class for S3A committers; allows for any commonality
|
||||||
|
@ -94,7 +100,8 @@ import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.SPARK_W
|
||||||
* created by a few tasks, <i>and</i> the HTTP connection pool in the S3A
|
* created by a few tasks, <i>and</i> the HTTP connection pool in the S3A
|
||||||
* committer was large enough for more all the parallel POST requests.
|
* committer was large enough for more all the parallel POST requests.
|
||||||
*/
|
*/
|
||||||
public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
public abstract class AbstractS3ACommitter extends PathOutputCommitter
|
||||||
|
implements IOStatisticsSource {
|
||||||
|
|
||||||
private static final Logger LOG =
|
private static final Logger LOG =
|
||||||
LoggerFactory.getLogger(AbstractS3ACommitter.class);
|
LoggerFactory.getLogger(AbstractS3ACommitter.class);
|
||||||
|
@ -166,6 +173,8 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
||||||
/** Should a job marker be created? */
|
/** Should a job marker be created? */
|
||||||
private final boolean createJobMarker;
|
private final boolean createJobMarker;
|
||||||
|
|
||||||
|
private final CommitterStatistics committerStatistics;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create a committer.
|
* Create a committer.
|
||||||
* This constructor binds the destination directory and configuration, but
|
* This constructor binds the destination directory and configuration, but
|
||||||
|
@ -197,7 +206,9 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
||||||
this.createJobMarker = context.getConfiguration().getBoolean(
|
this.createJobMarker = context.getConfiguration().getBoolean(
|
||||||
CREATE_SUCCESSFUL_JOB_OUTPUT_DIR_MARKER,
|
CREATE_SUCCESSFUL_JOB_OUTPUT_DIR_MARKER,
|
||||||
DEFAULT_CREATE_SUCCESSFUL_JOB_DIR_MARKER);
|
DEFAULT_CREATE_SUCCESSFUL_JOB_DIR_MARKER);
|
||||||
this.commitOperations = new CommitOperations(fs);
|
// the statistics are shared between this committer and its operations.
|
||||||
|
this.committerStatistics = fs.newCommitterStatistics();
|
||||||
|
this.commitOperations = new CommitOperations(fs, committerStatistics);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -437,7 +448,12 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
||||||
// The list of committed objects in pending is size limited in
|
// The list of committed objects in pending is size limited in
|
||||||
// ActiveCommit.uploadCommitted.
|
// ActiveCommit.uploadCommitted.
|
||||||
filenames.addAll(pending.committedObjects);
|
filenames.addAll(pending.committedObjects);
|
||||||
maybeCreateSuccessMarker(context, filenames);
|
// load in all the pending statistics
|
||||||
|
IOStatisticsSnapshot snapshot = new IOStatisticsSnapshot(
|
||||||
|
pending.getIOStatistics());
|
||||||
|
snapshot.aggregate(getIOStatistics());
|
||||||
|
|
||||||
|
maybeCreateSuccessMarker(context, filenames, snapshot);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -448,10 +464,12 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
||||||
* PUT up a the contents of a {@link SuccessData} file.
|
* PUT up a the contents of a {@link SuccessData} file.
|
||||||
* @param context job context
|
* @param context job context
|
||||||
* @param filenames list of filenames.
|
* @param filenames list of filenames.
|
||||||
|
* @param ioStatistics any IO Statistics to include
|
||||||
* @throws IOException IO failure
|
* @throws IOException IO failure
|
||||||
*/
|
*/
|
||||||
protected void maybeCreateSuccessMarker(JobContext context,
|
protected void maybeCreateSuccessMarker(JobContext context,
|
||||||
List<String> filenames)
|
List<String> filenames,
|
||||||
|
final IOStatisticsSnapshot ioStatistics)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
if (createJobMarker) {
|
if (createJobMarker) {
|
||||||
// create a success data structure and then save it
|
// create a success data structure and then save it
|
||||||
|
@ -465,6 +483,7 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
||||||
successData.setTimestamp(now.getTime());
|
successData.setTimestamp(now.getTime());
|
||||||
successData.setDate(now.toString());
|
successData.setDate(now.toString());
|
||||||
successData.setFilenames(filenames);
|
successData.setFilenames(filenames);
|
||||||
|
successData.getIOStatistics().aggregate(ioStatistics);
|
||||||
commitOperations.createSuccessMarker(getOutputPath(), successData, true);
|
commitOperations.createSuccessMarker(getOutputPath(), successData, true);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -644,6 +663,7 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
||||||
activeCommit.uploadCommitted(
|
activeCommit.uploadCommitted(
|
||||||
commit.getDestinationKey(), commit.getLength());
|
commit.getDestinationKey(), commit.getLength());
|
||||||
});
|
});
|
||||||
|
activeCommit.pendingsetCommitted(pendingSet.getIOStatistics());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -728,8 +748,9 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
||||||
protected void commitJobInternal(JobContext context,
|
protected void commitJobInternal(JobContext context,
|
||||||
ActiveCommit pending)
|
ActiveCommit pending)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
|
trackDurationOfInvocation(committerStatistics,
|
||||||
commitPendingUploads(context, pending);
|
COMMITTER_COMMIT_JOB.getSymbol(),
|
||||||
|
() -> commitPendingUploads(context, pending));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -1175,6 +1196,11 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public IOStatistics getIOStatistics() {
|
||||||
|
return committerStatistics.getIOStatistics();
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Scan for active uploads and list them along with a warning message.
|
* Scan for active uploads and list them along with a warning message.
|
||||||
* Errors are ignored.
|
* Errors are ignored.
|
||||||
|
@ -1385,6 +1411,13 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
||||||
*/
|
*/
|
||||||
private long committedBytes;
|
private long committedBytes;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Aggregate statistics of all supplied by
|
||||||
|
* committed uploads.
|
||||||
|
*/
|
||||||
|
private final IOStatisticsSnapshot ioStatistics =
|
||||||
|
new IOStatisticsSnapshot();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Construct from a source FS and list of files.
|
* Construct from a source FS and list of files.
|
||||||
* @param sourceFS filesystem containing the list of pending files
|
* @param sourceFS filesystem containing the list of pending files
|
||||||
|
@ -1433,7 +1466,8 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
||||||
* @param key key of the committed object.
|
* @param key key of the committed object.
|
||||||
* @param size size in bytes.
|
* @param size size in bytes.
|
||||||
*/
|
*/
|
||||||
public synchronized void uploadCommitted(String key, long size) {
|
public synchronized void uploadCommitted(String key,
|
||||||
|
long size) {
|
||||||
if (committedObjects.size() < SUCCESS_MARKER_FILE_LIMIT) {
|
if (committedObjects.size() < SUCCESS_MARKER_FILE_LIMIT) {
|
||||||
committedObjects.add(
|
committedObjects.add(
|
||||||
key.startsWith("/") ? key : ("/" + key));
|
key.startsWith("/") ? key : ("/" + key));
|
||||||
|
@ -1442,6 +1476,19 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
|
||||||
committedBytes += size;
|
committedBytes += size;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Callback when a pendingset has been committed,
|
||||||
|
* including any source statistics.
|
||||||
|
* @param sourceStatistics any source statistics
|
||||||
|
*/
|
||||||
|
public void pendingsetCommitted(final IOStatistics sourceStatistics) {
|
||||||
|
ioStatistics.aggregate(sourceStatistics);
|
||||||
|
}
|
||||||
|
|
||||||
|
public IOStatisticsSnapshot getIOStatistics() {
|
||||||
|
return ioStatistics;
|
||||||
|
}
|
||||||
|
|
||||||
public synchronized List<String> getCommittedObjects() {
|
public synchronized List<String> getCommittedObjects() {
|
||||||
return committedObjects;
|
return committedObjects;
|
||||||
}
|
}
|
||||||
|
|
|
@ -45,7 +45,6 @@ import org.apache.hadoop.fs.PathFilter;
|
||||||
import org.apache.hadoop.fs.PathIOException;
|
import org.apache.hadoop.fs.PathIOException;
|
||||||
import org.apache.hadoop.fs.RemoteIterator;
|
import org.apache.hadoop.fs.RemoteIterator;
|
||||||
import org.apache.hadoop.fs.s3a.S3AFileSystem;
|
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.S3AUtils;
|
||||||
import org.apache.hadoop.fs.s3a.WriteOperationHelper;
|
import org.apache.hadoop.fs.s3a.WriteOperationHelper;
|
||||||
import org.apache.hadoop.fs.s3a.commit.files.PendingSet;
|
import org.apache.hadoop.fs.s3a.commit.files.PendingSet;
|
||||||
|
@ -53,13 +52,21 @@ import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit;
|
||||||
import org.apache.hadoop.fs.s3a.commit.files.SuccessData;
|
import org.apache.hadoop.fs.s3a.commit.files.SuccessData;
|
||||||
import org.apache.hadoop.fs.s3a.impl.InternalConstants;
|
import org.apache.hadoop.fs.s3a.impl.InternalConstants;
|
||||||
import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
|
import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics;
|
||||||
|
import org.apache.hadoop.fs.statistics.DurationTracker;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatistics;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
|
||||||
import org.apache.hadoop.io.IOUtils;
|
import org.apache.hadoop.io.IOUtils;
|
||||||
import org.apache.hadoop.util.DurationInfo;
|
import org.apache.hadoop.util.DurationInfo;
|
||||||
import org.apache.hadoop.util.Progressable;
|
import org.apache.hadoop.util.Progressable;
|
||||||
|
|
||||||
|
import static java.util.Objects.requireNonNull;
|
||||||
import static org.apache.hadoop.fs.s3a.S3AUtils.*;
|
import static org.apache.hadoop.fs.s3a.S3AUtils.*;
|
||||||
|
import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_MATERIALIZE_FILE;
|
||||||
|
import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_STAGE_FILE_UPLOAD;
|
||||||
import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
|
import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
|
||||||
import static org.apache.hadoop.fs.s3a.Constants.*;
|
import static org.apache.hadoop.fs.s3a.Constants.*;
|
||||||
|
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The implementation of the various actions a committer needs.
|
* The implementation of the various actions a committer needs.
|
||||||
|
@ -71,7 +78,7 @@ import static org.apache.hadoop.fs.s3a.Constants.*;
|
||||||
* duplicate that work.
|
* duplicate that work.
|
||||||
*
|
*
|
||||||
*/
|
*/
|
||||||
public class CommitOperations {
|
public class CommitOperations implements IOStatisticsSource {
|
||||||
private static final Logger LOG = LoggerFactory.getLogger(
|
private static final Logger LOG = LoggerFactory.getLogger(
|
||||||
CommitOperations.class);
|
CommitOperations.class);
|
||||||
|
|
||||||
|
@ -81,7 +88,7 @@ public class CommitOperations {
|
||||||
private final S3AFileSystem fs;
|
private final S3AFileSystem fs;
|
||||||
|
|
||||||
/** Statistics. */
|
/** Statistics. */
|
||||||
private final S3AInstrumentation.CommitterStatistics statistics;
|
private final CommitterStatistics statistics;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Write operations for the destination fs.
|
* Write operations for the destination fs.
|
||||||
|
@ -105,9 +112,18 @@ public class CommitOperations {
|
||||||
* @param fs FS to bind to
|
* @param fs FS to bind to
|
||||||
*/
|
*/
|
||||||
public CommitOperations(S3AFileSystem fs) {
|
public CommitOperations(S3AFileSystem fs) {
|
||||||
Preconditions.checkArgument(fs != null, "null fs");
|
this(requireNonNull(fs), fs.newCommitterStatistics());
|
||||||
this.fs = fs;
|
}
|
||||||
statistics = fs.newCommitterStatistics();
|
|
||||||
|
/**
|
||||||
|
* Instantiate.
|
||||||
|
* @param fs FS to bind to
|
||||||
|
* @param committerStatistics committer statistics
|
||||||
|
*/
|
||||||
|
public CommitOperations(S3AFileSystem fs,
|
||||||
|
CommitterStatistics committerStatistics) {
|
||||||
|
this.fs = requireNonNull(fs);
|
||||||
|
statistics = requireNonNull(committerStatistics);
|
||||||
writeOperations = fs.getWriteOperationHelper();
|
writeOperations = fs.getWriteOperationHelper();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -128,10 +144,15 @@ public class CommitOperations {
|
||||||
}
|
}
|
||||||
|
|
||||||
/** @return statistics. */
|
/** @return statistics. */
|
||||||
protected S3AInstrumentation.CommitterStatistics getStatistics() {
|
protected CommitterStatistics getStatistics() {
|
||||||
return statistics;
|
return statistics;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public IOStatistics getIOStatistics() {
|
||||||
|
return statistics.getIOStatistics();
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Commit the operation, throwing an exception on any failure.
|
* Commit the operation, throwing an exception on any failure.
|
||||||
* @param commit commit to execute
|
* @param commit commit to execute
|
||||||
|
@ -166,7 +187,8 @@ public class CommitOperations {
|
||||||
|
|
||||||
commit.validate();
|
commit.validate();
|
||||||
destKey = commit.getDestinationKey();
|
destKey = commit.getDestinationKey();
|
||||||
long l = innerCommit(commit, operationState);
|
long l = trackDuration(statistics, COMMITTER_MATERIALIZE_FILE.getSymbol(),
|
||||||
|
() -> innerCommit(commit, operationState));
|
||||||
LOG.debug("Successful commit of file length {}", l);
|
LOG.debug("Successful commit of file length {}", l);
|
||||||
outcome = MaybeIOE.NONE;
|
outcome = MaybeIOE.NONE;
|
||||||
statistics.commitCompleted(commit.getLength());
|
statistics.commitCompleted(commit.getLength());
|
||||||
|
@ -449,7 +471,7 @@ public class CommitOperations {
|
||||||
* @return a pending upload entry
|
* @return a pending upload entry
|
||||||
* @throws IOException failure
|
* @throws IOException failure
|
||||||
*/
|
*/
|
||||||
public SinglePendingCommit uploadFileToPendingCommit(File localFile,
|
public SinglePendingCommit uploadFileToPendingCommit(File localFile,
|
||||||
Path destPath,
|
Path destPath,
|
||||||
String partition,
|
String partition,
|
||||||
long uploadPartSize,
|
long uploadPartSize,
|
||||||
|
@ -466,7 +488,11 @@ public class CommitOperations {
|
||||||
String destKey = fs.pathToKey(destPath);
|
String destKey = fs.pathToKey(destPath);
|
||||||
String uploadId = null;
|
String uploadId = null;
|
||||||
|
|
||||||
|
// flag to indicate to the finally clause that the operation
|
||||||
|
// failed. it is cleared as the last action in the try block.
|
||||||
boolean threw = true;
|
boolean threw = true;
|
||||||
|
final DurationTracker tracker = statistics.trackDuration(
|
||||||
|
COMMITTER_STAGE_FILE_UPLOAD.getSymbol());
|
||||||
try (DurationInfo d = new DurationInfo(LOG,
|
try (DurationInfo d = new DurationInfo(LOG,
|
||||||
"Upload staged file from %s to %s",
|
"Upload staged file from %s to %s",
|
||||||
localFile.getAbsolutePath(),
|
localFile.getAbsolutePath(),
|
||||||
|
@ -507,6 +533,7 @@ public class CommitOperations {
|
||||||
LOG.debug("File size is {}, number of parts to upload = {}",
|
LOG.debug("File size is {}, number of parts to upload = {}",
|
||||||
length, numParts);
|
length, numParts);
|
||||||
for (int partNumber = 1; partNumber <= numParts; partNumber += 1) {
|
for (int partNumber = 1; partNumber <= numParts; partNumber += 1) {
|
||||||
|
progress.progress();
|
||||||
long size = Math.min(length - offset, uploadPartSize);
|
long size = Math.min(length - offset, uploadPartSize);
|
||||||
UploadPartRequest part;
|
UploadPartRequest part;
|
||||||
part = writeOperations.newUploadPartRequest(
|
part = writeOperations.newUploadPartRequest(
|
||||||
|
@ -525,7 +552,7 @@ public class CommitOperations {
|
||||||
|
|
||||||
commitData.bindCommitData(parts);
|
commitData.bindCommitData(parts);
|
||||||
statistics.commitUploaded(length);
|
statistics.commitUploaded(length);
|
||||||
progress.progress();
|
// clear the threw flag.
|
||||||
threw = false;
|
threw = false;
|
||||||
return commitData;
|
return commitData;
|
||||||
} finally {
|
} finally {
|
||||||
|
@ -536,6 +563,11 @@ public class CommitOperations {
|
||||||
LOG.error("Failed to abort upload {} to {}", uploadId, destKey, e);
|
LOG.error("Failed to abort upload {} to {}", uploadId, destKey, e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
if (threw) {
|
||||||
|
tracker.failed();
|
||||||
|
}
|
||||||
|
// close tracker and so report statistics of success/failure
|
||||||
|
tracker.close();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -27,6 +27,7 @@ import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.fs.s3a.S3AFileSystem;
|
import org.apache.hadoop.fs.s3a.S3AFileSystem;
|
||||||
import org.apache.hadoop.fs.s3a.Statistic;
|
import org.apache.hadoop.fs.s3a.Statistic;
|
||||||
import org.apache.hadoop.fs.s3a.commit.magic.MagicCommitTracker;
|
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.*;
|
import static org.apache.hadoop.fs.s3a.commit.MagicCommitPaths.*;
|
||||||
|
|
||||||
|
@ -50,6 +51,8 @@ public class MagicCommitIntegration {
|
||||||
private final S3AFileSystem owner;
|
private final S3AFileSystem owner;
|
||||||
private final boolean magicCommitEnabled;
|
private final boolean magicCommitEnabled;
|
||||||
|
|
||||||
|
private final StoreContext storeContext;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Instantiate.
|
* Instantiate.
|
||||||
* @param owner owner class
|
* @param owner owner class
|
||||||
|
@ -59,6 +62,7 @@ public class MagicCommitIntegration {
|
||||||
boolean magicCommitEnabled) {
|
boolean magicCommitEnabled) {
|
||||||
this.owner = owner;
|
this.owner = owner;
|
||||||
this.magicCommitEnabled = magicCommitEnabled;
|
this.magicCommitEnabled = magicCommitEnabled;
|
||||||
|
this.storeContext = owner.createStoreContext();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -94,10 +98,10 @@ public class MagicCommitIntegration {
|
||||||
if (isMagicCommitPath(elements)) {
|
if (isMagicCommitPath(elements)) {
|
||||||
final String destKey = keyOfFinalDestination(elements, key);
|
final String destKey = keyOfFinalDestination(elements, key);
|
||||||
String pendingsetPath = key + CommitConstants.PENDING_SUFFIX;
|
String pendingsetPath = key + CommitConstants.PENDING_SUFFIX;
|
||||||
owner.getInstrumentation()
|
storeContext.incrementStatistic(
|
||||||
.incrementCounter(Statistic.COMMITTER_MAGIC_FILES_CREATED, 1);
|
Statistic.COMMITTER_MAGIC_FILES_CREATED);
|
||||||
tracker = new MagicCommitTracker(path,
|
tracker = new MagicCommitTracker(path,
|
||||||
owner.getBucket(),
|
storeContext.getBucket(),
|
||||||
key,
|
key,
|
||||||
destKey,
|
destKey,
|
||||||
pendingsetPath,
|
pendingsetPath,
|
||||||
|
|
|
@ -24,6 +24,7 @@ import java.util.List;
|
||||||
import com.amazonaws.services.s3.model.PartETag;
|
import com.amazonaws.services.s3.model.PartETag;
|
||||||
|
|
||||||
import org.apache.hadoop.classification.InterfaceAudience;
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatistics;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Multipart put tracker.
|
* Multipart put tracker.
|
||||||
|
@ -68,6 +69,7 @@ public class PutTracker {
|
||||||
* @param uploadId Upload ID
|
* @param uploadId Upload ID
|
||||||
* @param parts list of parts
|
* @param parts list of parts
|
||||||
* @param bytesWritten bytes written
|
* @param bytesWritten bytes written
|
||||||
|
* @param iostatistics nullable IO statistics
|
||||||
* @return true if the commit is to be initiated immediately.
|
* @return true if the commit is to be initiated immediately.
|
||||||
* False implies the output stream does not need to worry about
|
* False implies the output stream does not need to worry about
|
||||||
* what happens.
|
* what happens.
|
||||||
|
@ -75,7 +77,8 @@ public class PutTracker {
|
||||||
*/
|
*/
|
||||||
public boolean aboutToComplete(String uploadId,
|
public boolean aboutToComplete(String uploadId,
|
||||||
List<PartETag> parts,
|
List<PartETag> parts,
|
||||||
long bytesWritten)
|
long bytesWritten,
|
||||||
|
final IOStatistics iostatistics)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
|
@ -27,6 +27,7 @@ import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
@ -36,6 +37,8 @@ import org.apache.hadoop.fs.FileStatus;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.fs.s3a.commit.ValidationFailure;
|
import org.apache.hadoop.fs.s3a.commit.ValidationFailure;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
|
||||||
import org.apache.hadoop.util.JsonSerialization;
|
import org.apache.hadoop.util.JsonSerialization;
|
||||||
|
|
||||||
import static org.apache.hadoop.fs.s3a.commit.CommitUtils.validateCollectionClass;
|
import static org.apache.hadoop.fs.s3a.commit.CommitUtils.validateCollectionClass;
|
||||||
|
@ -45,11 +48,22 @@ import static org.apache.hadoop.fs.s3a.commit.ValidationFailure.verify;
|
||||||
* Persistent format for multiple pending commits.
|
* Persistent format for multiple pending commits.
|
||||||
* Contains 0 or more {@link SinglePendingCommit} entries; validation logic
|
* Contains 0 or more {@link SinglePendingCommit} entries; validation logic
|
||||||
* checks those values on load.
|
* checks those values on load.
|
||||||
|
* <p>
|
||||||
|
* The statistics published through the {@link IOStatisticsSource}
|
||||||
|
* interface are the static ones marshalled with the commit data;
|
||||||
|
* they may be empty.
|
||||||
|
* </p>
|
||||||
|
* <p>
|
||||||
|
* As single commits are added via {@link #add(SinglePendingCommit)},
|
||||||
|
* any statistics from those commits are merged into the aggregate
|
||||||
|
* statistics, <i>and those of the single commit cleared.</i>
|
||||||
|
* </p>
|
||||||
*/
|
*/
|
||||||
@SuppressWarnings("unused")
|
@SuppressWarnings("unused")
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
@InterfaceStability.Unstable
|
@InterfaceStability.Unstable
|
||||||
public class PendingSet extends PersistentCommitData {
|
public class PendingSet extends PersistentCommitData
|
||||||
|
implements IOStatisticsSource {
|
||||||
private static final Logger LOG = LoggerFactory.getLogger(PendingSet.class);
|
private static final Logger LOG = LoggerFactory.getLogger(PendingSet.class);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -57,7 +71,7 @@ public class PendingSet extends PersistentCommitData {
|
||||||
* If this is changed the value of {@link #serialVersionUID} will change,
|
* If this is changed the value of {@link #serialVersionUID} will change,
|
||||||
* to avoid deserialization problems.
|
* to avoid deserialization problems.
|
||||||
*/
|
*/
|
||||||
public static final int VERSION = 2;
|
public static final int VERSION = 3;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Serialization ID: {@value}.
|
* Serialization ID: {@value}.
|
||||||
|
@ -81,6 +95,12 @@ public class PendingSet extends PersistentCommitData {
|
||||||
*/
|
*/
|
||||||
private final Map<String, String> extraData = new HashMap<>(0);
|
private final Map<String, String> extraData = new HashMap<>(0);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* IOStatistics.
|
||||||
|
*/
|
||||||
|
@JsonProperty("iostatistics")
|
||||||
|
private IOStatisticsSnapshot iostats = new IOStatisticsSnapshot();
|
||||||
|
|
||||||
public PendingSet() {
|
public PendingSet() {
|
||||||
this(0);
|
this(0);
|
||||||
}
|
}
|
||||||
|
@ -133,6 +153,12 @@ public class PendingSet extends PersistentCommitData {
|
||||||
*/
|
*/
|
||||||
public void add(SinglePendingCommit commit) {
|
public void add(SinglePendingCommit commit) {
|
||||||
commits.add(commit);
|
commits.add(commit);
|
||||||
|
// add any statistics.
|
||||||
|
IOStatisticsSnapshot st = commit.getIOStatistics();
|
||||||
|
if (st != null) {
|
||||||
|
iostats.aggregate(st);
|
||||||
|
st.clear();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -225,4 +251,12 @@ public class PendingSet extends PersistentCommitData {
|
||||||
this.jobId = jobId;
|
this.jobId = jobId;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public IOStatisticsSnapshot getIOStatistics() {
|
||||||
|
return iostats;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setIOStatistics(final IOStatisticsSnapshot ioStatistics) {
|
||||||
|
this.iostats = ioStatistics;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -31,6 +31,8 @@ import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
import com.amazonaws.services.s3.model.PartETag;
|
import com.amazonaws.services.s3.model.PartETag;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
|
||||||
import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
|
import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
|
||||||
|
|
||||||
import org.apache.commons.lang3.StringUtils;
|
import org.apache.commons.lang3.StringUtils;
|
||||||
|
@ -39,6 +41,8 @@ import org.apache.hadoop.classification.InterfaceStability;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.fs.s3a.commit.ValidationFailure;
|
import org.apache.hadoop.fs.s3a.commit.ValidationFailure;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
|
||||||
import org.apache.hadoop.util.JsonSerialization;
|
import org.apache.hadoop.util.JsonSerialization;
|
||||||
|
|
||||||
import static org.apache.hadoop.fs.s3a.commit.CommitUtils.validateCollectionClass;
|
import static org.apache.hadoop.fs.s3a.commit.CommitUtils.validateCollectionClass;
|
||||||
|
@ -46,20 +50,25 @@ import static org.apache.hadoop.fs.s3a.commit.ValidationFailure.verify;
|
||||||
import static org.apache.hadoop.util.StringUtils.join;
|
import static org.apache.hadoop.util.StringUtils.join;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This is the serialization format for uploads yet to be committerd.
|
* This is the serialization format for uploads yet to be committed.
|
||||||
*
|
* <p>
|
||||||
* It's marked as {@link Serializable} so that it can be passed in RPC
|
* It's marked as {@link Serializable} so that it can be passed in RPC
|
||||||
* calls; for this to work it relies on the fact that java.io ArrayList
|
* calls; for this to work it relies on the fact that java.io ArrayList
|
||||||
* and LinkedList are serializable. If any other list type is used for etags,
|
* and LinkedList are serializable. If any other list type is used for etags,
|
||||||
* it must also be serialized. Jackson expects lists, and it is used
|
* it must also be serialized. Jackson expects lists, and it is used
|
||||||
* to persist to disk.
|
* to persist to disk.
|
||||||
*
|
* </p>
|
||||||
|
* <p>
|
||||||
|
* The statistics published through the {@link IOStatisticsSource}
|
||||||
|
* interface are the static ones marshalled with the commit data;
|
||||||
|
* they may be empty.
|
||||||
|
* </p>
|
||||||
*/
|
*/
|
||||||
@SuppressWarnings("unused")
|
@SuppressWarnings("unused")
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
@InterfaceStability.Unstable
|
@InterfaceStability.Unstable
|
||||||
public class SinglePendingCommit extends PersistentCommitData
|
public class SinglePendingCommit extends PersistentCommitData
|
||||||
implements Iterable<String> {
|
implements Iterable<String>, IOStatisticsSource {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Serialization ID: {@value}.
|
* Serialization ID: {@value}.
|
||||||
|
@ -113,6 +122,12 @@ public class SinglePendingCommit extends PersistentCommitData
|
||||||
*/
|
*/
|
||||||
private Map<String, String> extraData = new HashMap<>(0);
|
private Map<String, String> extraData = new HashMap<>(0);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* IOStatistics.
|
||||||
|
*/
|
||||||
|
@JsonProperty("iostatistics")
|
||||||
|
private IOStatisticsSnapshot iostats = new IOStatisticsSnapshot();
|
||||||
|
|
||||||
/** Destination file size. */
|
/** Destination file size. */
|
||||||
private long length;
|
private long length;
|
||||||
|
|
||||||
|
@ -439,4 +454,12 @@ public class SinglePendingCommit extends PersistentCommitData
|
||||||
this.length = length;
|
this.length = length;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public IOStatisticsSnapshot getIOStatistics() {
|
||||||
|
return iostats;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setIOStatistics(final IOStatisticsSnapshot ioStatistics) {
|
||||||
|
this.iostats = ioStatistics;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -25,6 +25,7 @@ import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
@ -34,6 +35,8 @@ import org.apache.hadoop.classification.InterfaceStability;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.fs.s3a.commit.ValidationFailure;
|
import org.apache.hadoop.fs.s3a.commit.ValidationFailure;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
|
||||||
import org.apache.hadoop.util.JsonSerialization;
|
import org.apache.hadoop.util.JsonSerialization;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -62,9 +65,18 @@ import org.apache.hadoop.util.JsonSerialization;
|
||||||
@SuppressWarnings("unused")
|
@SuppressWarnings("unused")
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
@InterfaceStability.Unstable
|
@InterfaceStability.Unstable
|
||||||
public class SuccessData extends PersistentCommitData {
|
public class SuccessData extends PersistentCommitData
|
||||||
|
implements IOStatisticsSource {
|
||||||
|
|
||||||
private static final Logger LOG = LoggerFactory.getLogger(SuccessData.class);
|
private static final Logger LOG = LoggerFactory.getLogger(SuccessData.class);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Supported version value: {@value}.
|
||||||
|
* If this is changed the value of {@link #serialVersionUID} will change,
|
||||||
|
* to avoid deserialization problems.
|
||||||
|
*/
|
||||||
|
public static final int VERSION = 1;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Serialization ID: {@value}.
|
* Serialization ID: {@value}.
|
||||||
*/
|
*/
|
||||||
|
@ -75,7 +87,7 @@ public class SuccessData extends PersistentCommitData {
|
||||||
* any other manifests: {@value}.
|
* any other manifests: {@value}.
|
||||||
*/
|
*/
|
||||||
public static final String NAME
|
public static final String NAME
|
||||||
= "org.apache.hadoop.fs.s3a.commit.files.SuccessData/1";
|
= "org.apache.hadoop.fs.s3a.commit.files.SuccessData/" + VERSION;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Name of file; includes version marker.
|
* Name of file; includes version marker.
|
||||||
|
@ -126,6 +138,12 @@ public class SuccessData extends PersistentCommitData {
|
||||||
*/
|
*/
|
||||||
private List<String> filenames = new ArrayList<>(0);
|
private List<String> filenames = new ArrayList<>(0);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* IOStatistics.
|
||||||
|
*/
|
||||||
|
@JsonProperty("iostatistics")
|
||||||
|
private IOStatisticsSnapshot iostats = new IOStatisticsSnapshot();
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void validate() throws ValidationFailure {
|
public void validate() throws ValidationFailure {
|
||||||
ValidationFailure.verify(name != null,
|
ValidationFailure.verify(name != null,
|
||||||
|
@ -350,4 +368,13 @@ public class SuccessData extends PersistentCommitData {
|
||||||
public void setJobIdSource(final String jobIdSource) {
|
public void setJobIdSource(final String jobIdSource) {
|
||||||
this.jobIdSource = jobIdSource;
|
this.jobIdSource = jobIdSource;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public IOStatisticsSnapshot getIOStatistics() {
|
||||||
|
return iostats;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setIOStatistics(final IOStatisticsSnapshot ioStatistics) {
|
||||||
|
this.iostats = ioStatistics;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -34,6 +34,8 @@ import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.fs.s3a.WriteOperationHelper;
|
import org.apache.hadoop.fs.s3a.WriteOperationHelper;
|
||||||
import org.apache.hadoop.fs.s3a.commit.PutTracker;
|
import org.apache.hadoop.fs.s3a.commit.PutTracker;
|
||||||
import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit;
|
import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatistics;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Put tracker for Magic commits.
|
* Put tracker for Magic commits.
|
||||||
|
@ -102,6 +104,7 @@ public class MagicCommitTracker extends PutTracker {
|
||||||
* @param uploadId Upload ID
|
* @param uploadId Upload ID
|
||||||
* @param parts list of parts
|
* @param parts list of parts
|
||||||
* @param bytesWritten bytes written
|
* @param bytesWritten bytes written
|
||||||
|
* @param iostatistics nullable IO statistics
|
||||||
* @return false, indicating that the commit must fail.
|
* @return false, indicating that the commit must fail.
|
||||||
* @throws IOException any IO problem.
|
* @throws IOException any IO problem.
|
||||||
* @throws IllegalArgumentException bad argument
|
* @throws IllegalArgumentException bad argument
|
||||||
|
@ -109,7 +112,8 @@ public class MagicCommitTracker extends PutTracker {
|
||||||
@Override
|
@Override
|
||||||
public boolean aboutToComplete(String uploadId,
|
public boolean aboutToComplete(String uploadId,
|
||||||
List<PartETag> parts,
|
List<PartETag> parts,
|
||||||
long bytesWritten)
|
long bytesWritten,
|
||||||
|
final IOStatistics iostatistics)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
Preconditions.checkArgument(StringUtils.isNotEmpty(uploadId),
|
Preconditions.checkArgument(StringUtils.isNotEmpty(uploadId),
|
||||||
"empty/null upload ID: "+ uploadId);
|
"empty/null upload ID: "+ uploadId);
|
||||||
|
@ -117,6 +121,15 @@ public class MagicCommitTracker extends PutTracker {
|
||||||
"No uploaded parts list");
|
"No uploaded parts list");
|
||||||
Preconditions.checkArgument(!parts.isEmpty(),
|
Preconditions.checkArgument(!parts.isEmpty(),
|
||||||
"No uploaded parts to save");
|
"No uploaded parts to save");
|
||||||
|
|
||||||
|
// put a 0-byte file with the name of the original under-magic path
|
||||||
|
PutObjectRequest originalDestPut = writer.createPutObjectRequest(
|
||||||
|
originalDestKey,
|
||||||
|
new ByteArrayInputStream(EMPTY),
|
||||||
|
0);
|
||||||
|
writer.uploadObject(originalDestPut);
|
||||||
|
|
||||||
|
// build the commit summary
|
||||||
SinglePendingCommit commitData = new SinglePendingCommit();
|
SinglePendingCommit commitData = new SinglePendingCommit();
|
||||||
commitData.touch(System.currentTimeMillis());
|
commitData.touch(System.currentTimeMillis());
|
||||||
commitData.setDestinationKey(getDestKey());
|
commitData.setDestinationKey(getDestKey());
|
||||||
|
@ -126,6 +139,8 @@ public class MagicCommitTracker extends PutTracker {
|
||||||
commitData.setText("");
|
commitData.setText("");
|
||||||
commitData.setLength(bytesWritten);
|
commitData.setLength(bytesWritten);
|
||||||
commitData.bindCommitData(parts);
|
commitData.bindCommitData(parts);
|
||||||
|
commitData.setIOStatistics(
|
||||||
|
new IOStatisticsSnapshot(iostatistics));
|
||||||
byte[] bytes = commitData.toBytes();
|
byte[] bytes = commitData.toBytes();
|
||||||
LOG.info("Uncommitted data pending to file {};"
|
LOG.info("Uncommitted data pending to file {};"
|
||||||
+ " commit metadata for {} parts in {}. sixe: {} byte(s)",
|
+ " commit metadata for {} parts in {}. sixe: {} byte(s)",
|
||||||
|
@ -138,12 +153,6 @@ public class MagicCommitTracker extends PutTracker {
|
||||||
bytes.length);
|
bytes.length);
|
||||||
writer.uploadObject(put);
|
writer.uploadObject(put);
|
||||||
|
|
||||||
// now put a 0-byte file with the name of the original under-magic path
|
|
||||||
PutObjectRequest originalDestPut = writer.createPutObjectRequest(
|
|
||||||
originalDestKey,
|
|
||||||
new ByteArrayInputStream(EMPTY),
|
|
||||||
0);
|
|
||||||
writer.uploadObject(originalDestPut);
|
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -37,6 +37,7 @@ import org.apache.hadoop.fs.s3a.commit.CommitConstants;
|
||||||
import org.apache.hadoop.fs.s3a.commit.CommitUtilsWithMR;
|
import org.apache.hadoop.fs.s3a.commit.CommitUtilsWithMR;
|
||||||
import org.apache.hadoop.fs.s3a.commit.files.PendingSet;
|
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.SinglePendingCommit;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatisticsLogging;
|
||||||
import org.apache.hadoop.mapreduce.JobContext;
|
import org.apache.hadoop.mapreduce.JobContext;
|
||||||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||||
import org.apache.hadoop.mapreduce.TaskAttemptID;
|
import org.apache.hadoop.mapreduce.TaskAttemptID;
|
||||||
|
@ -47,6 +48,7 @@ import static org.apache.hadoop.fs.s3a.commit.CommitConstants.TASK_ATTEMPT_ID;
|
||||||
import static org.apache.hadoop.fs.s3a.commit.CommitUtils.*;
|
import static org.apache.hadoop.fs.s3a.commit.CommitUtils.*;
|
||||||
import static org.apache.hadoop.fs.s3a.commit.MagicCommitPaths.*;
|
import static org.apache.hadoop.fs.s3a.commit.MagicCommitPaths.*;
|
||||||
import static org.apache.hadoop.fs.s3a.commit.CommitUtilsWithMR.*;
|
import static org.apache.hadoop.fs.s3a.commit.CommitUtilsWithMR.*;
|
||||||
|
import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.demandStringifyIOStatistics;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This is a dedicated committer which requires the "magic" directory feature
|
* This is a dedicated committer which requires the "magic" directory feature
|
||||||
|
@ -169,6 +171,8 @@ public class MagicS3GuardCommitter extends AbstractS3ACommitter {
|
||||||
destroyThreadPool();
|
destroyThreadPool();
|
||||||
}
|
}
|
||||||
getCommitOperations().taskCompleted(true);
|
getCommitOperations().taskCompleted(true);
|
||||||
|
LOG.debug("aggregate statistics\n{}",
|
||||||
|
demandStringifyIOStatistics(getIOStatistics()));
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -213,6 +217,8 @@ public class MagicS3GuardCommitter extends AbstractS3ACommitter {
|
||||||
taskAttemptID.getTaskID().toString() +
|
taskAttemptID.getTaskID().toString() +
|
||||||
CommitConstants.PENDINGSET_SUFFIX);
|
CommitConstants.PENDINGSET_SUFFIX);
|
||||||
LOG.info("Saving work of {} to {}", taskAttemptID, taskOutcomePath);
|
LOG.info("Saving work of {} to {}", taskAttemptID, taskOutcomePath);
|
||||||
|
LOG.debug("task statistics\n{}",
|
||||||
|
IOStatisticsLogging.demandStringifyIOStatisticsSource(pendingSet));
|
||||||
try {
|
try {
|
||||||
// We will overwrite if there exists a pendingSet file already
|
// We will overwrite if there exists a pendingSet file already
|
||||||
pendingSet.save(getDestFS(), taskOutcomePath, true);
|
pendingSet.save(getDestFS(), taskOutcomePath, true);
|
||||||
|
|
|
@ -0,0 +1,92 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.fs.s3a.impl;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
|
||||||
|
import java.util.Objects;
|
||||||
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
|
|
||||||
|
import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Context for any active operation.
|
||||||
|
*/
|
||||||
|
public class ActiveOperationContext {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* An operation ID; currently just for logging...proper tracing needs more.
|
||||||
|
*/
|
||||||
|
private final long operationId;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Statistics context.
|
||||||
|
*/
|
||||||
|
private final S3AStatisticsContext statisticsContext;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* S3Guard bulk operation state, if (currently) set.
|
||||||
|
*/
|
||||||
|
@Nullable private BulkOperationState bulkOperationState;
|
||||||
|
|
||||||
|
public ActiveOperationContext(
|
||||||
|
final long operationId,
|
||||||
|
final S3AStatisticsContext statisticsContext,
|
||||||
|
@Nullable final BulkOperationState bulkOperationState) {
|
||||||
|
this.operationId = operationId;
|
||||||
|
this.statisticsContext = Objects.requireNonNull(statisticsContext,
|
||||||
|
"null statistics context");
|
||||||
|
this.bulkOperationState = bulkOperationState;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
final StringBuilder sb = new StringBuilder(
|
||||||
|
"ActiveOperation{");
|
||||||
|
sb.append("operationId=").append(operationId);
|
||||||
|
sb.append(", bulkOperationState=").append(bulkOperationState);
|
||||||
|
sb.append('}');
|
||||||
|
return sb.toString();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
public BulkOperationState getBulkOperationState() {
|
||||||
|
return bulkOperationState;
|
||||||
|
}
|
||||||
|
|
||||||
|
public long getOperationId() {
|
||||||
|
return operationId;
|
||||||
|
}
|
||||||
|
|
||||||
|
public S3AStatisticsContext getS3AStatisticsContext() {
|
||||||
|
return statisticsContext;
|
||||||
|
}
|
||||||
|
|
||||||
|
private static final AtomicLong NEXT_OPERATION_ID = new AtomicLong(0);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create an operation ID. The nature of it should be opaque.
|
||||||
|
* @return an ID for the constructor.
|
||||||
|
*/
|
||||||
|
protected static long newOperationId() {
|
||||||
|
return NEXT_OPERATION_ID.incrementAndGet();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -26,9 +26,9 @@ import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import org.apache.hadoop.fs.s3a.AWSClientIOException;
|
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.S3AStorageStatistics;
|
||||||
import org.apache.hadoop.fs.s3a.Statistic;
|
import org.apache.hadoop.fs.s3a.Statistic;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
|
||||||
|
|
||||||
import static org.apache.hadoop.fs.s3a.S3AUtils.isThrottleException;
|
import static org.apache.hadoop.fs.s3a.S3AUtils.isThrottleException;
|
||||||
import static org.apache.hadoop.fs.s3a.Statistic.IGNORED_ERRORS;
|
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";
|
public static final String XML_PARSE_BROKEN = "Failed to parse XML document";
|
||||||
|
|
||||||
private final S3AInstrumentation instrumentation;
|
private final S3AStatisticsContext instrumentation;
|
||||||
|
|
||||||
private final S3AStorageStatistics storageStatistics;
|
private final S3AStorageStatistics storageStatistics;
|
||||||
|
|
||||||
|
@ -84,7 +84,6 @@ public class BulkDeleteRetryHandler extends AbstractStoreOperation {
|
||||||
*/
|
*/
|
||||||
protected void incrementStatistic(Statistic statistic, long count) {
|
protected void incrementStatistic(Statistic statistic, long count) {
|
||||||
instrumentation.incrementCounter(statistic, count);
|
instrumentation.incrementCounter(statistic, count);
|
||||||
storageStatistics.incrementCounter(statistic, count);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -20,6 +20,7 @@ package org.apache.hadoop.fs.s3a.impl;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.io.UncheckedIOException;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.concurrent.Callable;
|
import java.util.concurrent.Callable;
|
||||||
import java.util.concurrent.CancellationException;
|
import java.util.concurrent.CancellationException;
|
||||||
|
@ -31,7 +32,6 @@ import java.util.function.Supplier;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import org.apache.hadoop.fs.impl.WrappedIOException;
|
|
||||||
import org.apache.hadoop.util.DurationInfo;
|
import org.apache.hadoop.util.DurationInfo;
|
||||||
|
|
||||||
import static org.apache.hadoop.fs.impl.FutureIOSupport.raiseInnerCause;
|
import static org.apache.hadoop.fs.impl.FutureIOSupport.raiseInnerCause;
|
||||||
|
@ -63,9 +63,9 @@ public final class CallableSupplier<T> implements Supplier {
|
||||||
} catch (RuntimeException e) {
|
} catch (RuntimeException e) {
|
||||||
throw e;
|
throw e;
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
throw new WrappedIOException(e);
|
throw new UncheckedIOException(e);
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
throw new WrappedIOException(new IOException(e));
|
throw new UncheckedIOException(new IOException(e));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -73,7 +73,7 @@ public final class CallableSupplier<T> implements Supplier {
|
||||||
* Submit a callable into a completable future.
|
* Submit a callable into a completable future.
|
||||||
* RTEs are rethrown.
|
* RTEs are rethrown.
|
||||||
* Non RTEs are caught and wrapped; IOExceptions to
|
* Non RTEs are caught and wrapped; IOExceptions to
|
||||||
* {@link WrappedIOException} instances.
|
* {@code RuntimeIOException} instances.
|
||||||
* @param executor executor.
|
* @param executor executor.
|
||||||
* @param call call to invoke
|
* @param call call to invoke
|
||||||
* @param <T> type
|
* @param <T> type
|
||||||
|
|
|
@ -18,8 +18,6 @@
|
||||||
|
|
||||||
package org.apache.hadoop.fs.s3a.impl;
|
package org.apache.hadoop.fs.s3a.impl;
|
||||||
|
|
||||||
import java.util.concurrent.atomic.AtomicLong;
|
|
||||||
|
|
||||||
import com.amazonaws.AmazonServiceException;
|
import com.amazonaws.AmazonServiceException;
|
||||||
import com.amazonaws.SdkBaseException;
|
import com.amazonaws.SdkBaseException;
|
||||||
import com.amazonaws.services.s3.model.CopyObjectRequest;
|
import com.amazonaws.services.s3.model.CopyObjectRequest;
|
||||||
|
@ -39,6 +37,7 @@ import org.apache.hadoop.fs.PathIOException;
|
||||||
import org.apache.hadoop.fs.s3a.NoVersionAttributeException;
|
import org.apache.hadoop.fs.s3a.NoVersionAttributeException;
|
||||||
import org.apache.hadoop.fs.s3a.RemoteFileChangedException;
|
import org.apache.hadoop.fs.s3a.RemoteFileChangedException;
|
||||||
import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
|
import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.ChangeTrackerStatistics;
|
||||||
|
|
||||||
import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull;
|
import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull;
|
||||||
|
|
||||||
|
@ -72,7 +71,7 @@ public class ChangeTracker {
|
||||||
* Mismatch counter; expected to be wired up to StreamStatistics except
|
* Mismatch counter; expected to be wired up to StreamStatistics except
|
||||||
* during testing.
|
* during testing.
|
||||||
*/
|
*/
|
||||||
private final AtomicLong versionMismatches;
|
private final ChangeTrackerStatistics versionMismatches;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Revision identifier (e.g. eTag or versionId, depending on change
|
* Revision identifier (e.g. eTag or versionId, depending on change
|
||||||
|
@ -90,7 +89,7 @@ public class ChangeTracker {
|
||||||
*/
|
*/
|
||||||
public ChangeTracker(final String uri,
|
public ChangeTracker(final String uri,
|
||||||
final ChangeDetectionPolicy policy,
|
final ChangeDetectionPolicy policy,
|
||||||
final AtomicLong versionMismatches,
|
final ChangeTrackerStatistics versionMismatches,
|
||||||
final S3ObjectAttributes s3ObjectAttributes) {
|
final S3ObjectAttributes s3ObjectAttributes) {
|
||||||
this.policy = checkNotNull(policy);
|
this.policy = checkNotNull(policy);
|
||||||
this.uri = uri;
|
this.uri = uri;
|
||||||
|
@ -111,8 +110,8 @@ public class ChangeTracker {
|
||||||
}
|
}
|
||||||
|
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
public AtomicLong getVersionMismatches() {
|
public long getVersionMismatches() {
|
||||||
return versionMismatches;
|
return versionMismatches.getVersionMismatches();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -177,7 +176,7 @@ public class ChangeTracker {
|
||||||
if (revisionId != null) {
|
if (revisionId != null) {
|
||||||
// the requirements of the change detection policy wasn't met: the
|
// the requirements of the change detection policy wasn't met: the
|
||||||
// object was not returned.
|
// object was not returned.
|
||||||
versionMismatches.incrementAndGet();
|
versionMismatches.versionMismatchError();
|
||||||
throw new RemoteFileChangedException(uri, operation,
|
throw new RemoteFileChangedException(uri, operation,
|
||||||
String.format(CHANGE_REPORTED_BY_S3
|
String.format(CHANGE_REPORTED_BY_S3
|
||||||
+ " during %s"
|
+ " during %s"
|
||||||
|
@ -235,7 +234,7 @@ public class ChangeTracker {
|
||||||
// This isn't really going to be hit due to
|
// This isn't really going to be hit due to
|
||||||
// https://github.com/aws/aws-sdk-java/issues/1644
|
// https://github.com/aws/aws-sdk-java/issues/1644
|
||||||
if (serviceException.getStatusCode() == SC_PRECONDITION_FAILED) {
|
if (serviceException.getStatusCode() == SC_PRECONDITION_FAILED) {
|
||||||
versionMismatches.incrementAndGet();
|
versionMismatches.versionMismatchError();
|
||||||
throw new RemoteFileChangedException(uri, operation, String.format(
|
throw new RemoteFileChangedException(uri, operation, String.format(
|
||||||
RemoteFileChangedException.PRECONDITIONS_FAILED
|
RemoteFileChangedException.PRECONDITIONS_FAILED
|
||||||
+ " on %s."
|
+ " on %s."
|
||||||
|
@ -292,10 +291,10 @@ public class ChangeTracker {
|
||||||
uri,
|
uri,
|
||||||
pos,
|
pos,
|
||||||
operation,
|
operation,
|
||||||
versionMismatches.get());
|
versionMismatches.getVersionMismatches());
|
||||||
if (pair.left) {
|
if (pair.left) {
|
||||||
// an mismatch has occurred: note it.
|
// an mismatch has occurred: note it.
|
||||||
versionMismatches.incrementAndGet();
|
versionMismatches.versionMismatchError();
|
||||||
}
|
}
|
||||||
if (pair.right != null) {
|
if (pair.right != null) {
|
||||||
// there's an exception to raise: do it
|
// there's an exception to raise: do it
|
||||||
|
|
|
@ -54,6 +54,18 @@ public class ErrorTranslation {
|
||||||
&& AwsErrorCodes.E_NO_SUCH_BUCKET.equals(e.getErrorCode());
|
&& AwsErrorCodes.E_NO_SUCH_BUCKET.equals(e.getErrorCode());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Does this exception indicate that a reference to an object
|
||||||
|
* returned a 404. Unknown bucket errors do not match this
|
||||||
|
* predicate.
|
||||||
|
* @param e exception.
|
||||||
|
* @return true if the status code and error code mean that the
|
||||||
|
* HEAD request returned 404 but the bucket was there.
|
||||||
|
*/
|
||||||
|
public static boolean isObjectNotFound(AmazonServiceException e) {
|
||||||
|
return e.getStatusCode() == SC_404 && !isUnknownBucket(e);
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* AWS error codes explicitly recognized and processes specially;
|
* AWS error codes explicitly recognized and processes specially;
|
||||||
* kept in their own class for isolation.
|
* kept in their own class for isolation.
|
||||||
|
|
|
@ -110,4 +110,11 @@ public final class InternalConstants {
|
||||||
* Value: {@value}.
|
* Value: {@value}.
|
||||||
*/
|
*/
|
||||||
public static final int DEFAULT_UPLOAD_PART_COUNT_LIMIT = 10000;
|
public static final int DEFAULT_UPLOAD_PART_COUNT_LIMIT = 10000;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Flag to enable AWS Statistics binding. As this is triggering
|
||||||
|
* problems related to region/endpoint setup, it is currently
|
||||||
|
* disabled.
|
||||||
|
*/
|
||||||
|
public static final boolean AWS_SDK_METRICS_ENABLED = false;
|
||||||
}
|
}
|
||||||
|
|
|
@ -29,6 +29,7 @@ import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus;
|
||||||
import org.apache.hadoop.fs.s3a.S3ListRequest;
|
import org.apache.hadoop.fs.s3a.S3ListRequest;
|
||||||
import org.apache.hadoop.fs.s3a.S3ListResult;
|
import org.apache.hadoop.fs.s3a.S3ListResult;
|
||||||
import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider;
|
import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider;
|
||||||
|
import org.apache.hadoop.fs.statistics.DurationTrackerFactory;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* These are all the callbacks which
|
* These are all the callbacks which
|
||||||
|
@ -44,12 +45,14 @@ public interface ListingOperationCallbacks {
|
||||||
*
|
*
|
||||||
* Retry policy: retry untranslated.
|
* Retry policy: retry untranslated.
|
||||||
* @param request request to initiate
|
* @param request request to initiate
|
||||||
|
* @param trackerFactory tracker with statistics to update
|
||||||
* @return the results
|
* @return the results
|
||||||
* @throws IOException if the retry invocation raises one (it shouldn't).
|
* @throws IOException if the retry invocation raises one (it shouldn't).
|
||||||
*/
|
*/
|
||||||
@Retries.RetryRaw
|
@Retries.RetryRaw
|
||||||
CompletableFuture<S3ListResult> listObjectsAsync(
|
CompletableFuture<S3ListResult> listObjectsAsync(
|
||||||
S3ListRequest request)
|
S3ListRequest request,
|
||||||
|
DurationTrackerFactory trackerFactory)
|
||||||
throws IOException;
|
throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -57,13 +60,15 @@ public interface ListingOperationCallbacks {
|
||||||
* Retry policy: retry untranslated.
|
* Retry policy: retry untranslated.
|
||||||
* @param request last list objects request to continue
|
* @param request last list objects request to continue
|
||||||
* @param prevResult last paged result to continue from
|
* @param prevResult last paged result to continue from
|
||||||
|
* @param trackerFactory tracker with statistics to update
|
||||||
* @return the next result object
|
* @return the next result object
|
||||||
* @throws IOException none, just there for retryUntranslated.
|
* @throws IOException none, just there for retryUntranslated.
|
||||||
*/
|
*/
|
||||||
@Retries.RetryRaw
|
@Retries.RetryRaw
|
||||||
CompletableFuture<S3ListResult> continueListObjectsAsync(
|
CompletableFuture<S3ListResult> continueListObjectsAsync(
|
||||||
S3ListRequest request,
|
S3ListRequest request,
|
||||||
S3ListResult prevResult)
|
S3ListResult prevResult,
|
||||||
|
DurationTrackerFactory trackerFactory)
|
||||||
throws IOException;
|
throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -117,4 +122,5 @@ public interface ListingOperationCallbacks {
|
||||||
* @return true iff the path is authoritative on the client.
|
* @return true iff the path is authoritative on the client.
|
||||||
*/
|
*/
|
||||||
boolean allowAuthoritative(Path p);
|
boolean allowAuthoritative(Path p);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -51,22 +51,18 @@ public class NetworkBinding {
|
||||||
".thirdparty.apache.http.conn.ssl.SSLConnectionSocketFactory";
|
".thirdparty.apache.http.conn.ssl.SSLConnectionSocketFactory";
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Configures the {@link com.amazonaws.thirdparty.apache.http.conn.ssl
|
* Configures the {@code SSLConnectionSocketFactory} used by the AWS SDK.
|
||||||
* .SSLConnectionSocketFactory} used by the AWS SDK. A custom
|
* A custom Socket Factory can be set using the method
|
||||||
* SSLConnectionSocketFactory can be set using the method
|
* {@code setSslSocketFactory()}.
|
||||||
* {@link com.amazonaws.ApacheHttpClientConfig#setSslSocketFactory(
|
* If {@code SSLConnectionSocketFactory} cannot be found on the classpath, the value
|
||||||
* com.amazonaws.thirdparty.apache.http.conn.socket.ConnectionSocketFactory)}.
|
|
||||||
* If {@link com.amazonaws.thirdparty.apache.http.conn.ssl
|
|
||||||
* .SSLConnectionSocketFactory} cannot be found on the classpath, the value
|
|
||||||
* of {@link org.apache.hadoop.fs.s3a.Constants#SSL_CHANNEL_MODE} is ignored.
|
* of {@link org.apache.hadoop.fs.s3a.Constants#SSL_CHANNEL_MODE} is ignored.
|
||||||
*
|
*
|
||||||
* @param conf the {@link Configuration} used to get the client specified
|
* @param conf the {@link Configuration} used to get the client specified
|
||||||
* value of {@link org.apache.hadoop.fs.s3a.Constants
|
* value of {@code SSL_CHANNEL_MODE}
|
||||||
* #SSL_CHANNEL_MODE}
|
* @param awsConf the {@code ClientConfiguration} to set the
|
||||||
* @param awsConf the {@link ClientConfiguration} to set the
|
|
||||||
* SSLConnectionSocketFactory for.
|
* SSLConnectionSocketFactory for.
|
||||||
* @throws IOException if there is an error while initializing the
|
* @throws IOException if there is an error while initializing the
|
||||||
* {@link SSLSocketFactory} other than classloader problems.
|
* {@code SSLSocketFactory} other than classloader problems.
|
||||||
*/
|
*/
|
||||||
public static void bindSSLChannelMode(Configuration conf,
|
public static void bindSSLChannelMode(Configuration conf,
|
||||||
ClientConfiguration awsConf) throws IOException {
|
ClientConfiguration awsConf) throws IOException {
|
||||||
|
|
|
@ -30,6 +30,7 @@ import java.util.Comparator;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.Objects;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.CompletableFuture;
|
import java.util.concurrent.CompletableFuture;
|
||||||
|
|
||||||
|
@ -53,8 +54,11 @@ import org.apache.hadoop.fs.PathIOException;
|
||||||
import org.apache.hadoop.fs.UploadHandle;
|
import org.apache.hadoop.fs.UploadHandle;
|
||||||
import org.apache.hadoop.fs.impl.AbstractMultipartUploader;
|
import org.apache.hadoop.fs.impl.AbstractMultipartUploader;
|
||||||
import org.apache.hadoop.fs.s3a.WriteOperations;
|
import org.apache.hadoop.fs.s3a.WriteOperations;
|
||||||
import org.apache.hadoop.fs.s3a.impl.statistics.S3AMultipartUploaderStatistics;
|
import org.apache.hadoop.fs.s3a.statistics.S3AMultipartUploaderStatistics;
|
||||||
import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
|
import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatistics;
|
||||||
|
|
||||||
|
import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToString;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* MultipartUploader for S3AFileSystem. This uses the S3 multipart
|
* MultipartUploader for S3AFileSystem. This uses the S3 multipart
|
||||||
|
@ -102,7 +106,7 @@ class S3AMultipartUploader extends AbstractMultipartUploader {
|
||||||
this.builder = builder;
|
this.builder = builder;
|
||||||
this.writeOperations = writeOperations;
|
this.writeOperations = writeOperations;
|
||||||
this.context = context;
|
this.context = context;
|
||||||
this.statistics = statistics;
|
this.statistics = Objects.requireNonNull(statistics);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -113,6 +117,22 @@ class S3AMultipartUploader extends AbstractMultipartUploader {
|
||||||
super.close();
|
super.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public IOStatistics getIOStatistics() {
|
||||||
|
return statistics.getIOStatistics();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
final StringBuilder sb = new StringBuilder(
|
||||||
|
"S3AMultipartUploader{");
|
||||||
|
sb.append("base=").append(getBasePath());
|
||||||
|
sb.append("; statistics=").append(
|
||||||
|
ioStatisticsToString(statistics.getIOStatistics()));
|
||||||
|
sb.append('}');
|
||||||
|
return sb.toString();
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Retrieve the operation state; create one on demand if needed
|
* Retrieve the operation state; create one on demand if needed
|
||||||
* <i>and there has been no unsuccessful attempt to create one.</i>
|
* <i>and there has been no unsuccessful attempt to create one.</i>
|
||||||
|
|
|
@ -25,7 +25,7 @@ import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.fs.impl.MultipartUploaderBuilderImpl;
|
import org.apache.hadoop.fs.impl.MultipartUploaderBuilderImpl;
|
||||||
import org.apache.hadoop.fs.s3a.S3AFileSystem;
|
import org.apache.hadoop.fs.s3a.S3AFileSystem;
|
||||||
import org.apache.hadoop.fs.s3a.WriteOperations;
|
import org.apache.hadoop.fs.s3a.WriteOperations;
|
||||||
import org.apache.hadoop.fs.s3a.impl.statistics.S3AMultipartUploaderStatistics;
|
import org.apache.hadoop.fs.s3a.statistics.S3AMultipartUploaderStatistics;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Builder for S3A multipart uploaders.
|
* Builder for S3A multipart uploaders.
|
||||||
|
|
|
@ -33,9 +33,9 @@ import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.fs.s3a.Invoker;
|
import org.apache.hadoop.fs.s3a.Invoker;
|
||||||
import org.apache.hadoop.fs.s3a.S3AFileStatus;
|
import org.apache.hadoop.fs.s3a.S3AFileStatus;
|
||||||
import org.apache.hadoop.fs.s3a.S3AInputPolicy;
|
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.S3AStorageStatistics;
|
||||||
import org.apache.hadoop.fs.s3a.Statistic;
|
import org.apache.hadoop.fs.s3a.Statistic;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
|
||||||
import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider;
|
import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider;
|
||||||
import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
|
import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
|
@ -88,7 +88,8 @@ public class StoreContext {
|
||||||
private final Invoker invoker;
|
private final Invoker invoker;
|
||||||
|
|
||||||
/** Instrumentation and statistics. */
|
/** Instrumentation and statistics. */
|
||||||
private final S3AInstrumentation instrumentation;
|
private final S3AStatisticsContext instrumentation;
|
||||||
|
|
||||||
private final S3AStorageStatistics storageStatistics;
|
private final S3AStorageStatistics storageStatistics;
|
||||||
|
|
||||||
/** Seek policy. */
|
/** Seek policy. */
|
||||||
|
@ -129,7 +130,7 @@ public class StoreContext {
|
||||||
final ListeningExecutorService executor,
|
final ListeningExecutorService executor,
|
||||||
final int executorCapacity,
|
final int executorCapacity,
|
||||||
final Invoker invoker,
|
final Invoker invoker,
|
||||||
final S3AInstrumentation instrumentation,
|
final S3AStatisticsContext instrumentation,
|
||||||
final S3AStorageStatistics storageStatistics,
|
final S3AStorageStatistics storageStatistics,
|
||||||
final S3AInputPolicy inputPolicy,
|
final S3AInputPolicy inputPolicy,
|
||||||
final ChangeDetectionPolicy changeDetectionPolicy,
|
final ChangeDetectionPolicy changeDetectionPolicy,
|
||||||
|
@ -186,7 +187,12 @@ public class StoreContext {
|
||||||
return invoker;
|
return invoker;
|
||||||
}
|
}
|
||||||
|
|
||||||
public S3AInstrumentation getInstrumentation() {
|
/**
|
||||||
|
* Get the statistics context for this StoreContext.
|
||||||
|
* @return the statistics context this store context was created
|
||||||
|
* with.
|
||||||
|
*/
|
||||||
|
public S3AStatisticsContext getInstrumentation() {
|
||||||
return instrumentation;
|
return instrumentation;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -268,7 +274,6 @@ public class StoreContext {
|
||||||
*/
|
*/
|
||||||
public void incrementStatistic(Statistic statistic, long count) {
|
public void incrementStatistic(Statistic statistic, long count) {
|
||||||
instrumentation.incrementCounter(statistic, count);
|
instrumentation.incrementCounter(statistic, count);
|
||||||
storageStatistics.incrementCounter(statistic, count);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -25,8 +25,8 @@ import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListeningE
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.s3a.Invoker;
|
import org.apache.hadoop.fs.s3a.Invoker;
|
||||||
import org.apache.hadoop.fs.s3a.S3AInputPolicy;
|
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.S3AStorageStatistics;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
|
||||||
import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider;
|
import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider;
|
||||||
import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
|
import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
|
@ -52,7 +52,7 @@ public class StoreContextBuilder {
|
||||||
|
|
||||||
private Invoker invoker;
|
private Invoker invoker;
|
||||||
|
|
||||||
private S3AInstrumentation instrumentation;
|
private S3AStatisticsContext instrumentation;
|
||||||
|
|
||||||
private S3AStorageStatistics storageStatistics;
|
private S3AStorageStatistics storageStatistics;
|
||||||
|
|
||||||
|
@ -113,7 +113,7 @@ public class StoreContextBuilder {
|
||||||
}
|
}
|
||||||
|
|
||||||
public StoreContextBuilder setInstrumentation(
|
public StoreContextBuilder setInstrumentation(
|
||||||
final S3AInstrumentation instr) {
|
final S3AStatisticsContext instr) {
|
||||||
this.instrumentation = instr;
|
this.instrumentation = instr;
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.hadoop.fs.s3a.s3guard;
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.InterruptedIOException;
|
import java.io.InterruptedIOException;
|
||||||
|
import java.io.UncheckedIOException;
|
||||||
import java.net.URI;
|
import java.net.URI;
|
||||||
import java.nio.file.AccessDeniedException;
|
import java.nio.file.AccessDeniedException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
@ -79,8 +80,8 @@ import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.fs.PathIOException;
|
import org.apache.hadoop.fs.PathIOException;
|
||||||
import org.apache.hadoop.fs.RemoteIterator;
|
import org.apache.hadoop.fs.RemoteIterator;
|
||||||
import org.apache.hadoop.fs.impl.FunctionsRaisingIOE;
|
import org.apache.hadoop.util.functional.CallableRaisingIOE;
|
||||||
import org.apache.hadoop.fs.impl.WrappedIOException;
|
import org.apache.hadoop.util.functional.RemoteIterators;
|
||||||
import org.apache.hadoop.fs.s3a.AWSCredentialProviderList;
|
import org.apache.hadoop.fs.s3a.AWSCredentialProviderList;
|
||||||
import org.apache.hadoop.fs.s3a.AWSServiceThrottledException;
|
import org.apache.hadoop.fs.s3a.AWSServiceThrottledException;
|
||||||
import org.apache.hadoop.fs.s3a.Constants;
|
import org.apache.hadoop.fs.s3a.Constants;
|
||||||
|
@ -450,7 +451,8 @@ public class DynamoDBMetadataStore implements MetadataStore,
|
||||||
owner = fs;
|
owner = fs;
|
||||||
conf = owner.getConf();
|
conf = owner.getConf();
|
||||||
StoreContext context = owner.createStoreContext();
|
StoreContext context = owner.createStoreContext();
|
||||||
instrumentation = context.getInstrumentation().getS3GuardInstrumentation();
|
instrumentation = context.getInstrumentation()
|
||||||
|
.getS3GuardInstrumentation();
|
||||||
username = context.getUsername();
|
username = context.getUsername();
|
||||||
executor = context.createThrottledExecutor();
|
executor = context.createThrottledExecutor();
|
||||||
ttlTimeProvider = Preconditions.checkNotNull(
|
ttlTimeProvider = Preconditions.checkNotNull(
|
||||||
|
@ -638,8 +640,9 @@ public class DynamoDBMetadataStore implements MetadataStore,
|
||||||
LOG.debug("Subtree path {} is deleted; this will be a no-op", path);
|
LOG.debug("Subtree path {} is deleted; this will be a no-op", path);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
deleteEntries(new InternalIterators.PathFromRemoteStatusIterator(
|
deleteEntries(RemoteIterators.mappingRemoteIterator(
|
||||||
new DescendantsIterator(this, meta)),
|
new DescendantsIterator(this, meta),
|
||||||
|
FileStatus::getPath),
|
||||||
operationState);
|
operationState);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -648,8 +651,7 @@ public class DynamoDBMetadataStore implements MetadataStore,
|
||||||
public void deletePaths(Collection<Path> paths,
|
public void deletePaths(Collection<Path> paths,
|
||||||
final BulkOperationState operationState)
|
final BulkOperationState operationState)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
deleteEntries(
|
deleteEntries(RemoteIterators.remoteIteratorFromIterable(paths),
|
||||||
new InternalIterators.RemoteIteratorFromIterator<>(paths.iterator()),
|
|
||||||
operationState);
|
operationState);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -826,7 +828,7 @@ public class DynamoDBMetadataStore implements MetadataStore,
|
||||||
for (Item item : wrapWithRetries(items)) {
|
for (Item item : wrapWithRetries(items)) {
|
||||||
metas.add(itemToPathMetadata(item, username));
|
metas.add(itemToPathMetadata(item, username));
|
||||||
}
|
}
|
||||||
} catch (WrappedIOException e) {
|
} catch (UncheckedIOException e) {
|
||||||
// failure in the iterators; unwrap.
|
// failure in the iterators; unwrap.
|
||||||
throw e.getCause();
|
throw e.getCause();
|
||||||
}
|
}
|
||||||
|
@ -1634,7 +1636,7 @@ public class DynamoDBMetadataStore implements MetadataStore,
|
||||||
Set<Path> clearedParentPathSet = new HashSet<>();
|
Set<Path> clearedParentPathSet = new HashSet<>();
|
||||||
// declare the operation to delete a batch as a function so
|
// declare the operation to delete a batch as a function so
|
||||||
// as to keep the code consistent across multiple uses.
|
// as to keep the code consistent across multiple uses.
|
||||||
FunctionsRaisingIOE.CallableRaisingIOE<Void> deleteBatchOperation =
|
CallableRaisingIOE<Void> deleteBatchOperation =
|
||||||
() -> {
|
() -> {
|
||||||
// lowest path entries get deleted first.
|
// lowest path entries get deleted first.
|
||||||
deletionBatch.sort(PathOrderComparators.TOPMOST_PATH_LAST);
|
deletionBatch.sort(PathOrderComparators.TOPMOST_PATH_LAST);
|
||||||
|
|
|
@ -1,91 +0,0 @@
|
||||||
/*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
|
||||||
* or more contributor license agreements. See the NOTICE file
|
|
||||||
* distributed with this work for additional information
|
|
||||||
* regarding copyright ownership. The ASF licenses this file
|
|
||||||
* to you under the Apache License, Version 2.0 (the
|
|
||||||
* "License"); you may not use this file except in compliance
|
|
||||||
* with the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
|
|
||||||
package org.apache.hadoop.fs.s3a.s3guard;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.Iterator;
|
|
||||||
|
|
||||||
import org.apache.hadoop.fs.Path;
|
|
||||||
import org.apache.hadoop.fs.RemoteIterator;
|
|
||||||
import org.apache.hadoop.fs.s3a.S3AFileStatus;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Internal iterators.
|
|
||||||
*/
|
|
||||||
final class InternalIterators {
|
|
||||||
|
|
||||||
private InternalIterators() {
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* From a remote status iterator, build a path iterator.
|
|
||||||
*/
|
|
||||||
static final class PathFromRemoteStatusIterator implements
|
|
||||||
RemoteIterator<Path> {
|
|
||||||
|
|
||||||
private final RemoteIterator<S3AFileStatus> source;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Construct.
|
|
||||||
* @param source source iterator.
|
|
||||||
*/
|
|
||||||
PathFromRemoteStatusIterator(final RemoteIterator<S3AFileStatus> source) {
|
|
||||||
this.source = source;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean hasNext() throws IOException {
|
|
||||||
return source.hasNext();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Path next() throws IOException {
|
|
||||||
return source.next().getPath();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* From a classic java.util.Iterator, build a Hadoop remote iterator.
|
|
||||||
* @param <T> type of iterated value.
|
|
||||||
*/
|
|
||||||
static final class RemoteIteratorFromIterator<T> implements
|
|
||||||
RemoteIterator<T> {
|
|
||||||
|
|
||||||
private final Iterator<T> source;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Construct.
|
|
||||||
* @param source source iterator.
|
|
||||||
*/
|
|
||||||
RemoteIteratorFromIterator(final Iterator<T> source) {
|
|
||||||
this.source = source;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean hasNext() {
|
|
||||||
return source.hasNext();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public T next() {
|
|
||||||
return source.next();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
|
@ -19,9 +19,9 @@
|
||||||
package org.apache.hadoop.fs.s3a.s3guard;
|
package org.apache.hadoop.fs.s3a.s3guard;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.io.UncheckedIOException;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
|
|
||||||
import org.apache.hadoop.fs.impl.WrappedIOException;
|
|
||||||
import org.apache.hadoop.fs.s3a.Invoker;
|
import org.apache.hadoop.fs.s3a.Invoker;
|
||||||
import org.apache.hadoop.fs.s3a.Retries;
|
import org.apache.hadoop.fs.s3a.Retries;
|
||||||
|
|
||||||
|
@ -88,7 +88,7 @@ class RetryingCollection<T> implements Iterable<T> {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* {@inheritDoc}.
|
* {@inheritDoc}.
|
||||||
* @throws WrappedIOException for IO failure, including throttling.
|
* @throws UncheckedIOException for IO failure, including throttling.
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
@Retries.RetryTranslated
|
@Retries.RetryTranslated
|
||||||
|
@ -100,13 +100,13 @@ class RetryingCollection<T> implements Iterable<T> {
|
||||||
true,
|
true,
|
||||||
iterator::hasNext);
|
iterator::hasNext);
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
throw new WrappedIOException(e);
|
throw new UncheckedIOException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* {@inheritDoc}.
|
* {@inheritDoc}.
|
||||||
* @throws WrappedIOException for IO failure, including throttling.
|
* @throws UncheckedIOException for IO failure, including throttling.
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
@Retries.RetryTranslated
|
@Retries.RetryTranslated
|
||||||
|
@ -118,7 +118,7 @@ class RetryingCollection<T> implements Iterable<T> {
|
||||||
true,
|
true,
|
||||||
iterator::next);
|
iterator::next);
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
throw new WrappedIOException(e);
|
throw new UncheckedIOException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -39,9 +39,9 @@ import org.apache.hadoop.fs.FSExceptionMessages;
|
||||||
import org.apache.hadoop.fs.FSInputStream;
|
import org.apache.hadoop.fs.FSInputStream;
|
||||||
import org.apache.hadoop.fs.PathIOException;
|
import org.apache.hadoop.fs.PathIOException;
|
||||||
import org.apache.hadoop.fs.s3a.Retries;
|
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.S3AReadOpContext;
|
||||||
import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
|
import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
|
||||||
import org.apache.hadoop.io.IOUtils;
|
import org.apache.hadoop.io.IOUtils;
|
||||||
|
|
||||||
import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull;
|
import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull;
|
||||||
|
@ -103,7 +103,7 @@ public class SelectInputStream extends FSInputStream implements
|
||||||
|
|
||||||
private final S3AReadOpContext readContext;
|
private final S3AReadOpContext readContext;
|
||||||
|
|
||||||
private final S3AInstrumentation.InputStreamStatistics streamStatistics;
|
private final S3AInputStreamStatistics streamStatistics;
|
||||||
|
|
||||||
private long readahead;
|
private long readahead;
|
||||||
|
|
||||||
|
@ -130,7 +130,7 @@ public class SelectInputStream extends FSInputStream implements
|
||||||
this.uri = "s3a://" + this.bucket + "/" + this.key;
|
this.uri = "s3a://" + this.bucket + "/" + this.key;
|
||||||
this.readContext = readContext;
|
this.readContext = readContext;
|
||||||
this.readahead = readContext.getReadahead();
|
this.readahead = readContext.getReadahead();
|
||||||
this.streamStatistics = readContext.getInstrumentation()
|
this.streamStatistics = readContext.getS3AStatisticsContext()
|
||||||
.newInputStreamStatistics();
|
.newInputStreamStatistics();
|
||||||
SelectRecordsInputStream stream = once(
|
SelectRecordsInputStream stream = once(
|
||||||
"S3 Select",
|
"S3 Select",
|
||||||
|
@ -204,7 +204,7 @@ public class SelectInputStream extends FSInputStream implements
|
||||||
long skipped = once("skip", uri, () -> wrappedStream.skip(n));
|
long skipped = once("skip", uri, () -> wrappedStream.skip(n));
|
||||||
pos.addAndGet(skipped);
|
pos.addAndGet(skipped);
|
||||||
// treat as a forward skip for stats
|
// treat as a forward skip for stats
|
||||||
streamStatistics.seekForwards(skipped);
|
streamStatistics.seekForwards(skipped, skipped);
|
||||||
return skipped;
|
return skipped;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -331,7 +331,7 @@ public class SelectInputStream extends FSInputStream implements
|
||||||
bytesSkipped++;
|
bytesSkipped++;
|
||||||
}
|
}
|
||||||
// read has finished.
|
// read has finished.
|
||||||
streamStatistics.seekForwards(bytesSkipped);
|
streamStatistics.seekForwards(bytesSkipped, bytesSkipped);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -428,7 +428,7 @@ public class SelectInputStream extends FSInputStream implements
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
@InterfaceStability.Unstable
|
@InterfaceStability.Unstable
|
||||||
public S3AInstrumentation.InputStreamStatistics getS3AStreamStatistics() {
|
public S3AInputStreamStatistics getS3AStreamStatistics() {
|
||||||
return streamStatistics;
|
return streamStatistics;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,137 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.fs.s3a.statistics;
|
||||||
|
|
||||||
|
import java.io.Closeable;
|
||||||
|
import java.time.Duration;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Block output stream statistics.
|
||||||
|
*/
|
||||||
|
public interface BlockOutputStreamStatistics extends Closeable,
|
||||||
|
S3AStatisticInterface {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Block is queued for upload.
|
||||||
|
* @param blockSize block size.
|
||||||
|
*/
|
||||||
|
void blockUploadQueued(int blockSize);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Queued block has been scheduled for upload.
|
||||||
|
* @param timeInQueue time in the queue.
|
||||||
|
* @param blockSize block size.
|
||||||
|
*/
|
||||||
|
void blockUploadStarted(Duration timeInQueue, int blockSize);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A block upload has completed. Duration excludes time in the queue.
|
||||||
|
* @param timeSinceUploadStarted time in since the transfer began.
|
||||||
|
* @param blockSize block size
|
||||||
|
*/
|
||||||
|
void blockUploadCompleted(Duration timeSinceUploadStarted, int blockSize);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A block upload has failed. Duration excludes time in the queue.
|
||||||
|
* <p>
|
||||||
|
* A final transfer completed event is still expected, so this
|
||||||
|
* does not decrement the active block counter.
|
||||||
|
* </p>
|
||||||
|
* @param timeSinceUploadStarted time in since the transfer began.
|
||||||
|
* @param blockSize block size
|
||||||
|
*/
|
||||||
|
void blockUploadFailed(Duration timeSinceUploadStarted, int blockSize);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Intermediate report of bytes uploaded.
|
||||||
|
* @param byteCount bytes uploaded
|
||||||
|
*/
|
||||||
|
void bytesTransferred(long byteCount);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Note exception in a multipart complete.
|
||||||
|
* @param count count of exceptions
|
||||||
|
*/
|
||||||
|
void exceptionInMultipartComplete(int count);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Note an exception in a multipart abort.
|
||||||
|
*/
|
||||||
|
void exceptionInMultipartAbort();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the number of bytes pending upload.
|
||||||
|
* @return the number of bytes in the pending upload state.
|
||||||
|
*/
|
||||||
|
long getBytesPendingUpload();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Data has been uploaded to be committed in a subsequent operation;
|
||||||
|
* to be called at the end of the write.
|
||||||
|
* @param size size in bytes
|
||||||
|
*/
|
||||||
|
void commitUploaded(long size);
|
||||||
|
|
||||||
|
int getBlocksAllocated();
|
||||||
|
|
||||||
|
int getBlocksReleased();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get counters of blocks actively allocated; may be inaccurate
|
||||||
|
* if the numbers change during the (non-synchronized) calculation.
|
||||||
|
* @return the number of actively allocated blocks.
|
||||||
|
*/
|
||||||
|
int getBlocksActivelyAllocated();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Record bytes written.
|
||||||
|
* @param count number of bytes
|
||||||
|
*/
|
||||||
|
void writeBytes(long count);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the current count of bytes written.
|
||||||
|
* @return the counter value.
|
||||||
|
*/
|
||||||
|
long getBytesWritten();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A block has been allocated.
|
||||||
|
*/
|
||||||
|
void blockAllocated();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A block has been released.
|
||||||
|
*/
|
||||||
|
void blockReleased();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the value of a counter.
|
||||||
|
* @param name counter name
|
||||||
|
* @return the value or null if no matching counter was found.
|
||||||
|
*/
|
||||||
|
Long lookupCounterValue(String name);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the value of a gauge.
|
||||||
|
* @param name gauge name
|
||||||
|
* @return the value or null if no matching gauge was found.
|
||||||
|
*/
|
||||||
|
Long lookupGaugeValue(String name);
|
||||||
|
}
|
|
@ -0,0 +1,36 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.fs.s3a.statistics;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Interface for change tracking statistics.
|
||||||
|
*/
|
||||||
|
public interface ChangeTrackerStatistics {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A version mismatch was detected.
|
||||||
|
*/
|
||||||
|
void versionMismatchError();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* How many version mismatches have occurred.
|
||||||
|
* @return counter of mismatches.
|
||||||
|
*/
|
||||||
|
long getVersionMismatches();
|
||||||
|
}
|
|
@ -0,0 +1,66 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.fs.s3a.statistics;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Statistics for S3A committers.
|
||||||
|
*/
|
||||||
|
public interface CommitterStatistics
|
||||||
|
extends S3AStatisticInterface {
|
||||||
|
|
||||||
|
/** A commit has been created. */
|
||||||
|
void commitCreated();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Data has been uploaded to be committed in a subsequent operation.
|
||||||
|
* @param size size in bytes
|
||||||
|
*/
|
||||||
|
void commitUploaded(long size);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A commit has been completed.
|
||||||
|
* @param size size in bytes
|
||||||
|
*/
|
||||||
|
void commitCompleted(long size);
|
||||||
|
|
||||||
|
/** A commit has been aborted. */
|
||||||
|
void commitAborted();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A commit was reverted.
|
||||||
|
*/
|
||||||
|
void commitReverted();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A commit failed.
|
||||||
|
*/
|
||||||
|
void commitFailed();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Note that a task has completed.
|
||||||
|
* @param success success flag
|
||||||
|
*/
|
||||||
|
void taskCompleted(boolean success);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Note that a job has completed.
|
||||||
|
* @param success success flag
|
||||||
|
*/
|
||||||
|
void jobCompleted(boolean success);
|
||||||
|
}
|
|
@ -0,0 +1,72 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.fs.s3a.statistics;
|
||||||
|
|
||||||
|
import java.time.Duration;
|
||||||
|
|
||||||
|
import org.apache.hadoop.fs.s3a.Statistic;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This is the foundational API for collecting S3A statistics.
|
||||||
|
*/
|
||||||
|
public interface CountersAndGauges {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Increment a specific counter.
|
||||||
|
* No-op if not defined.
|
||||||
|
* @param op operation
|
||||||
|
* @param count increment value
|
||||||
|
*/
|
||||||
|
void incrementCounter(Statistic op, long count);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Increment a specific gauge.
|
||||||
|
* No-op if not defined.
|
||||||
|
* @param op operation
|
||||||
|
* @param count increment value
|
||||||
|
* @throws ClassCastException if the metric is of the wrong type
|
||||||
|
*/
|
||||||
|
void incrementGauge(Statistic op, long count);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Decrement a specific gauge.
|
||||||
|
* No-op if not defined.
|
||||||
|
* @param op operation
|
||||||
|
* @param count increment value
|
||||||
|
* @throws ClassCastException if the metric is of the wrong type
|
||||||
|
*/
|
||||||
|
void decrementGauge(Statistic op, long count);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Add a value to a quantiles statistic. No-op if the quantile
|
||||||
|
* isn't found.
|
||||||
|
* @param op operation to look up.
|
||||||
|
* @param value value to add.
|
||||||
|
* @throws ClassCastException if the metric is not a Quantiles.
|
||||||
|
*/
|
||||||
|
void addValueToQuantiles(Statistic op, long value);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Record a duration.
|
||||||
|
* @param op operation
|
||||||
|
* @param success was the operation a success?
|
||||||
|
* @param duration how long did it take
|
||||||
|
*/
|
||||||
|
void recordDuration(Statistic op, boolean success, Duration duration);
|
||||||
|
}
|
|
@ -0,0 +1,28 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.fs.s3a.statistics;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Instrumentation exported to for S3A Delegation Token support.
|
||||||
|
*/
|
||||||
|
public interface DelegationTokenStatistics extends S3AStatisticInterface {
|
||||||
|
|
||||||
|
/** A token has been issued. */
|
||||||
|
void tokenIssued();
|
||||||
|
}
|
|
@ -0,0 +1,191 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.fs.s3a.statistics;
|
||||||
|
|
||||||
|
import org.apache.hadoop.fs.statistics.DurationTracker;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Statistics updated by a
|
||||||
|
* {@link org.apache.hadoop.fs.s3a.S3AInputStream} during its use.
|
||||||
|
* It also contains getters for tests.
|
||||||
|
*/
|
||||||
|
public interface S3AInputStreamStatistics extends AutoCloseable,
|
||||||
|
S3AStatisticInterface {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Seek backwards, incrementing the seek and backward seek counters.
|
||||||
|
* @param negativeOffset how far was the seek?
|
||||||
|
* This is expected to be negative.
|
||||||
|
*/
|
||||||
|
void seekBackwards(long negativeOffset);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Record a forward seek, adding a seek operation, a forward
|
||||||
|
* seek operation, and any bytes skipped.
|
||||||
|
* @param skipped bytes moved forward in stream
|
||||||
|
* @param bytesReadInSeek number of bytes skipped by reading from the stream.
|
||||||
|
* If the seek was implemented by a close + reopen, set this to zero.
|
||||||
|
*/
|
||||||
|
void seekForwards(long skipped, long bytesReadInSeek);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The inner stream was opened.
|
||||||
|
* The return value is used in the input stream to decide whether it is
|
||||||
|
* the initial vs later count.
|
||||||
|
* @return the previous count or zero if this is the first opening.
|
||||||
|
*/
|
||||||
|
long streamOpened();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The inner stream was closed.
|
||||||
|
* @param abortedConnection flag to indicate the stream was aborted,
|
||||||
|
* rather than closed cleanly
|
||||||
|
* @param remainingInCurrentRequest the number of bytes remaining in
|
||||||
|
* the current request.
|
||||||
|
*/
|
||||||
|
void streamClose(boolean abortedConnection,
|
||||||
|
long remainingInCurrentRequest);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* An ignored stream read exception was received.
|
||||||
|
*/
|
||||||
|
void readException();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Increment the bytes read counter by the number of bytes;
|
||||||
|
* no-op if the argument is negative.
|
||||||
|
* @param bytes number of bytes read
|
||||||
|
*/
|
||||||
|
void bytesRead(long bytes);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A {@code read(byte[] buf, int off, int len)} operation has started.
|
||||||
|
* @param pos starting position of the read
|
||||||
|
* @param len length of bytes to read
|
||||||
|
*/
|
||||||
|
void readOperationStarted(long pos, long len);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A {@code PositionedRead.read(position, buffer, offset, length)}
|
||||||
|
* operation has just started.
|
||||||
|
* @param pos starting position of the read
|
||||||
|
* @param len length of bytes to read
|
||||||
|
*/
|
||||||
|
void readFullyOperationStarted(long pos, long len);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A read operation has completed.
|
||||||
|
* @param requested number of requested bytes
|
||||||
|
* @param actual the actual number of bytes
|
||||||
|
*/
|
||||||
|
void readOperationCompleted(int requested, int actual);
|
||||||
|
|
||||||
|
@Override
|
||||||
|
void close();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The input policy has been switched.
|
||||||
|
* @param updatedPolicy enum value of new policy.
|
||||||
|
*/
|
||||||
|
void inputPolicySet(int updatedPolicy);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get a reference to the change tracker statistics for this
|
||||||
|
* stream.
|
||||||
|
* @return a reference to the change tracker statistics
|
||||||
|
*/
|
||||||
|
ChangeTrackerStatistics getChangeTrackerStatistics();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A stream {@code unbuffer()} call has been made.
|
||||||
|
*/
|
||||||
|
void unbuffered();
|
||||||
|
|
||||||
|
long getCloseOperations();
|
||||||
|
|
||||||
|
long getClosed();
|
||||||
|
|
||||||
|
long getAborted();
|
||||||
|
|
||||||
|
long getForwardSeekOperations();
|
||||||
|
|
||||||
|
long getBackwardSeekOperations();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The bytes read in read() operations.
|
||||||
|
* @return the number of bytes returned to the caller.
|
||||||
|
*/
|
||||||
|
long getBytesRead();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The total number of bytes read, including
|
||||||
|
* all read and discarded when closing streams
|
||||||
|
* or skipped during seek calls.
|
||||||
|
* @return the total number of bytes read from
|
||||||
|
* S3.
|
||||||
|
*/
|
||||||
|
long getTotalBytesRead();
|
||||||
|
|
||||||
|
long getBytesSkippedOnSeek();
|
||||||
|
|
||||||
|
long getBytesBackwardsOnSeek();
|
||||||
|
|
||||||
|
long getBytesReadInClose();
|
||||||
|
|
||||||
|
long getBytesDiscardedInAbort();
|
||||||
|
|
||||||
|
long getOpenOperations();
|
||||||
|
|
||||||
|
long getSeekOperations();
|
||||||
|
|
||||||
|
long getReadExceptions();
|
||||||
|
|
||||||
|
long getReadOperations();
|
||||||
|
|
||||||
|
long getReadFullyOperations();
|
||||||
|
|
||||||
|
long getReadsIncomplete();
|
||||||
|
|
||||||
|
long getPolicySetCount();
|
||||||
|
|
||||||
|
long getVersionMismatches();
|
||||||
|
|
||||||
|
long getInputPolicy();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the value of a counter.
|
||||||
|
* @param name counter name
|
||||||
|
* @return the value or null if no matching counter was found.
|
||||||
|
*/
|
||||||
|
Long lookupCounterValue(String name);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the value of a gauge.
|
||||||
|
* @param name gauge name
|
||||||
|
* @return the value or null if no matching gauge was found.
|
||||||
|
*/
|
||||||
|
Long lookupGaugeValue(String name);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Initiate a GET request.
|
||||||
|
* @return duration tracker;
|
||||||
|
*/
|
||||||
|
DurationTracker initiateGetRequest();
|
||||||
|
|
||||||
|
}
|
|
@ -16,14 +16,17 @@
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
package org.apache.hadoop.fs.s3a.impl.statistics;
|
package org.apache.hadoop.fs.s3a.statistics;
|
||||||
|
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Statistics for the S3A multipart uploader.
|
* Statistics for the S3A multipart uploader.
|
||||||
|
* It is expected to act as the statistics source for
|
||||||
|
* the uploader.
|
||||||
*/
|
*/
|
||||||
public interface S3AMultipartUploaderStatistics extends Closeable {
|
public interface S3AMultipartUploaderStatistics extends Closeable,
|
||||||
|
S3AStatisticInterface {
|
||||||
|
|
||||||
void instantiated();
|
void instantiated();
|
||||||
|
|
|
@ -0,0 +1,32 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.fs.s3a.statistics;
|
||||||
|
|
||||||
|
import org.apache.hadoop.fs.statistics.DurationTrackerFactory;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Base Interface for S3A Statistics, as a way of managing
|
||||||
|
* them.
|
||||||
|
*/
|
||||||
|
public interface S3AStatisticInterface extends IOStatisticsSource,
|
||||||
|
DurationTrackerFactory {
|
||||||
|
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,69 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.fs.s3a.statistics;
|
||||||
|
|
||||||
|
import org.apache.hadoop.fs.s3a.s3guard.MetastoreInstrumentation;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This is the statistics context for ongoing operations in S3A.
|
||||||
|
*/
|
||||||
|
public interface S3AStatisticsContext extends CountersAndGauges {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the metastore instrumentation.
|
||||||
|
* @return an instance of the metastore statistics tracking.
|
||||||
|
*/
|
||||||
|
MetastoreInstrumentation getS3GuardInstrumentation();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a stream input statistics instance.
|
||||||
|
* @return the new instance
|
||||||
|
*/
|
||||||
|
S3AInputStreamStatistics newInputStreamStatistics();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a new instance of the committer statistics.
|
||||||
|
* @return a new committer statistics instance
|
||||||
|
*/
|
||||||
|
CommitterStatistics newCommitterStatistics();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a stream output statistics instance.
|
||||||
|
* @return the new instance
|
||||||
|
*/
|
||||||
|
BlockOutputStreamStatistics newOutputStreamStatistics();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a delegation token statistics instance.
|
||||||
|
* @return an instance of delegation token statistics
|
||||||
|
*/
|
||||||
|
DelegationTokenStatistics newDelegationTokenStatistics();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a StatisticsFromAwsSdk instance.
|
||||||
|
* @return an instance of StatisticsFromAwsSdk
|
||||||
|
*/
|
||||||
|
StatisticsFromAwsSdk newStatisticsFromAwsSdk();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creaet a multipart statistics collector.
|
||||||
|
* @return an instance
|
||||||
|
*/
|
||||||
|
S3AMultipartUploaderStatistics createMultipartUploaderStatistics();
|
||||||
|
}
|
|
@ -0,0 +1,46 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.fs.s3a.statistics;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Enum of statistic types.
|
||||||
|
*/
|
||||||
|
public enum StatisticTypeEnum {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Counter.
|
||||||
|
*/
|
||||||
|
TYPE_COUNTER,
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Duration.
|
||||||
|
*/
|
||||||
|
TYPE_DURATION,
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Gauge.
|
||||||
|
*/
|
||||||
|
TYPE_GAUGE,
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Quantile.
|
||||||
|
*/
|
||||||
|
TYPE_QUANTILE,
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,81 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.fs.s3a.statistics;
|
||||||
|
|
||||||
|
import java.time.Duration;
|
||||||
|
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.impl.AwsStatisticsCollector;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* interface to receive statistics events from the AWS SDK
|
||||||
|
* by way of {@link AwsStatisticsCollector}.
|
||||||
|
*/
|
||||||
|
public interface StatisticsFromAwsSdk {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Record a number of AWS requests.
|
||||||
|
* @param count number of events.
|
||||||
|
*/
|
||||||
|
void updateAwsRequestCount(long count);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Record a number of AWS request retries.
|
||||||
|
* @param count number of events.
|
||||||
|
*/
|
||||||
|
void updateAwsRetryCount(long count);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Record a number of throttle exceptions received.
|
||||||
|
* @param count number of events.
|
||||||
|
*/
|
||||||
|
void updateAwsThrottleExceptionsCount(long count);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Record how long a request took overall.
|
||||||
|
* @param duration duration of operation.
|
||||||
|
*/
|
||||||
|
void noteAwsRequestTime(Duration duration);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Record how long a request took to execute on the
|
||||||
|
* client.
|
||||||
|
* @param duration duration of operation.
|
||||||
|
*/
|
||||||
|
void noteAwsClientExecuteTime(Duration duration);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Record how long a request took to marshall into
|
||||||
|
* XML.
|
||||||
|
* @param duration duration of operation.
|
||||||
|
*/
|
||||||
|
void noteRequestMarshallTime(Duration duration);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Record how long a request took to sign, including
|
||||||
|
* any calls to EC2 credential endpoints.
|
||||||
|
* @param duration duration of operation.
|
||||||
|
*/
|
||||||
|
void noteRequestSigningTime(Duration duration);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Record how long it took to process the response.
|
||||||
|
* @param duration duration of operation.
|
||||||
|
*/
|
||||||
|
void noteResponseProcessingTime(Duration duration);
|
||||||
|
}
|
|
@ -0,0 +1,111 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.fs.s3a.statistics.impl;
|
||||||
|
|
||||||
|
import org.apache.hadoop.fs.statistics.DurationTracker;
|
||||||
|
import org.apache.hadoop.fs.statistics.DurationTrackerFactory;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
|
||||||
|
import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Base class for implementing IOStatistics sources in the S3 module.
|
||||||
|
* <p>
|
||||||
|
* A lot of the methods are very terse, because S3AInstrumentation has
|
||||||
|
* verbose methods of similar names; the short ones always
|
||||||
|
* refer to the inner class and not any superclass method.
|
||||||
|
* </p>
|
||||||
|
*/
|
||||||
|
public abstract class AbstractS3AStatisticsSource implements
|
||||||
|
IOStatisticsSource, DurationTrackerFactory {
|
||||||
|
|
||||||
|
private IOStatisticsStore ioStatistics;
|
||||||
|
|
||||||
|
protected AbstractS3AStatisticsSource() {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public IOStatisticsStore getIOStatistics() {
|
||||||
|
return ioStatistics;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Setter.
|
||||||
|
* this must be called in the subclass constructor with
|
||||||
|
* whatever
|
||||||
|
* @param statistics statistics to set
|
||||||
|
*/
|
||||||
|
protected void setIOStatistics(final IOStatisticsStore statistics) {
|
||||||
|
this.ioStatistics = statistics;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Increment a named counter by 1.
|
||||||
|
* @param name counter name
|
||||||
|
* @return the updated value or, if the counter is unknown: 0
|
||||||
|
*/
|
||||||
|
public long incCounter(String name) {
|
||||||
|
return incCounter(name, 1);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**DefaultS3ClientFactoryDefaultS3ClientFactory
|
||||||
|
* Increment a named counter by 1.
|
||||||
|
* @param name counter name
|
||||||
|
* @param value value to increment by
|
||||||
|
* @return the updated value or, if the counter is unknown: 0
|
||||||
|
*/
|
||||||
|
public long incCounter(String name, long value) {
|
||||||
|
return ioStatistics.incrementCounter(name, value);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* {@inheritDoc}
|
||||||
|
*/
|
||||||
|
public Long lookupCounterValue(final String name) {
|
||||||
|
return ioStatistics.counters().get(name);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* {@inheritDoc}
|
||||||
|
*/
|
||||||
|
public Long lookupGaugeValue(final String name) {
|
||||||
|
return ioStatistics.gauges().get(name);
|
||||||
|
}
|
||||||
|
|
||||||
|
public long incGauge(String name, long v) {
|
||||||
|
return ioStatistics.incrementGauge(name, v);
|
||||||
|
}
|
||||||
|
|
||||||
|
public long incGauge(String name) {
|
||||||
|
return incGauge(name, 1);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
final StringBuilder sb = new StringBuilder(
|
||||||
|
"AbstractS3AStatisticsSource{");
|
||||||
|
sb.append(ioStatistics);
|
||||||
|
sb.append('}');
|
||||||
|
return sb.toString();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DurationTracker trackDuration(final String key, final long count) {
|
||||||
|
return getIOStatistics().trackDuration(key, count);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,129 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.fs.s3a.statistics.impl;
|
||||||
|
|
||||||
|
import java.time.Duration;
|
||||||
|
import java.util.function.Consumer;
|
||||||
|
import java.util.function.LongConsumer;
|
||||||
|
|
||||||
|
import com.amazonaws.Request;
|
||||||
|
import com.amazonaws.Response;
|
||||||
|
import com.amazonaws.metrics.RequestMetricCollector;
|
||||||
|
import com.amazonaws.util.TimingInfo;
|
||||||
|
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.StatisticsFromAwsSdk;
|
||||||
|
|
||||||
|
import static com.amazonaws.util.AWSRequestMetrics.Field.ClientExecuteTime;
|
||||||
|
import static com.amazonaws.util.AWSRequestMetrics.Field.HttpClientRetryCount;
|
||||||
|
import static com.amazonaws.util.AWSRequestMetrics.Field.HttpRequestTime;
|
||||||
|
import static com.amazonaws.util.AWSRequestMetrics.Field.RequestCount;
|
||||||
|
import static com.amazonaws.util.AWSRequestMetrics.Field.RequestMarshallTime;
|
||||||
|
import static com.amazonaws.util.AWSRequestMetrics.Field.RequestSigningTime;
|
||||||
|
import static com.amazonaws.util.AWSRequestMetrics.Field.ResponseProcessingTime;
|
||||||
|
import static com.amazonaws.util.AWSRequestMetrics.Field.ThrottleException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Collect statistics from the AWS SDK and forward to an instance of
|
||||||
|
* {@link StatisticsFromAwsSdk} and thence into the S3A statistics.
|
||||||
|
* <p>
|
||||||
|
* See {@code com.facebook.presto.hive.s3.PrestoS3FileSystemMetricCollector}
|
||||||
|
* for the inspiration for this.
|
||||||
|
* <p>
|
||||||
|
* See {@code com.amazonaws.util.AWSRequestMetrics} for metric names.
|
||||||
|
*/
|
||||||
|
public class AwsStatisticsCollector extends RequestMetricCollector {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* final destination of updates.
|
||||||
|
*/
|
||||||
|
private final StatisticsFromAwsSdk collector;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Instantiate.
|
||||||
|
* @param collector final destination of updates
|
||||||
|
*/
|
||||||
|
public AwsStatisticsCollector(final StatisticsFromAwsSdk collector) {
|
||||||
|
this.collector = collector;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This is the callback from the AWS SDK where metrics
|
||||||
|
* can be collected.
|
||||||
|
* @param request AWS request
|
||||||
|
* @param response AWS response
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public void collectMetrics(
|
||||||
|
final Request<?> request,
|
||||||
|
final Response<?> response) {
|
||||||
|
|
||||||
|
TimingInfo timingInfo = request.getAWSRequestMetrics().getTimingInfo();
|
||||||
|
|
||||||
|
counter(timingInfo, HttpClientRetryCount.name(),
|
||||||
|
collector::updateAwsRetryCount);
|
||||||
|
counter(timingInfo, RequestCount.name(),
|
||||||
|
collector::updateAwsRequestCount);
|
||||||
|
counter(timingInfo, ThrottleException.name(),
|
||||||
|
collector::updateAwsThrottleExceptionsCount);
|
||||||
|
|
||||||
|
timing(timingInfo, ClientExecuteTime.name(),
|
||||||
|
collector::noteAwsClientExecuteTime);
|
||||||
|
timing(timingInfo, HttpRequestTime.name(),
|
||||||
|
collector::noteAwsRequestTime);
|
||||||
|
timing(timingInfo, RequestMarshallTime.name(),
|
||||||
|
collector::noteRequestMarshallTime);
|
||||||
|
timing(timingInfo, RequestSigningTime.name(),
|
||||||
|
collector::noteRequestSigningTime);
|
||||||
|
timing(timingInfo, ResponseProcessingTime.name(),
|
||||||
|
collector::noteResponseProcessingTime);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Process a timing.
|
||||||
|
* @param timingInfo timing info
|
||||||
|
* @param subMeasurementName sub measurement
|
||||||
|
* @param durationConsumer consumer
|
||||||
|
*/
|
||||||
|
private void timing(
|
||||||
|
TimingInfo timingInfo,
|
||||||
|
String subMeasurementName,
|
||||||
|
Consumer<Duration> durationConsumer) {
|
||||||
|
TimingInfo t1 = timingInfo.getSubMeasurement(subMeasurementName);
|
||||||
|
if (t1 != null && t1.getTimeTakenMillisIfKnown() != null) {
|
||||||
|
durationConsumer.accept(Duration.ofMillis(
|
||||||
|
t1.getTimeTakenMillisIfKnown().longValue()));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Process a counter.
|
||||||
|
* @param timingInfo timing info
|
||||||
|
* @param subMeasurementName sub measurement
|
||||||
|
* @param consumer consumer
|
||||||
|
*/
|
||||||
|
private void counter(
|
||||||
|
TimingInfo timingInfo,
|
||||||
|
String subMeasurementName,
|
||||||
|
LongConsumer consumer) {
|
||||||
|
Number n = timingInfo.getCounter(subMeasurementName);
|
||||||
|
if (n != null) {
|
||||||
|
consumer.accept(n.longValue());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,234 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.fs.s3a.statistics.impl;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
|
||||||
|
import java.time.Duration;
|
||||||
|
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.s3a.S3AInstrumentation;
|
||||||
|
import org.apache.hadoop.fs.s3a.Statistic;
|
||||||
|
import org.apache.hadoop.fs.s3a.s3guard.MetastoreInstrumentation;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.DelegationTokenStatistics;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.S3AMultipartUploaderStatistics;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.StatisticsFromAwsSdk;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* An S3A statistics context which is bonded to a
|
||||||
|
* S3AInstrumentation instance -inevitably that of an S3AFileSystem
|
||||||
|
* instance.
|
||||||
|
* <p>
|
||||||
|
* An interface is used to bind to the relevant fields, rather
|
||||||
|
* than have them passed in the constructor because some
|
||||||
|
* production code, specifically, DelegateToFileSystem,
|
||||||
|
* patches the protected field after initialization.
|
||||||
|
* </p>
|
||||||
|
* <p>
|
||||||
|
* All operations are passed through directly to that class.
|
||||||
|
* </p>
|
||||||
|
* <p>
|
||||||
|
* If an instance of FileSystem.Statistics is passed in, it
|
||||||
|
* will be used whenever input stream statistics are created -
|
||||||
|
* However, Internally always increments the statistics in the
|
||||||
|
* current thread.
|
||||||
|
* </p>
|
||||||
|
* <p>
|
||||||
|
* As a result, cross-thread IO will under-report.
|
||||||
|
* </p>
|
||||||
|
*
|
||||||
|
* This is addressed through the stream statistics classes
|
||||||
|
* only updating the stats in the close() call. Provided
|
||||||
|
* they are closed in the worker thread, all stats collected in
|
||||||
|
* helper threads will be included.
|
||||||
|
*/
|
||||||
|
public class BondedS3AStatisticsContext implements S3AStatisticsContext {
|
||||||
|
|
||||||
|
/** Source of statistics services. */
|
||||||
|
private final S3AFSStatisticsSource statisticsSource;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Instantiate.
|
||||||
|
* @param statisticsSource integration binding
|
||||||
|
*/
|
||||||
|
public BondedS3AStatisticsContext(
|
||||||
|
final S3AFSStatisticsSource statisticsSource) {
|
||||||
|
this.statisticsSource = statisticsSource;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the instrumentation from the FS integration.
|
||||||
|
* @return instrumentation instance.
|
||||||
|
*/
|
||||||
|
private S3AInstrumentation getInstrumentation() {
|
||||||
|
return statisticsSource.getInstrumentation();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The filesystem statistics: know this is thread-local.
|
||||||
|
* @return FS statistics.
|
||||||
|
*/
|
||||||
|
private FileSystem.Statistics getInstanceStatistics() {
|
||||||
|
return statisticsSource.getInstanceStatistics();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get a MetastoreInstrumentation getInstrumentation() instance for this
|
||||||
|
* context.
|
||||||
|
* @return the S3Guard getInstrumentation() point.
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public MetastoreInstrumentation getS3GuardInstrumentation() {
|
||||||
|
return getInstrumentation().getS3GuardInstrumentation();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a stream input statistics instance.
|
||||||
|
* The FileSystem.Statistics instance of the {@link #statisticsSource}
|
||||||
|
* is used as the reference to FileSystem statistics to update
|
||||||
|
* @return the new instance
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public S3AInputStreamStatistics newInputStreamStatistics() {
|
||||||
|
return getInstrumentation().newInputStreamStatistics(
|
||||||
|
statisticsSource.getInstanceStatistics());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a new instance of the committer statistics.
|
||||||
|
* @return a new committer statistics instance
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public CommitterStatistics newCommitterStatistics() {
|
||||||
|
return getInstrumentation().newCommitterStatistics();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a stream output statistics instance.
|
||||||
|
* @return the new instance
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public BlockOutputStreamStatistics newOutputStreamStatistics() {
|
||||||
|
return getInstrumentation()
|
||||||
|
.newOutputStreamStatistics(getInstanceStatistics());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Increment a specific counter.
|
||||||
|
* <p>
|
||||||
|
* No-op if not defined.
|
||||||
|
* @param op operation
|
||||||
|
* @param count increment value
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public void incrementCounter(Statistic op, long count) {
|
||||||
|
getInstrumentation().incrementCounter(op, count);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Increment a specific gauge.
|
||||||
|
* <p>
|
||||||
|
* No-op if not defined.
|
||||||
|
* @param op operation
|
||||||
|
* @param count increment value
|
||||||
|
* @throws ClassCastException if the metric is of the wrong type
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public void incrementGauge(Statistic op, long count) {
|
||||||
|
getInstrumentation().incrementGauge(op, count);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Decrement a specific gauge.
|
||||||
|
* <p>
|
||||||
|
* No-op if not defined.
|
||||||
|
* @param op operation
|
||||||
|
* @param count increment value
|
||||||
|
* @throws ClassCastException if the metric is of the wrong type
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public void decrementGauge(Statistic op, long count) {
|
||||||
|
getInstrumentation().decrementGauge(op, count);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Add a value to a quantiles statistic. No-op if the quantile
|
||||||
|
* isn't found.
|
||||||
|
* @param op operation to look up.
|
||||||
|
* @param value value to add.
|
||||||
|
* @throws ClassCastException if the metric is not a Quantiles.
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public void addValueToQuantiles(Statistic op, long value) {
|
||||||
|
getInstrumentation().addValueToQuantiles(op, value);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void recordDuration(final Statistic op,
|
||||||
|
final boolean success,
|
||||||
|
final Duration duration) {
|
||||||
|
getInstrumentation().recordDuration(op, success, duration);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a delegation token statistics instance.
|
||||||
|
* @return an instance of delegation token statistics
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public DelegationTokenStatistics newDelegationTokenStatistics() {
|
||||||
|
return getInstrumentation().newDelegationTokenStatistics();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public StatisticsFromAwsSdk newStatisticsFromAwsSdk() {
|
||||||
|
return new StatisticsFromAwsSdkImpl(getInstrumentation());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public S3AMultipartUploaderStatistics createMultipartUploaderStatistics() {
|
||||||
|
return new S3AMultipartUploaderStatisticsImpl(this::incrementCounter);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This is the interface which an integration source must implement
|
||||||
|
* for the integration.
|
||||||
|
* Note that the FileSystem.statistics field may be null for a class;
|
||||||
|
*/
|
||||||
|
public interface S3AFSStatisticsSource {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the S3A Instrumentation.
|
||||||
|
* @return a non-null instrumentation instance
|
||||||
|
*/
|
||||||
|
S3AInstrumentation getInstrumentation();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the statistics of the FS instance, shared across all threads.
|
||||||
|
* @return filesystem statistics
|
||||||
|
*/
|
||||||
|
@Nullable
|
||||||
|
FileSystem.Statistics getInstanceStatistics();
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,53 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.fs.s3a.statistics.impl;
|
||||||
|
|
||||||
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
|
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.ChangeTrackerStatistics;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A change tracker which increments an atomic long.
|
||||||
|
*/
|
||||||
|
public class CountingChangeTracker implements
|
||||||
|
ChangeTrackerStatistics {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The counter which is updated on every mismatch.
|
||||||
|
*/
|
||||||
|
private final AtomicLong counter;
|
||||||
|
|
||||||
|
public CountingChangeTracker(final AtomicLong counter) {
|
||||||
|
this.counter = counter;
|
||||||
|
}
|
||||||
|
|
||||||
|
public CountingChangeTracker() {
|
||||||
|
this(new AtomicLong());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void versionMismatchError() {
|
||||||
|
counter.incrementAndGet();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getVersionMismatches() {
|
||||||
|
return counter.get();
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,597 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.fs.s3a.statistics.impl;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.time.Duration;
|
||||||
|
|
||||||
|
import org.apache.hadoop.fs.s3a.Statistic;
|
||||||
|
import org.apache.hadoop.fs.s3a.s3guard.MetastoreInstrumentation;
|
||||||
|
import org.apache.hadoop.fs.s3a.s3guard.MetastoreInstrumentationImpl;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.ChangeTrackerStatistics;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.DelegationTokenStatistics;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.S3AMultipartUploaderStatistics;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.S3AStatisticInterface;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.StatisticsFromAwsSdk;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatistics;
|
||||||
|
import org.apache.hadoop.fs.statistics.DurationTracker;
|
||||||
|
|
||||||
|
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.emptyStatistics;
|
||||||
|
import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.stubDurationTracker;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Special statistics context, all of whose context operations are no-ops.
|
||||||
|
* All statistics instances it returns are also empty.
|
||||||
|
* <p>
|
||||||
|
* This class is here primarily to aid in testing, but it also allows for
|
||||||
|
* classes to require a non-empty statistics context in their constructor -yet
|
||||||
|
* still be instantiated without one bound to any filesystem.
|
||||||
|
*/
|
||||||
|
public final class EmptyS3AStatisticsContext implements S3AStatisticsContext {
|
||||||
|
|
||||||
|
public static final MetastoreInstrumentation
|
||||||
|
METASTORE_INSTRUMENTATION = new MetastoreInstrumentationImpl();
|
||||||
|
|
||||||
|
public static final S3AInputStreamStatistics
|
||||||
|
EMPTY_INPUT_STREAM_STATISTICS = new EmptyInputStreamStatistics();
|
||||||
|
|
||||||
|
public static final CommitterStatistics
|
||||||
|
EMPTY_COMMITTER_STATISTICS = new EmptyCommitterStatistics();
|
||||||
|
|
||||||
|
@SuppressWarnings("IOResourceOpenedButNotSafelyClosed")
|
||||||
|
public static final BlockOutputStreamStatistics
|
||||||
|
EMPTY_BLOCK_OUTPUT_STREAM_STATISTICS
|
||||||
|
= new EmptyBlockOutputStreamStatistics();
|
||||||
|
|
||||||
|
public static final DelegationTokenStatistics
|
||||||
|
EMPTY_DELEGATION_TOKEN_STATISTICS = new EmptyDelegationTokenStatistics();
|
||||||
|
|
||||||
|
public static final StatisticsFromAwsSdk
|
||||||
|
EMPTY_STATISTICS_FROM_AWS_SDK = new EmptyStatisticsFromAwsSdk();
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public MetastoreInstrumentation getS3GuardInstrumentation() {
|
||||||
|
return METASTORE_INSTRUMENTATION;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public S3AInputStreamStatistics newInputStreamStatistics() {
|
||||||
|
return EMPTY_INPUT_STREAM_STATISTICS;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public CommitterStatistics newCommitterStatistics() {
|
||||||
|
return EMPTY_COMMITTER_STATISTICS;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public BlockOutputStreamStatistics newOutputStreamStatistics() {
|
||||||
|
return EMPTY_BLOCK_OUTPUT_STREAM_STATISTICS;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DelegationTokenStatistics newDelegationTokenStatistics() {
|
||||||
|
return EMPTY_DELEGATION_TOKEN_STATISTICS;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public StatisticsFromAwsSdk newStatisticsFromAwsSdk() {
|
||||||
|
return EMPTY_STATISTICS_FROM_AWS_SDK;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public S3AMultipartUploaderStatistics createMultipartUploaderStatistics() {
|
||||||
|
return new EmptyMultipartUploaderStatistics();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void incrementCounter(final Statistic op, final long count) {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void incrementGauge(final Statistic op, final long count) {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void decrementGauge(final Statistic op, final long count) {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void addValueToQuantiles(final Statistic op, final long value) {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void recordDuration(final Statistic op,
|
||||||
|
final boolean success,
|
||||||
|
final Duration duration) {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Base class for all the empty implementations.
|
||||||
|
*/
|
||||||
|
private static class EmptyS3AStatisticImpl implements
|
||||||
|
S3AStatisticInterface {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Always return the stub duration tracker.
|
||||||
|
* @param key statistic key prefix
|
||||||
|
* @param count #of times to increment the matching counter in this
|
||||||
|
* operation.
|
||||||
|
* @return stub tracker.
|
||||||
|
*/
|
||||||
|
public DurationTracker trackDuration(String key, long count) {
|
||||||
|
return stubDurationTracker();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Input Stream statistics callbacks.
|
||||||
|
*/
|
||||||
|
private static final class EmptyInputStreamStatistics
|
||||||
|
extends EmptyS3AStatisticImpl
|
||||||
|
implements S3AInputStreamStatistics {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void seekBackwards(final long negativeOffset) {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void seekForwards(final long skipped,
|
||||||
|
final long bytesReadInSeek) {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long streamOpened() {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void streamClose(final boolean abortedConnection,
|
||||||
|
final long remainingInCurrentRequest) {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void readException() {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void bytesRead(final long bytes) {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void readOperationStarted(final long pos, final long len) {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void readFullyOperationStarted(final long pos, final long len) {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void readOperationCompleted(final int requested, final int actual) {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void inputPolicySet(final int updatedPolicy) {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void unbuffered() {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return an IO statistics instance.
|
||||||
|
* @return an empty IO statistics instance.
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public IOStatistics getIOStatistics() {
|
||||||
|
return emptyStatistics();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getCloseOperations() {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getClosed() {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getAborted() {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getForwardSeekOperations() {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getBackwardSeekOperations() {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getBytesRead() {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getTotalBytesRead() {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getBytesSkippedOnSeek() {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getBytesBackwardsOnSeek() {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getBytesReadInClose() {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getBytesDiscardedInAbort() {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getOpenOperations() {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getSeekOperations() {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getReadExceptions() {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getReadOperations() {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getReadFullyOperations() {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getReadsIncomplete() {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getPolicySetCount() {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getVersionMismatches() {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getInputPolicy() {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Long lookupCounterValue(final String name) {
|
||||||
|
return 0L;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Long lookupGaugeValue(final String name) {
|
||||||
|
return 0L;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ChangeTrackerStatistics getChangeTrackerStatistics() {
|
||||||
|
return new CountingChangeTracker();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DurationTracker initiateGetRequest() {
|
||||||
|
return stubDurationTracker();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Committer statistics.
|
||||||
|
*/
|
||||||
|
private static final class EmptyCommitterStatistics
|
||||||
|
extends EmptyS3AStatisticImpl
|
||||||
|
implements CommitterStatistics {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void commitCreated() {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void commitUploaded(final long size) {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void commitCompleted(final long size) {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void commitAborted() {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void commitReverted() {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void commitFailed() {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void taskCompleted(final boolean success) {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void jobCompleted(final boolean success) {
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static final class EmptyBlockOutputStreamStatistics
|
||||||
|
extends EmptyS3AStatisticImpl
|
||||||
|
implements BlockOutputStreamStatistics {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void blockUploadQueued(final int blockSize) {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void blockUploadStarted(final Duration timeInQueue,
|
||||||
|
final int blockSize) {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void blockUploadCompleted(final Duration timeSinceUploadStarted,
|
||||||
|
final int blockSize) {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void blockUploadFailed(final Duration timeSinceUploadStarted,
|
||||||
|
final int blockSize) {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void bytesTransferred(final long byteCount) {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void exceptionInMultipartComplete(final int count) {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void exceptionInMultipartAbort() {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getBytesPendingUpload() {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void commitUploaded(final long size) {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getBlocksAllocated() {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getBlocksReleased() {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getBlocksActivelyAllocated() {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public IOStatistics getIOStatistics() {
|
||||||
|
return emptyStatistics();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void blockAllocated() {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void blockReleased() {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeBytes(final long count) {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getBytesWritten() {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Long lookupCounterValue(final String name) {
|
||||||
|
return 0L;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Long lookupGaugeValue(final String name) {
|
||||||
|
return 0L;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() throws IOException {
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Delegation Token Statistics.
|
||||||
|
*/
|
||||||
|
private static final class EmptyDelegationTokenStatistics
|
||||||
|
extends EmptyS3AStatisticImpl
|
||||||
|
implements DelegationTokenStatistics {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void tokenIssued() {
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* AWS SDK Callbacks.
|
||||||
|
*/
|
||||||
|
private static final class EmptyStatisticsFromAwsSdk
|
||||||
|
implements StatisticsFromAwsSdk {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void updateAwsRequestCount(final long longValue) {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void updateAwsRetryCount(final long longValue) {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void updateAwsThrottleExceptionsCount(final long longValue) {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void noteAwsRequestTime(final Duration ofMillis) {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void noteAwsClientExecuteTime(final Duration ofMillis) {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void noteRequestMarshallTime(final Duration duration) {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void noteRequestSigningTime(final Duration duration) {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void noteResponseProcessingTime(final Duration duration) {
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Multipart Uploader.
|
||||||
|
*/
|
||||||
|
public static final class EmptyMultipartUploaderStatistics
|
||||||
|
implements S3AMultipartUploaderStatistics {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void instantiated() {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void uploadStarted() {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void partPut(final long lengthInBytes) {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void uploadCompleted() {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void uploadAborted() {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void abortUploadsUnderPathInvoked() {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() throws IOException {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DurationTracker trackDuration(final String key, final long count) {
|
||||||
|
return stubDurationTracker();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -16,30 +16,42 @@
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
package org.apache.hadoop.fs.s3a.impl.statistics;
|
package org.apache.hadoop.fs.s3a.statistics.impl;
|
||||||
|
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.Objects;
|
||||||
import java.util.function.BiConsumer;
|
import java.util.function.BiConsumer;
|
||||||
|
|
||||||
import org.apache.hadoop.fs.s3a.Statistic;
|
import org.apache.hadoop.fs.s3a.Statistic;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.S3AMultipartUploaderStatistics;
|
||||||
|
import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
|
||||||
|
|
||||||
import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_ABORT_UNDER_PATH_INVOKED;
|
|
||||||
import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_INSTANTIATED;
|
|
||||||
import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_PART_PUT;
|
|
||||||
import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_PART_PUT_BYTES;
|
|
||||||
import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_ABORTED;
|
import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_ABORTED;
|
||||||
|
import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_ABORT_UNDER_PATH_INVOKED;
|
||||||
import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_COMPLETED;
|
import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_COMPLETED;
|
||||||
|
import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_INSTANTIATED;
|
||||||
|
import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_PART_PUT;
|
||||||
|
import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_PART_PUT_BYTES;
|
||||||
import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_STARTED;
|
import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_STARTED;
|
||||||
|
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Implementation of the uploader statistics.
|
* Implementation of the uploader statistics.
|
||||||
|
* <p>
|
||||||
* This takes a function to update some counter and will update
|
* This takes a function to update some counter and will update
|
||||||
* this value when things change, so it can be bonded to arbitrary
|
* this value when things change, so it can be bonded to arbitrary
|
||||||
* statistic collectors.
|
* statistic collectors.
|
||||||
|
* </p>
|
||||||
|
* <p>
|
||||||
|
* Internally it builds a map of the relevant multipart statistics,
|
||||||
|
* increments as appropriate and serves this data back through
|
||||||
|
* the {@code IOStatisticsSource} API.
|
||||||
|
* </p>
|
||||||
*/
|
*/
|
||||||
public final class S3AMultipartUploaderStatisticsImpl implements
|
public final class S3AMultipartUploaderStatisticsImpl
|
||||||
S3AMultipartUploaderStatistics {
|
extends AbstractS3AStatisticsSource
|
||||||
|
implements S3AMultipartUploaderStatistics {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The operation to increment a counter/statistic by a value.
|
* The operation to increment a counter/statistic by a value.
|
||||||
|
@ -53,16 +65,28 @@ public final class S3AMultipartUploaderStatisticsImpl implements
|
||||||
*/
|
*/
|
||||||
public S3AMultipartUploaderStatisticsImpl(
|
public S3AMultipartUploaderStatisticsImpl(
|
||||||
final BiConsumer<Statistic, Long> incrementCallback) {
|
final BiConsumer<Statistic, Long> incrementCallback) {
|
||||||
this.incrementCallback = incrementCallback;
|
this.incrementCallback = Objects.requireNonNull(incrementCallback);
|
||||||
|
IOStatisticsStore st = iostatisticsStore()
|
||||||
|
.withCounters(
|
||||||
|
MULTIPART_UPLOAD_INSTANTIATED.getSymbol(),
|
||||||
|
MULTIPART_UPLOAD_PART_PUT.getSymbol(),
|
||||||
|
MULTIPART_UPLOAD_PART_PUT_BYTES.getSymbol(),
|
||||||
|
MULTIPART_UPLOAD_ABORTED.getSymbol(),
|
||||||
|
MULTIPART_UPLOAD_ABORT_UNDER_PATH_INVOKED.getSymbol(),
|
||||||
|
MULTIPART_UPLOAD_COMPLETED.getSymbol(),
|
||||||
|
MULTIPART_UPLOAD_STARTED.getSymbol())
|
||||||
|
.build();
|
||||||
|
setIOStatistics(st);
|
||||||
}
|
}
|
||||||
|
|
||||||
private void inc(Statistic op, long count) {
|
private void inc(Statistic op, long count) {
|
||||||
incrementCallback.accept(op, count);
|
incrementCallback.accept(op, count);
|
||||||
|
incCounter(op.getSymbol(), count);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void instantiated() {
|
public void instantiated() {
|
||||||
inc(MULTIPART_INSTANTIATED, 1);
|
inc(MULTIPART_UPLOAD_INSTANTIATED, 1);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -72,8 +96,8 @@ public final class S3AMultipartUploaderStatisticsImpl implements
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void partPut(final long lengthInBytes) {
|
public void partPut(final long lengthInBytes) {
|
||||||
inc(MULTIPART_PART_PUT, 1);
|
inc(MULTIPART_UPLOAD_PART_PUT, 1);
|
||||||
inc(MULTIPART_PART_PUT_BYTES, lengthInBytes);
|
inc(MULTIPART_UPLOAD_PART_PUT_BYTES, lengthInBytes);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
|
@ -0,0 +1,88 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.fs.s3a.statistics.impl;
|
||||||
|
|
||||||
|
import java.time.Duration;
|
||||||
|
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.CountersAndGauges;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.StatisticsFromAwsSdk;
|
||||||
|
|
||||||
|
import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_REQUEST;
|
||||||
|
import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_RETRY;
|
||||||
|
import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_THROTTLED;
|
||||||
|
import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_THROTTLE_RATE;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Hook up AWS SDK Statistics to the S3 counters.
|
||||||
|
* <p>
|
||||||
|
* Durations are not currently being used; that could be
|
||||||
|
* changed in future once an effective strategy for reporting
|
||||||
|
* them is determined.
|
||||||
|
*/
|
||||||
|
public final class StatisticsFromAwsSdkImpl implements
|
||||||
|
StatisticsFromAwsSdk {
|
||||||
|
|
||||||
|
private final CountersAndGauges countersAndGauges;
|
||||||
|
|
||||||
|
public StatisticsFromAwsSdkImpl(
|
||||||
|
final CountersAndGauges countersAndGauges) {
|
||||||
|
this.countersAndGauges = countersAndGauges;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void updateAwsRequestCount(final long count) {
|
||||||
|
countersAndGauges.incrementCounter(STORE_IO_REQUEST, count);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void updateAwsRetryCount(final long count) {
|
||||||
|
countersAndGauges.incrementCounter(STORE_IO_RETRY, count);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void updateAwsThrottleExceptionsCount(final long count) {
|
||||||
|
countersAndGauges.incrementCounter(STORE_IO_THROTTLED, count);
|
||||||
|
countersAndGauges.addValueToQuantiles(STORE_IO_THROTTLE_RATE, count);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void noteAwsRequestTime(final Duration duration) {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void noteAwsClientExecuteTime(final Duration duration) {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void noteRequestMarshallTime(final Duration duration) {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void noteRequestSigningTime(final Duration duration) {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void noteResponseProcessingTime(final Duration duration) {
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,29 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Statistics collection for the S3A connector: implementation.
|
||||||
|
* Not for use by anything outside the hadoop-aws source tree.
|
||||||
|
*/
|
||||||
|
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
@InterfaceStability.Unstable
|
||||||
|
package org.apache.hadoop.fs.s3a.statistics.impl;
|
||||||
|
|
||||||
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
@ -0,0 +1,31 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Statistics collection for the S3A connector: interfaces.
|
||||||
|
* This is private, though there's a risk that some extension
|
||||||
|
* points (delegation tokens?) may need access to the internal
|
||||||
|
* API. Hence the split packaging...with a java 9 module, the
|
||||||
|
* implementation classes would be declared internal.
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
@InterfaceStability.Unstable
|
||||||
|
package org.apache.hadoop.fs.s3a.statistics;
|
||||||
|
|
||||||
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
@ -65,6 +65,7 @@ import static org.apache.hadoop.fs.s3a.Constants.AUTHORITATIVE_PATH;
|
||||||
import static org.apache.hadoop.fs.s3a.Constants.BULK_DELETE_PAGE_SIZE;
|
import static org.apache.hadoop.fs.s3a.Constants.BULK_DELETE_PAGE_SIZE;
|
||||||
import static org.apache.hadoop.fs.s3a.Constants.BULK_DELETE_PAGE_SIZE_DEFAULT;
|
import static org.apache.hadoop.fs.s3a.Constants.BULK_DELETE_PAGE_SIZE_DEFAULT;
|
||||||
import static org.apache.hadoop.fs.s3a.Invoker.once;
|
import static org.apache.hadoop.fs.s3a.Invoker.once;
|
||||||
|
import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsSourceToString;
|
||||||
import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_INTERRUPTED;
|
import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_INTERRUPTED;
|
||||||
import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_NOT_ACCEPTABLE;
|
import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_NOT_ACCEPTABLE;
|
||||||
import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_NOT_FOUND;
|
import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_NOT_FOUND;
|
||||||
|
@ -672,6 +673,7 @@ public final class MarkerTool extends S3GuardTool {
|
||||||
final int limit) throws IOException {
|
final int limit) throws IOException {
|
||||||
|
|
||||||
int count = 0;
|
int count = 0;
|
||||||
|
boolean result = true;
|
||||||
RemoteIterator<S3AFileStatus> listing = operations
|
RemoteIterator<S3AFileStatus> listing = operations
|
||||||
.listObjects(path, storeContext.pathToKey(path));
|
.listObjects(path, storeContext.pathToKey(path));
|
||||||
while (listing.hasNext()) {
|
while (listing.hasNext()) {
|
||||||
|
@ -700,10 +702,16 @@ public final class MarkerTool extends S3GuardTool {
|
||||||
if (limit > 0 && count >= limit) {
|
if (limit > 0 && count >= limit) {
|
||||||
println(out, "Limit of scan reached - %,d object%s",
|
println(out, "Limit of scan reached - %,d object%s",
|
||||||
limit, suffix(limit));
|
limit, suffix(limit));
|
||||||
return false;
|
result = false;
|
||||||
|
break;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return true;
|
LOG.debug("Listing summary {}", listing);
|
||||||
|
if (verbose) {
|
||||||
|
println(out, "%nListing statistics:%n %s%n",
|
||||||
|
ioStatisticsSourceToString(listing));
|
||||||
|
}
|
||||||
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -79,15 +79,15 @@ public class ITestS3AContractDistCp extends AbstractContractDistCpTest {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void testDirectWrite() throws Exception {
|
public void testDirectWrite() throws Exception {
|
||||||
resetStorageStatistics();
|
final long renames = getRenameOperationCount();
|
||||||
super.testDirectWrite();
|
super.testDirectWrite();
|
||||||
assertEquals("Expected no renames for a direct write distcp", 0L,
|
assertEquals("Expected no renames for a direct write distcp", 0L,
|
||||||
getRenameOperationCount());
|
getRenameOperationCount() - renames);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void testNonDirectWrite() throws Exception {
|
public void testNonDirectWrite() throws Exception {
|
||||||
resetStorageStatistics();
|
final long renames = getRenameOperationCount();
|
||||||
try {
|
try {
|
||||||
super.testNonDirectWrite();
|
super.testNonDirectWrite();
|
||||||
} catch (FileNotFoundException e) {
|
} catch (FileNotFoundException e) {
|
||||||
|
@ -96,11 +96,7 @@ public class ITestS3AContractDistCp extends AbstractContractDistCpTest {
|
||||||
// S3Guard is not enabled
|
// S3Guard is not enabled
|
||||||
}
|
}
|
||||||
assertEquals("Expected 2 renames for a non-direct write distcp", 2L,
|
assertEquals("Expected 2 renames for a non-direct write distcp", 2L,
|
||||||
getRenameOperationCount());
|
getRenameOperationCount() - renames);
|
||||||
}
|
|
||||||
|
|
||||||
private void resetStorageStatistics() {
|
|
||||||
getFileSystem().getStorageStatistics().reset();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private long getRenameOperationCount() {
|
private long getRenameOperationCount() {
|
||||||
|
|
|
@ -26,7 +26,10 @@ import org.apache.hadoop.fs.contract.AbstractFSContractTestBase;
|
||||||
import org.apache.hadoop.fs.contract.ContractTestUtils;
|
import org.apache.hadoop.fs.contract.ContractTestUtils;
|
||||||
import org.apache.hadoop.fs.contract.s3a.S3AContract;
|
import org.apache.hadoop.fs.contract.s3a.S3AContract;
|
||||||
import org.apache.hadoop.fs.s3a.tools.MarkerTool;
|
import org.apache.hadoop.fs.s3a.tools.MarkerTool;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
|
||||||
import org.apache.hadoop.io.IOUtils;
|
import org.apache.hadoop.io.IOUtils;
|
||||||
|
|
||||||
|
import org.junit.AfterClass;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
@ -39,6 +42,8 @@ import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestDynamoTablePrefix;
|
||||||
import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestPropertyBool;
|
import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestPropertyBool;
|
||||||
import static org.apache.hadoop.fs.s3a.S3AUtils.E_FS_CLOSED;
|
import static org.apache.hadoop.fs.s3a.S3AUtils.E_FS_CLOSED;
|
||||||
import static org.apache.hadoop.fs.s3a.tools.MarkerTool.UNLIMITED_LISTING;
|
import static org.apache.hadoop.fs.s3a.tools.MarkerTool.UNLIMITED_LISTING;
|
||||||
|
import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString;
|
||||||
|
import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.snapshotIOStatistics;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* An extension of the contract test base set up for S3A tests.
|
* An extension of the contract test base set up for S3A tests.
|
||||||
|
@ -48,6 +53,12 @@ public abstract class AbstractS3ATestBase extends AbstractFSContractTestBase
|
||||||
protected static final Logger LOG =
|
protected static final Logger LOG =
|
||||||
LoggerFactory.getLogger(AbstractS3ATestBase.class);
|
LoggerFactory.getLogger(AbstractS3ATestBase.class);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* FileSystem statistics are collected across every test case.
|
||||||
|
*/
|
||||||
|
protected static final IOStatisticsSnapshot FILESYSTEM_IOSTATS =
|
||||||
|
snapshotIOStatistics();
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected AbstractFSContract createContract(Configuration conf) {
|
protected AbstractFSContract createContract(Configuration conf) {
|
||||||
return new S3AContract(conf, false);
|
return new S3AContract(conf, false);
|
||||||
|
@ -73,10 +84,22 @@ public abstract class AbstractS3ATestBase extends AbstractFSContractTestBase
|
||||||
maybeAuditTestPath();
|
maybeAuditTestPath();
|
||||||
|
|
||||||
super.teardown();
|
super.teardown();
|
||||||
|
if (getFileSystem() != null) {
|
||||||
|
FILESYSTEM_IOSTATS.aggregate(getFileSystem().getIOStatistics());
|
||||||
|
}
|
||||||
describe("closing file system");
|
describe("closing file system");
|
||||||
IOUtils.closeStream(getFileSystem());
|
IOUtils.closeStream(getFileSystem());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Dump the filesystem statistics after the class.
|
||||||
|
*/
|
||||||
|
@AfterClass
|
||||||
|
public static void dumpFileSystemIOStatistics() {
|
||||||
|
LOG.info("Aggregate FileSystem Statistics {}",
|
||||||
|
ioStatisticsToPrettyString(FILESYSTEM_IOSTATS));
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Audit the FS under {@link #methodPath()} if
|
* Audit the FS under {@link #methodPath()} if
|
||||||
* the test option {@link #DIRECTORY_MARKER_AUDIT} is
|
* the test option {@link #DIRECTORY_MARKER_AUDIT} is
|
||||||
|
|
|
@ -18,23 +18,287 @@
|
||||||
|
|
||||||
package org.apache.hadoop.fs.s3a;
|
package org.apache.hadoop.fs.s3a;
|
||||||
|
|
||||||
|
import java.nio.charset.StandardCharsets;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Collection;
|
||||||
|
|
||||||
|
import org.assertj.core.api.Assertions;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
import org.junit.runner.RunWith;
|
||||||
|
import org.junit.runners.Parameterized;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.fs.PathFilter;
|
||||||
|
import org.apache.hadoop.fs.s3a.auth.ITestRestrictedReadAccess;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatistics;
|
||||||
|
import org.apache.hadoop.mapred.LocatedFileStatusFetcher;
|
||||||
|
|
||||||
|
import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
|
||||||
|
import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
|
||||||
|
import static org.apache.hadoop.fs.s3a.Constants.AUTHORITATIVE_PATH;
|
||||||
|
import static org.apache.hadoop.fs.s3a.Constants.METADATASTORE_AUTHORITATIVE;
|
||||||
|
import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL;
|
||||||
|
import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching;
|
||||||
|
import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName;
|
||||||
|
import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
|
||||||
|
import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBucketOverrides;
|
||||||
|
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticCounter;
|
||||||
|
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.extractStatistics;
|
||||||
|
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_LIST_REQUEST;
|
||||||
|
import static org.apache.hadoop.mapreduce.lib.input.FileInputFormat.LIST_STATUS_NUM_THREADS;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Test the LocatedFileStatusFetcher can do.
|
* Test the LocatedFileStatusFetcher can do.
|
||||||
* This is related to HADOOP-16458.
|
* This is related to HADOOP-16458.
|
||||||
* There's basic tests in ITestS3AFSMainOperations; this
|
* There's basic tests in ITestS3AFSMainOperations; this
|
||||||
* is see if we can create better corner cases.
|
* is see if we can create better corner cases.
|
||||||
|
* <p></p>
|
||||||
|
* Much of the class is based on tests in {@link ITestRestrictedReadAccess},
|
||||||
|
* but whereas that tests failure paths, this looks at the performance
|
||||||
|
* of successful invocations.
|
||||||
*/
|
*/
|
||||||
|
@RunWith(Parameterized.class)
|
||||||
public class ITestLocatedFileStatusFetcher extends AbstractS3ATestBase {
|
public class ITestLocatedFileStatusFetcher extends AbstractS3ATestBase {
|
||||||
|
|
||||||
private static final Logger LOG =
|
private static final Logger LOG =
|
||||||
LoggerFactory.getLogger(ITestLocatedFileStatusFetcher.class);
|
LoggerFactory.getLogger(ITestLocatedFileStatusFetcher.class);
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testGlobScan() throws Throwable {
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Parameterization.
|
||||||
|
*/
|
||||||
|
@Parameterized.Parameters(name = "{0}")
|
||||||
|
public static Collection<Object[]> params() {
|
||||||
|
return Arrays.asList(new Object[][]{
|
||||||
|
{"raw", false},
|
||||||
|
{"nonauth", true}
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/** Filter to select everything. */
|
||||||
|
private static final PathFilter EVERYTHING = t -> true;
|
||||||
|
|
||||||
|
/** Filter to select .txt files. */
|
||||||
|
private static final PathFilter TEXT_FILE =
|
||||||
|
path -> path.toUri().toString().endsWith(".txt");
|
||||||
|
|
||||||
|
/** The same path filter used in FileInputFormat. */
|
||||||
|
private static final PathFilter HIDDEN_FILE_FILTER =
|
||||||
|
(p) -> {
|
||||||
|
String n = p.getName();
|
||||||
|
return !n.startsWith("_") && !n.startsWith(".");
|
||||||
|
};
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Text found in LocatedFileStatusFetcher exception when the glob
|
||||||
|
* returned "null".
|
||||||
|
*/
|
||||||
|
private static final String DOES_NOT_EXIST = "does not exist";
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Text found in LocatedFileStatusFetcher exception when
|
||||||
|
* the glob returned an empty list.
|
||||||
|
*/
|
||||||
|
private static final String MATCHES_0_FILES = "matches 0 files";
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Text used in files.
|
||||||
|
*/
|
||||||
|
public static final byte[] HELLO = "hello".getBytes(StandardCharsets.UTF_8);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* How many list calls are expected in a run which collects them: {@value}.
|
||||||
|
*/
|
||||||
|
private static final int EXPECTED_LIST_COUNT = 4;
|
||||||
|
|
||||||
|
private final String name;
|
||||||
|
|
||||||
|
private final boolean s3guard;
|
||||||
|
|
||||||
|
private Path basePath;
|
||||||
|
|
||||||
|
private Path emptyDir;
|
||||||
|
|
||||||
|
private Path emptyFile;
|
||||||
|
|
||||||
|
private Path subDir;
|
||||||
|
|
||||||
|
private Path subdirFile;
|
||||||
|
|
||||||
|
private Path subDir2;
|
||||||
|
|
||||||
|
private Path subdir2File1;
|
||||||
|
|
||||||
|
private Path subdir2File2;
|
||||||
|
|
||||||
|
private Configuration listConfig;
|
||||||
|
|
||||||
|
public ITestLocatedFileStatusFetcher(final String name,
|
||||||
|
final boolean s3guard) {
|
||||||
|
this.name = name;
|
||||||
|
this.s3guard = s3guard;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Configuration createConfiguration() {
|
||||||
|
Configuration conf = super.createConfiguration();
|
||||||
|
String bucketName = getTestBucketName(conf);
|
||||||
|
|
||||||
|
removeBaseAndBucketOverrides(bucketName, conf,
|
||||||
|
METADATASTORE_AUTHORITATIVE,
|
||||||
|
AUTHORITATIVE_PATH);
|
||||||
|
removeBucketOverrides(bucketName, conf,
|
||||||
|
S3_METADATA_STORE_IMPL);
|
||||||
|
if (!s3guard) {
|
||||||
|
removeBaseAndBucketOverrides(bucketName, conf,
|
||||||
|
S3_METADATA_STORE_IMPL);
|
||||||
|
}
|
||||||
|
conf.setBoolean(METADATASTORE_AUTHORITATIVE, false);
|
||||||
|
disableFilesystemCaching(conf);
|
||||||
|
return conf;
|
||||||
|
}
|
||||||
|
@Override
|
||||||
|
public void setup() throws Exception {
|
||||||
|
super.setup();
|
||||||
|
S3AFileSystem fs
|
||||||
|
= getFileSystem();
|
||||||
|
// avoiding the parameterization to steer clear of accidentally creating
|
||||||
|
// patterns; a timestamp is used to ensure tombstones from previous runs
|
||||||
|
// do not interfere
|
||||||
|
basePath = path("ITestLocatedFileStatusFetcher-" + name
|
||||||
|
+ "-" + System.currentTimeMillis() / 1000);
|
||||||
|
|
||||||
|
// define the paths and create them.
|
||||||
|
describe("Creating test directories and files");
|
||||||
|
|
||||||
|
// an empty directory directory under the noReadDir
|
||||||
|
emptyDir = new Path(basePath, "emptyDir");
|
||||||
|
fs.mkdirs(emptyDir);
|
||||||
|
|
||||||
|
// an empty file directory under the noReadDir
|
||||||
|
emptyFile = new Path(basePath, "emptyFile.txt");
|
||||||
|
touch(fs, emptyFile);
|
||||||
|
|
||||||
|
// a subdirectory
|
||||||
|
subDir = new Path(basePath, "subDir");
|
||||||
|
|
||||||
|
// and a file in that subdirectory
|
||||||
|
subdirFile = new Path(subDir, "subdirFile.txt");
|
||||||
|
createFile(fs, subdirFile, true, HELLO);
|
||||||
|
subDir2 = new Path(subDir, "subDir2");
|
||||||
|
subdir2File1 = new Path(subDir2, "subdir2File1.txt");
|
||||||
|
subdir2File2 = new Path(subDir2, "subdir2File2.txt");
|
||||||
|
createFile(fs, subdir2File1, true, HELLO);
|
||||||
|
createFile(fs, subdir2File2, true, HELLO);
|
||||||
|
listConfig = new Configuration(getConfiguration());
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Assert that the fetcher stats logs the expected number of calls.
|
||||||
|
* @param fetcher fetcher
|
||||||
|
* @param expectedListCount expected number of list calls
|
||||||
|
*/
|
||||||
|
private void assertListCount(final LocatedFileStatusFetcher fetcher,
|
||||||
|
final int expectedListCount) {
|
||||||
|
IOStatistics iostats = extractStatistics(fetcher);
|
||||||
|
LOG.info("Statistics of fetcher: {}", iostats);
|
||||||
|
assertThatStatisticCounter(iostats,
|
||||||
|
OBJECT_LIST_REQUEST)
|
||||||
|
.describedAs("stats of %s", iostats)
|
||||||
|
.isEqualTo(expectedListCount);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Run a located file status fetcher against the directory tree.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testSingleThreadedLocatedFileStatus() throws Throwable {
|
||||||
|
|
||||||
|
describe("LocatedFileStatusFetcher operations");
|
||||||
|
// use the same filter as FileInputFormat; single thread.
|
||||||
|
|
||||||
|
listConfig.setInt(LIST_STATUS_NUM_THREADS, 1);
|
||||||
|
LocatedFileStatusFetcher fetcher =
|
||||||
|
new LocatedFileStatusFetcher(
|
||||||
|
listConfig,
|
||||||
|
new Path[]{basePath},
|
||||||
|
true,
|
||||||
|
HIDDEN_FILE_FILTER,
|
||||||
|
true);
|
||||||
|
Iterable<FileStatus> stats = fetcher.getFileStatuses();
|
||||||
|
Assertions.assertThat(stats)
|
||||||
|
.describedAs("result of located scan")
|
||||||
|
.flatExtracting(FileStatus::getPath)
|
||||||
|
.containsExactlyInAnyOrder(
|
||||||
|
emptyFile,
|
||||||
|
subdirFile,
|
||||||
|
subdir2File1,
|
||||||
|
subdir2File2);
|
||||||
|
assertListCount(fetcher, EXPECTED_LIST_COUNT);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Run a located file status fetcher against the directory tree.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testLocatedFileStatusFourThreads() throws Throwable {
|
||||||
|
|
||||||
|
// four threads and the text filter.
|
||||||
|
int threads = 4;
|
||||||
|
describe("LocatedFileStatusFetcher with %d", threads);
|
||||||
|
listConfig.setInt(LIST_STATUS_NUM_THREADS, threads);
|
||||||
|
LocatedFileStatusFetcher fetcher =
|
||||||
|
new LocatedFileStatusFetcher(
|
||||||
|
listConfig,
|
||||||
|
new Path[]{basePath},
|
||||||
|
true,
|
||||||
|
EVERYTHING,
|
||||||
|
true);
|
||||||
|
Iterable<FileStatus> stats = fetcher.getFileStatuses();
|
||||||
|
IOStatistics iostats = extractStatistics(fetcher);
|
||||||
|
LOG.info("Statistics of fetcher: {}", iostats);
|
||||||
|
Assertions.assertThat(stats)
|
||||||
|
.describedAs("result of located scan")
|
||||||
|
.isNotNull()
|
||||||
|
.flatExtracting(FileStatus::getPath)
|
||||||
|
.containsExactlyInAnyOrder(
|
||||||
|
emptyFile,
|
||||||
|
subdirFile,
|
||||||
|
subdir2File1,
|
||||||
|
subdir2File2);
|
||||||
|
assertListCount(fetcher, EXPECTED_LIST_COUNT);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Run a located file status fetcher against a file.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testLocatedFileStatusScanFile() throws Throwable {
|
||||||
|
// pass in a file as the base of the scan.
|
||||||
|
describe("LocatedFileStatusFetcher with file %s", subdirFile);
|
||||||
|
listConfig.setInt(LIST_STATUS_NUM_THREADS, 16);
|
||||||
|
LocatedFileStatusFetcher fetcher
|
||||||
|
= new LocatedFileStatusFetcher(
|
||||||
|
listConfig,
|
||||||
|
new Path[]{subdirFile},
|
||||||
|
true,
|
||||||
|
TEXT_FILE,
|
||||||
|
true);
|
||||||
|
Iterable<FileStatus> stats = fetcher.getFileStatuses();
|
||||||
|
Assertions.assertThat(stats)
|
||||||
|
.describedAs("result of located scan")
|
||||||
|
.isNotNull()
|
||||||
|
.flatExtracting(FileStatus::getPath)
|
||||||
|
.containsExactly(subdirFile);
|
||||||
|
IOStatistics ioStatistics = fetcher.getIOStatistics();
|
||||||
|
Assertions.assertThat(ioStatistics)
|
||||||
|
.describedAs("IO statistics of %s", fetcher)
|
||||||
|
.isNull();
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,6 +22,7 @@ import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.fs.contract.ContractTestUtils;
|
import org.apache.hadoop.fs.contract.ContractTestUtils;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics;
|
||||||
import org.apache.hadoop.io.IOUtils;
|
import org.apache.hadoop.io.IOUtils;
|
||||||
|
|
||||||
import org.junit.BeforeClass;
|
import org.junit.BeforeClass;
|
||||||
|
@ -94,16 +95,16 @@ public class ITestS3ABlockOutputArray extends AbstractS3ATestBase {
|
||||||
Path dest = path("testBlocksClosed");
|
Path dest = path("testBlocksClosed");
|
||||||
describe(" testBlocksClosed");
|
describe(" testBlocksClosed");
|
||||||
FSDataOutputStream stream = getFileSystem().create(dest, true);
|
FSDataOutputStream stream = getFileSystem().create(dest, true);
|
||||||
S3AInstrumentation.OutputStreamStatistics statistics
|
BlockOutputStreamStatistics statistics
|
||||||
= S3ATestUtils.getOutputStreamStatistics(stream);
|
= S3ATestUtils.getOutputStreamStatistics(stream);
|
||||||
byte[] data = ContractTestUtils.dataset(16, 'a', 26);
|
byte[] data = ContractTestUtils.dataset(16, 'a', 26);
|
||||||
stream.write(data);
|
stream.write(data);
|
||||||
LOG.info("closing output stream");
|
LOG.info("closing output stream");
|
||||||
stream.close();
|
stream.close();
|
||||||
assertEquals("total allocated blocks in " + statistics,
|
assertEquals("total allocated blocks in " + statistics,
|
||||||
1, statistics.blocksAllocated());
|
1, statistics.getBlocksAllocated());
|
||||||
assertEquals("actively allocated blocks in " + statistics,
|
assertEquals("actively allocated blocks in " + statistics,
|
||||||
0, statistics.blocksActivelyAllocated());
|
0, statistics.getBlocksActivelyAllocated());
|
||||||
LOG.info("end of test case");
|
LOG.info("end of test case");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -129,7 +130,7 @@ public class ITestS3ABlockOutputArray extends AbstractS3ATestBase {
|
||||||
throws Exception {
|
throws Exception {
|
||||||
S3AInstrumentation instrumentation =
|
S3AInstrumentation instrumentation =
|
||||||
new S3AInstrumentation(new URI("s3a://example"));
|
new S3AInstrumentation(new URI("s3a://example"));
|
||||||
S3AInstrumentation.OutputStreamStatistics outstats
|
BlockOutputStreamStatistics outstats
|
||||||
= instrumentation.newOutputStreamStatistics(null);
|
= instrumentation.newOutputStreamStatistics(null);
|
||||||
S3ADataBlocks.DataBlock block = factory.create(1, BLOCK_SIZE, outstats);
|
S3ADataBlocks.DataBlock block = factory.create(1, BLOCK_SIZE, outstats);
|
||||||
block.write(dataset, 0, dataset.length);
|
block.write(dataset, 0, dataset.length);
|
||||||
|
|
|
@ -21,11 +21,15 @@ package org.apache.hadoop.fs.s3a;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.fs.contract.ContractTestUtils;
|
import org.apache.hadoop.fs.contract.ContractTestUtils;
|
||||||
import org.apache.hadoop.metrics2.lib.MutableCounterLong;
|
import org.apache.hadoop.metrics2.lib.MutableCounterLong;
|
||||||
|
|
||||||
|
import org.assertj.core.api.Assertions;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.InputStream;
|
import java.io.InputStream;
|
||||||
|
|
||||||
|
import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsSourceToString;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Test s3a performance metrics register and output.
|
* Test s3a performance metrics register and output.
|
||||||
*/
|
*/
|
||||||
|
@ -51,17 +55,34 @@ public class ITestS3AMetrics extends AbstractS3ATestBase {
|
||||||
Path file = path("testStreamStatistics");
|
Path file = path("testStreamStatistics");
|
||||||
byte[] data = "abcdefghijklmnopqrstuvwxyz".getBytes();
|
byte[] data = "abcdefghijklmnopqrstuvwxyz".getBytes();
|
||||||
ContractTestUtils.createFile(fs, file, false, data);
|
ContractTestUtils.createFile(fs, file, false, data);
|
||||||
|
InputStream inputStream = fs.open(file);
|
||||||
try (InputStream inputStream = fs.open(file)) {
|
try {
|
||||||
while (inputStream.read(data) != -1) {
|
while (inputStream.read(data) != -1) {
|
||||||
LOG.debug("Read batch of data from input stream...");
|
LOG.debug("Read batch of data from input stream...");
|
||||||
}
|
}
|
||||||
|
LOG.info("Final stream statistics: {}",
|
||||||
|
ioStatisticsSourceToString(inputStream));
|
||||||
|
} finally {
|
||||||
|
// this is not try-with-resources only to aid debugging
|
||||||
|
inputStream.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
final String statName = Statistic.STREAM_READ_BYTES.getSymbol();
|
||||||
|
|
||||||
|
final S3AInstrumentation instrumentation = fs.getInstrumentation();
|
||||||
|
|
||||||
|
final long counterValue = instrumentation.getCounterValue(statName);
|
||||||
|
|
||||||
|
final int expectedBytesRead = 26;
|
||||||
|
Assertions.assertThat(counterValue)
|
||||||
|
.describedAs("Counter %s from instrumentation %s",
|
||||||
|
statName, instrumentation)
|
||||||
|
.isEqualTo(expectedBytesRead);
|
||||||
MutableCounterLong read = (MutableCounterLong)
|
MutableCounterLong read = (MutableCounterLong)
|
||||||
fs.getInstrumentation().getRegistry()
|
instrumentation.getRegistry()
|
||||||
.get(Statistic.STREAM_SEEK_BYTES_READ.getSymbol());
|
.get(statName);
|
||||||
assertEquals("Stream statistics were not merged", 26, read.value());
|
assertEquals("Stream statistics were not merged", expectedBytesRead,
|
||||||
|
read.value());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -276,8 +276,14 @@ public class ITestS3AMiscOperations extends AbstractS3ATestBase {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testS3AIOStatisticsUninitialized() throws Throwable {
|
||||||
|
try (S3AFileSystem fs = new S3AFileSystem()) {
|
||||||
|
fs.getIOStatistics();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
/**
|
/**
|
||||||
<<<<<<< ours
|
|
||||||
* Verify that paths with a trailing "/" are fixed up.
|
* Verify that paths with a trailing "/" are fixed up.
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
|
|
|
@ -21,13 +21,23 @@ package org.apache.hadoop.fs.s3a;
|
||||||
import org.apache.hadoop.fs.FSDataInputStream;
|
import org.apache.hadoop.fs.FSDataInputStream;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.fs.contract.ContractTestUtils;
|
import org.apache.hadoop.fs.contract.ContractTestUtils;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatistics;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
|
||||||
|
import org.apache.hadoop.fs.statistics.StoreStatisticNames;
|
||||||
|
import org.apache.hadoop.fs.statistics.StreamStatisticNames;
|
||||||
import org.apache.hadoop.io.IOUtils;
|
import org.apache.hadoop.io.IOUtils;
|
||||||
|
|
||||||
|
import org.assertj.core.api.Assertions;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
|
||||||
import static org.apache.hadoop.fs.s3a.Statistic.STREAM_SEEK_BYTES_READ;
|
import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_BYTES;
|
||||||
|
import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_BYTES_READ_CLOSE;
|
||||||
|
import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_TOTAL_BYTES;
|
||||||
|
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue;
|
||||||
|
import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.demandStringifyIOStatisticsSource;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Integration test for calling
|
* Integration test for calling
|
||||||
|
@ -40,6 +50,8 @@ import static org.apache.hadoop.fs.s3a.Statistic.STREAM_SEEK_BYTES_READ;
|
||||||
*/
|
*/
|
||||||
public class ITestS3AUnbuffer extends AbstractS3ATestBase {
|
public class ITestS3AUnbuffer extends AbstractS3ATestBase {
|
||||||
|
|
||||||
|
public static final int FILE_LENGTH = 16;
|
||||||
|
|
||||||
private Path dest;
|
private Path dest;
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -48,7 +60,7 @@ public class ITestS3AUnbuffer extends AbstractS3ATestBase {
|
||||||
dest = path("ITestS3AUnbuffer");
|
dest = path("ITestS3AUnbuffer");
|
||||||
describe("ITestS3AUnbuffer");
|
describe("ITestS3AUnbuffer");
|
||||||
|
|
||||||
byte[] data = ContractTestUtils.dataset(16, 'a', 26);
|
byte[] data = ContractTestUtils.dataset(FILE_LENGTH, 'a', 26);
|
||||||
ContractTestUtils.writeDataset(getFileSystem(), dest, data, data.length,
|
ContractTestUtils.writeDataset(getFileSystem(), dest, data, data.length,
|
||||||
16, true);
|
16, true);
|
||||||
}
|
}
|
||||||
|
@ -57,13 +69,41 @@ public class ITestS3AUnbuffer extends AbstractS3ATestBase {
|
||||||
public void testUnbuffer() throws IOException {
|
public void testUnbuffer() throws IOException {
|
||||||
describe("testUnbuffer");
|
describe("testUnbuffer");
|
||||||
|
|
||||||
|
IOStatisticsSnapshot iostats = new IOStatisticsSnapshot();
|
||||||
// Open file, read half the data, and then call unbuffer
|
// Open file, read half the data, and then call unbuffer
|
||||||
try (FSDataInputStream inputStream = getFileSystem().open(dest)) {
|
try (FSDataInputStream inputStream = getFileSystem().open(dest)) {
|
||||||
assertTrue(inputStream.getWrappedStream() instanceof S3AInputStream);
|
assertTrue(inputStream.getWrappedStream() instanceof S3AInputStream);
|
||||||
readAndAssertBytesRead(inputStream, 8);
|
int bytesToRead = 8;
|
||||||
|
readAndAssertBytesRead(inputStream, bytesToRead);
|
||||||
assertTrue(isObjectStreamOpen(inputStream));
|
assertTrue(isObjectStreamOpen(inputStream));
|
||||||
|
assertTrue("No IOstatistics from " + inputStream,
|
||||||
|
iostats.aggregate(inputStream.getIOStatistics()));
|
||||||
|
verifyStatisticCounterValue(iostats,
|
||||||
|
StreamStatisticNames.STREAM_READ_BYTES,
|
||||||
|
bytesToRead);
|
||||||
|
verifyStatisticCounterValue(iostats,
|
||||||
|
StoreStatisticNames.ACTION_HTTP_GET_REQUEST,
|
||||||
|
1);
|
||||||
|
|
||||||
|
// do the unbuffering
|
||||||
inputStream.unbuffer();
|
inputStream.unbuffer();
|
||||||
|
|
||||||
|
// audit the updated statistics
|
||||||
|
IOStatistics st2 = inputStream.getIOStatistics();
|
||||||
|
|
||||||
|
// the unbuffered operation must be tracked
|
||||||
|
verifyStatisticCounterValue(st2,
|
||||||
|
StreamStatisticNames.STREAM_READ_UNBUFFERED,
|
||||||
|
1);
|
||||||
|
|
||||||
|
// all other counter values consistent.
|
||||||
|
verifyStatisticCounterValue(st2,
|
||||||
|
StreamStatisticNames.STREAM_READ_BYTES,
|
||||||
|
bytesToRead);
|
||||||
|
verifyStatisticCounterValue(st2,
|
||||||
|
StoreStatisticNames.ACTION_HTTP_GET_REQUEST,
|
||||||
|
1);
|
||||||
|
|
||||||
// Check the the wrapped stream is closed
|
// Check the the wrapped stream is closed
|
||||||
assertFalse(isObjectStreamOpen(inputStream));
|
assertFalse(isObjectStreamOpen(inputStream));
|
||||||
}
|
}
|
||||||
|
@ -71,7 +111,7 @@ public class ITestS3AUnbuffer extends AbstractS3ATestBase {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Test that calling {@link S3AInputStream#unbuffer()} merges a stream's
|
* 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.
|
* into the {@link S3AFileSystem}'s {@link S3AInstrumentation} instance.
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
|
@ -79,36 +119,73 @@ public class ITestS3AUnbuffer extends AbstractS3ATestBase {
|
||||||
describe("testUnbufferStreamStatistics");
|
describe("testUnbufferStreamStatistics");
|
||||||
|
|
||||||
// Validate bytesRead is updated correctly
|
// Validate bytesRead is updated correctly
|
||||||
|
S3AFileSystem fs = getFileSystem();
|
||||||
S3ATestUtils.MetricDiff bytesRead = new S3ATestUtils.MetricDiff(
|
S3ATestUtils.MetricDiff bytesRead = new S3ATestUtils.MetricDiff(
|
||||||
getFileSystem(), STREAM_SEEK_BYTES_READ);
|
fs, STREAM_READ_BYTES);
|
||||||
|
S3ATestUtils.MetricDiff totalBytesRead = new S3ATestUtils.MetricDiff(
|
||||||
|
fs, STREAM_READ_TOTAL_BYTES);
|
||||||
|
S3ATestUtils.MetricDiff bytesReadInClose = new S3ATestUtils.MetricDiff(
|
||||||
|
fs, STREAM_READ_BYTES_READ_CLOSE);
|
||||||
|
|
||||||
// Open file, read half the data, and then call unbuffer
|
// Open file, read half the data, and then call unbuffer
|
||||||
FSDataInputStream inputStream = null;
|
FSDataInputStream inputStream = null;
|
||||||
try {
|
int firstBytesToRead = 8;
|
||||||
inputStream = getFileSystem().open(dest);
|
|
||||||
|
|
||||||
readAndAssertBytesRead(inputStream, 8);
|
int secondBytesToRead = 1;
|
||||||
|
long expectedFinalBytesRead;
|
||||||
|
long expectedTotalBytesRead;
|
||||||
|
|
||||||
|
Object streamStatsStr;
|
||||||
|
try {
|
||||||
|
inputStream = fs.open(dest);
|
||||||
|
streamStatsStr = demandStringifyIOStatisticsSource(inputStream);
|
||||||
|
|
||||||
|
LOG.info("initial stream statistics {}", streamStatsStr);
|
||||||
|
readAndAssertBytesRead(inputStream, firstBytesToRead);
|
||||||
|
LOG.info("stream statistics after read {}", streamStatsStr);
|
||||||
inputStream.unbuffer();
|
inputStream.unbuffer();
|
||||||
|
|
||||||
// Validate that calling unbuffer updates the input stream statistics
|
// Validate that calling unbuffer updates the input stream statistics
|
||||||
bytesRead.assertDiffEquals(8);
|
bytesRead.assertDiffEquals(firstBytesToRead);
|
||||||
|
final long bytesInUnbuffer = bytesReadInClose.diff();
|
||||||
|
totalBytesRead.assertDiffEquals(firstBytesToRead + bytesInUnbuffer);
|
||||||
|
|
||||||
// Validate that calling unbuffer twice in a row updates the statistics
|
// Validate that calling unbuffer twice in a row updates the statistics
|
||||||
// correctly
|
// correctly
|
||||||
readAndAssertBytesRead(inputStream, 4);
|
bytesReadInClose.reset();
|
||||||
|
bytesRead.reset();
|
||||||
|
readAndAssertBytesRead(inputStream, secondBytesToRead);
|
||||||
inputStream.unbuffer();
|
inputStream.unbuffer();
|
||||||
bytesRead.assertDiffEquals(12);
|
LOG.info("stream statistics after second read {}", streamStatsStr);
|
||||||
|
bytesRead.assertDiffEquals(secondBytesToRead);
|
||||||
|
final long bytesInClose = bytesReadInClose.diff();
|
||||||
|
expectedFinalBytesRead = firstBytesToRead + secondBytesToRead;
|
||||||
|
expectedTotalBytesRead = expectedFinalBytesRead
|
||||||
|
+ bytesInUnbuffer + bytesInClose;
|
||||||
|
|
||||||
|
totalBytesRead.assertDiffEquals(expectedTotalBytesRead);
|
||||||
} finally {
|
} finally {
|
||||||
|
LOG.info("Closing stream");
|
||||||
IOUtils.closeStream(inputStream);
|
IOUtils.closeStream(inputStream);
|
||||||
}
|
}
|
||||||
|
LOG.info("stream statistics after close {}", streamStatsStr);
|
||||||
|
|
||||||
// Validate that closing the file does not further change the statistics
|
// Validate that closing the file does not further change the statistics
|
||||||
bytesRead.assertDiffEquals(12);
|
totalBytesRead.assertDiffEquals(expectedTotalBytesRead);
|
||||||
|
|
||||||
// Validate that the input stream stats are correct when the file is closed
|
// Validate that the input stream stats are correct when the file is closed
|
||||||
assertEquals("S3AInputStream statistics were not updated properly", 12,
|
S3AInputStreamStatistics streamStatistics = ((S3AInputStream) inputStream
|
||||||
((S3AInputStream) inputStream.getWrappedStream())
|
.getWrappedStream())
|
||||||
.getS3AStreamStatistics().bytesRead);
|
.getS3AStreamStatistics();
|
||||||
|
Assertions.assertThat(streamStatistics)
|
||||||
|
.describedAs("Stream statistics %s", streamStatistics)
|
||||||
|
.hasFieldOrPropertyWithValue("bytesRead",
|
||||||
|
expectedFinalBytesRead)
|
||||||
|
.hasFieldOrPropertyWithValue("totalBytesRead", expectedTotalBytesRead);
|
||||||
|
assertEquals("S3AInputStream statistics were not updated properly in "
|
||||||
|
+ streamStatsStr,
|
||||||
|
expectedFinalBytesRead,
|
||||||
|
streamStatistics.getBytesRead());
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean isObjectStreamOpen(FSDataInputStream inputStream) {
|
private boolean isObjectStreamOpen(FSDataInputStream inputStream) {
|
||||||
|
|
|
@ -225,7 +225,7 @@ public class ITestS3GuardEmptyDirs extends AbstractS3ATestBase {
|
||||||
// if DDB is the metastore, then we expect no FS requests to be made
|
// if DDB is the metastore, then we expect no FS requests to be made
|
||||||
// at all.
|
// at all.
|
||||||
S3ATestUtils.MetricDiff listMetric = new S3ATestUtils.MetricDiff(fs,
|
S3ATestUtils.MetricDiff listMetric = new S3ATestUtils.MetricDiff(fs,
|
||||||
Statistic.OBJECT_LIST_REQUESTS);
|
Statistic.OBJECT_LIST_REQUEST);
|
||||||
S3ATestUtils.MetricDiff getMetric = new S3ATestUtils.MetricDiff(fs,
|
S3ATestUtils.MetricDiff getMetric = new S3ATestUtils.MetricDiff(fs,
|
||||||
Statistic.OBJECT_METADATA_REQUESTS);
|
Statistic.OBJECT_METADATA_REQUESTS);
|
||||||
// do a getFile status with empty dir flag
|
// do a getFile status with empty dir flag
|
||||||
|
|
|
@ -39,11 +39,14 @@ import org.apache.hadoop.fs.RemoteIterator;
|
||||||
import org.apache.hadoop.fs.permission.FsPermission;
|
import org.apache.hadoop.fs.permission.FsPermission;
|
||||||
import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets;
|
import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets;
|
||||||
import org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase;
|
import org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext;
|
||||||
import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
|
import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
|
||||||
|
import org.apache.hadoop.fs.statistics.DurationTrackerFactory;
|
||||||
import org.apache.hadoop.util.Progressable;
|
import org.apache.hadoop.util.Progressable;
|
||||||
|
|
||||||
|
import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.stubDurationTrackerFactory;
|
||||||
import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull;
|
import static org.apache.hadoop.thirdparty.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
|
* Relays FS calls to the mocked FS, allows for some extra logging with
|
||||||
|
@ -83,8 +86,6 @@ public class MockS3AFileSystem extends S3AFileSystem {
|
||||||
* mock FS.
|
* mock FS.
|
||||||
*/
|
*/
|
||||||
private int logEvents = LOG_NAME;
|
private int logEvents = LOG_NAME;
|
||||||
private final S3AInstrumentation instrumentation =
|
|
||||||
new S3AInstrumentation(FS_URI);
|
|
||||||
private Configuration conf;
|
private Configuration conf;
|
||||||
private WriteOperationHelper writeHelper;
|
private WriteOperationHelper writeHelper;
|
||||||
|
|
||||||
|
@ -146,12 +147,12 @@ public class MockS3AFileSystem extends S3AFileSystem {
|
||||||
public void initialize(URI name, Configuration originalConf)
|
public void initialize(URI name, Configuration originalConf)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
conf = originalConf;
|
conf = originalConf;
|
||||||
writeHelper = new WriteOperationHelper(this, conf);
|
writeHelper = new WriteOperationHelper(this, conf,
|
||||||
|
new EmptyS3AStatisticsContext());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void close() {
|
public void close() {
|
||||||
cleanupWithLogger(LOG, instrumentation);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -359,12 +360,17 @@ public class MockS3AFileSystem extends S3AFileSystem {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public S3AInstrumentation.CommitterStatistics newCommitterStatistics() {
|
public CommitterStatistics newCommitterStatistics() {
|
||||||
return instrumentation.newCommitterStatistics();
|
return EmptyS3AStatisticsContext.EMPTY_COMMITTER_STATISTICS;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void operationRetried(Exception ex) {
|
public void operationRetried(Exception ex) {
|
||||||
/** no-op */
|
/** no-op */
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected DurationTrackerFactory getDurationTrackerFactory() {
|
||||||
|
return stubDurationTrackerFactory();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -28,6 +28,8 @@ import com.amazonaws.services.s3.AmazonS3;
|
||||||
import com.amazonaws.services.s3.model.MultipartUploadListing;
|
import com.amazonaws.services.s3.model.MultipartUploadListing;
|
||||||
import com.amazonaws.services.s3.model.Region;
|
import com.amazonaws.services.s3.model.Region;
|
||||||
|
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.StatisticsFromAwsSdk;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* An {@link S3ClientFactory} that returns Mockito mocks of the {@link AmazonS3}
|
* An {@link S3ClientFactory} that returns Mockito mocks of the {@link AmazonS3}
|
||||||
* interface suitable for unit testing.
|
* interface suitable for unit testing.
|
||||||
|
@ -38,7 +40,8 @@ public class MockS3ClientFactory implements S3ClientFactory {
|
||||||
public AmazonS3 createS3Client(URI name,
|
public AmazonS3 createS3Client(URI name,
|
||||||
final String bucket,
|
final String bucket,
|
||||||
final AWSCredentialsProvider credentialSet,
|
final AWSCredentialsProvider credentialSet,
|
||||||
final String userAgentSuffix) {
|
final String userAgentSuffix,
|
||||||
|
final StatisticsFromAwsSdk statisticsFromAwsSdks) {
|
||||||
AmazonS3 s3 = mock(AmazonS3.class);
|
AmazonS3 s3 = mock(AmazonS3.class);
|
||||||
when(s3.doesBucketExist(bucket)).thenReturn(true);
|
when(s3.doesBucketExist(bucket)).thenReturn(true);
|
||||||
when(s3.doesBucketExistV2(bucket)).thenReturn(true);
|
when(s3.doesBucketExistV2(bucket)).thenReturn(true);
|
||||||
|
|
|
@ -40,6 +40,8 @@ import org.apache.hadoop.fs.s3a.impl.ContextAccessors;
|
||||||
import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum;
|
import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum;
|
||||||
import org.apache.hadoop.fs.s3a.impl.StoreContext;
|
import org.apache.hadoop.fs.s3a.impl.StoreContext;
|
||||||
import org.apache.hadoop.fs.s3a.impl.StoreContextBuilder;
|
import org.apache.hadoop.fs.s3a.impl.StoreContextBuilder;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext;
|
||||||
import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
|
import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
|
||||||
import org.apache.hadoop.fs.s3a.s3guard.MetadataStoreCapabilities;
|
import org.apache.hadoop.fs.s3a.s3guard.MetadataStoreCapabilities;
|
||||||
import org.apache.hadoop.fs.s3a.s3guard.S3Guard;
|
import org.apache.hadoop.fs.s3a.s3guard.S3Guard;
|
||||||
|
@ -54,6 +56,7 @@ import org.apache.hadoop.service.Service;
|
||||||
import org.apache.hadoop.service.ServiceOperations;
|
import org.apache.hadoop.service.ServiceOperations;
|
||||||
import org.apache.hadoop.util.BlockingThreadPoolExecutorService;
|
import org.apache.hadoop.util.BlockingThreadPoolExecutorService;
|
||||||
import org.apache.hadoop.util.ReflectionUtils;
|
import org.apache.hadoop.util.ReflectionUtils;
|
||||||
|
import org.apache.hadoop.util.functional.CallableRaisingIOE;
|
||||||
|
|
||||||
import com.amazonaws.auth.AWSCredentialsProvider;
|
import com.amazonaws.auth.AWSCredentialsProvider;
|
||||||
import org.hamcrest.core.Is;
|
import org.hamcrest.core.Is;
|
||||||
|
@ -836,9 +839,9 @@ public final class S3ATestUtils {
|
||||||
* @param <T> type of operation.
|
* @param <T> type of operation.
|
||||||
*/
|
*/
|
||||||
public static <T> void callQuietly(final Logger log,
|
public static <T> void callQuietly(final Logger log,
|
||||||
final Invoker.Operation<T> operation) {
|
final CallableRaisingIOE<T> operation) {
|
||||||
try {
|
try {
|
||||||
operation.execute();
|
operation.apply();
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
log.info(e.toString(), e);
|
log.info(e.toString(), e);
|
||||||
}
|
}
|
||||||
|
@ -933,7 +936,7 @@ public final class S3ATestUtils {
|
||||||
.setExecutorCapacity(DEFAULT_EXECUTOR_CAPACITY)
|
.setExecutorCapacity(DEFAULT_EXECUTOR_CAPACITY)
|
||||||
.setInvoker(
|
.setInvoker(
|
||||||
new Invoker(RetryPolicies.TRY_ONCE_THEN_FAIL, Invoker.LOG_EVENT))
|
new Invoker(RetryPolicies.TRY_ONCE_THEN_FAIL, Invoker.LOG_EVENT))
|
||||||
.setInstrumentation(new S3AInstrumentation(name))
|
.setInstrumentation(new EmptyS3AStatisticsContext())
|
||||||
.setStorageStatistics(new S3AStorageStatistics())
|
.setStorageStatistics(new S3AStorageStatistics())
|
||||||
.setInputPolicy(S3AInputPolicy.Normal)
|
.setInputPolicy(S3AInputPolicy.Normal)
|
||||||
.setChangeDetectionPolicy(
|
.setChangeDetectionPolicy(
|
||||||
|
@ -1230,7 +1233,7 @@ public final class S3ATestUtils {
|
||||||
* @param out output stream
|
* @param out output stream
|
||||||
* @return the (active) stats of the write
|
* @return the (active) stats of the write
|
||||||
*/
|
*/
|
||||||
public static S3AInstrumentation.OutputStreamStatistics
|
public static BlockOutputStreamStatistics
|
||||||
getOutputStreamStatistics(FSDataOutputStream out) {
|
getOutputStreamStatistics(FSDataOutputStream out) {
|
||||||
S3ABlockOutputStream blockOutputStream
|
S3ABlockOutputStream blockOutputStream
|
||||||
= (S3ABlockOutputStream) out.getWrappedStream();
|
= (S3ABlockOutputStream) out.getWrappedStream();
|
||||||
|
|
|
@ -18,9 +18,13 @@
|
||||||
|
|
||||||
package org.apache.hadoop.fs.s3a;
|
package org.apache.hadoop.fs.s3a;
|
||||||
|
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.fs.RemoteIterator;
|
import org.apache.hadoop.fs.RemoteIterator;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatistics;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
|
||||||
|
import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
|
||||||
|
|
||||||
|
import org.assertj.core.api.Assertions;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
|
@ -31,8 +35,11 @@ import java.util.Iterator;
|
||||||
import java.util.NoSuchElementException;
|
import java.util.NoSuchElementException;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
|
||||||
import static org.apache.hadoop.fs.s3a.S3AUtils.ACCEPT_ALL;
|
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.extractStatistics;
|
||||||
import static org.apache.hadoop.fs.s3a.Listing.ProvidedFileStatusIterator;
|
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue;
|
||||||
|
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_LIST_REQUEST;
|
||||||
|
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore;
|
||||||
|
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Place for the S3A listing classes; keeps all the small classes under control.
|
* Place for the S3A listing classes; keeps all the small classes under control.
|
||||||
|
@ -40,11 +47,18 @@ import static org.apache.hadoop.fs.s3a.Listing.ProvidedFileStatusIterator;
|
||||||
public class TestListing extends AbstractS3AMockTest {
|
public class TestListing extends AbstractS3AMockTest {
|
||||||
|
|
||||||
private static class MockRemoteIterator<S3AFileStatus> implements
|
private static class MockRemoteIterator<S3AFileStatus> implements
|
||||||
RemoteIterator<S3AFileStatus> {
|
RemoteIterator<S3AFileStatus>, IOStatisticsSource {
|
||||||
|
|
||||||
|
private final IOStatisticsStore ioStatistics;
|
||||||
|
|
||||||
private Iterator<S3AFileStatus> iterator;
|
private Iterator<S3AFileStatus> iterator;
|
||||||
|
|
||||||
MockRemoteIterator(Collection<S3AFileStatus> source) {
|
MockRemoteIterator(Collection<S3AFileStatus> source) {
|
||||||
iterator = source.iterator();
|
iterator = source.iterator();
|
||||||
|
this.ioStatistics = iostatisticsStore()
|
||||||
|
.withDurationTracking(OBJECT_LIST_REQUEST)
|
||||||
|
.build();
|
||||||
|
ioStatistics.incrementCounter(OBJECT_LIST_REQUEST);
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean hasNext() {
|
public boolean hasNext() {
|
||||||
|
@ -54,6 +68,11 @@ public class TestListing extends AbstractS3AMockTest {
|
||||||
public S3AFileStatus next() {
|
public S3AFileStatus next() {
|
||||||
return iterator.next();
|
return iterator.next();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public IOStatistics getIOStatistics() {
|
||||||
|
return ioStatistics;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private S3AFileStatus blankFileStatus(Path path) {
|
private S3AFileStatus blankFileStatus(Path path) {
|
||||||
|
@ -65,11 +84,9 @@ public class TestListing extends AbstractS3AMockTest {
|
||||||
Path parent = new Path("/parent");
|
Path parent = new Path("/parent");
|
||||||
Path liveChild = new Path(parent, "/liveChild");
|
Path liveChild = new Path(parent, "/liveChild");
|
||||||
Path deletedChild = new Path(parent, "/deletedChild");
|
Path deletedChild = new Path(parent, "/deletedChild");
|
||||||
Path[] allFiles = {parent, liveChild, deletedChild};
|
|
||||||
Path[] liveFiles = {parent, liveChild};
|
|
||||||
|
|
||||||
Listing listing = fs.getListing();
|
Listing listing = fs.getListing();
|
||||||
Collection<FileStatus> statuses = new ArrayList<>();
|
Collection<S3AFileStatus> statuses = new ArrayList<>();
|
||||||
statuses.add(blankFileStatus(parent));
|
statuses.add(blankFileStatus(parent));
|
||||||
statuses.add(blankFileStatus(liveChild));
|
statuses.add(blankFileStatus(liveChild));
|
||||||
statuses.add(blankFileStatus(deletedChild));
|
statuses.add(blankFileStatus(deletedChild));
|
||||||
|
@ -92,7 +109,13 @@ public class TestListing extends AbstractS3AMockTest {
|
||||||
while (reconcilingIterator.hasNext()) {
|
while (reconcilingIterator.hasNext()) {
|
||||||
actualPaths.add(reconcilingIterator.next().getPath());
|
actualPaths.add(reconcilingIterator.next().getPath());
|
||||||
}
|
}
|
||||||
Assert.assertTrue(actualPaths.equals(expectedPaths));
|
Assertions.assertThat(actualPaths)
|
||||||
|
.describedAs("paths from iterator")
|
||||||
|
.isEqualTo(expectedPaths);
|
||||||
|
|
||||||
|
// now verify the stats went all the way through.
|
||||||
|
IOStatistics iostats = extractStatistics(reconcilingIterator);
|
||||||
|
verifyStatisticCounterValue(iostats, OBJECT_LIST_REQUEST, 1);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -104,18 +127,13 @@ public class TestListing extends AbstractS3AMockTest {
|
||||||
S3AFileStatus[] statuses = {
|
S3AFileStatus[] statuses = {
|
||||||
s3aStatus
|
s3aStatus
|
||||||
};
|
};
|
||||||
ProvidedFileStatusIterator it = new ProvidedFileStatusIterator(statuses,
|
RemoteIterator<S3AFileStatus> it = Listing.toProvidedFileStatusIterator(
|
||||||
ACCEPT_ALL, new Listing.AcceptAllButS3nDirs());
|
statuses);
|
||||||
|
|
||||||
Assert.assertTrue("hasNext() should return true first time", it.hasNext());
|
Assert.assertTrue("hasNext() should return true first time", it.hasNext());
|
||||||
Assert.assertNotNull("first element should not be null", it.next());
|
Assert.assertEquals("first element from iterator",
|
||||||
|
s3aStatus, it.next());
|
||||||
Assert.assertFalse("hasNext() should now be false", it.hasNext());
|
Assert.assertFalse("hasNext() should now be false", it.hasNext());
|
||||||
try {
|
intercept(NoSuchElementException.class, it::next);
|
||||||
it.next();
|
|
||||||
Assert.fail("next() should have thrown exception");
|
|
||||||
} catch (NoSuchElementException e) {
|
|
||||||
// Correct behavior. Any other exceptions are propagated as failure.
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.hadoop.fs.s3a;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.PathIOException;
|
import org.apache.hadoop.fs.PathIOException;
|
||||||
import org.apache.hadoop.fs.s3a.commit.PutTracker;
|
import org.apache.hadoop.fs.s3a.commit.PutTracker;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext;
|
||||||
import org.apache.hadoop.util.Progressable;
|
import org.apache.hadoop.util.Progressable;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
@ -30,7 +31,6 @@ import java.io.IOException;
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
|
|
||||||
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
|
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
|
||||||
import static org.junit.Assert.*;
|
|
||||||
import static org.mockito.Mockito.doThrow;
|
import static org.mockito.Mockito.doThrow;
|
||||||
import static org.mockito.Mockito.mock;
|
import static org.mockito.Mockito.mock;
|
||||||
import static org.mockito.Mockito.spy;
|
import static org.mockito.Mockito.spy;
|
||||||
|
@ -50,11 +50,10 @@ public class TestS3ABlockOutputStream extends AbstractS3AMockTest {
|
||||||
S3ADataBlocks.BlockFactory blockFactory =
|
S3ADataBlocks.BlockFactory blockFactory =
|
||||||
mock(S3ADataBlocks.BlockFactory.class);
|
mock(S3ADataBlocks.BlockFactory.class);
|
||||||
long blockSize = Constants.DEFAULT_MULTIPART_SIZE;
|
long blockSize = Constants.DEFAULT_MULTIPART_SIZE;
|
||||||
S3AInstrumentation.OutputStreamStatistics statistics = null;
|
|
||||||
WriteOperationHelper oHelper = mock(WriteOperationHelper.class);
|
WriteOperationHelper oHelper = mock(WriteOperationHelper.class);
|
||||||
PutTracker putTracker = mock(PutTracker.class);
|
PutTracker putTracker = mock(PutTracker.class);
|
||||||
stream = spy(new S3ABlockOutputStream(fs, "", executorService,
|
stream = spy(new S3ABlockOutputStream(fs, "", executorService,
|
||||||
progressable, blockSize, blockFactory, statistics, oHelper,
|
progressable, blockSize, blockFactory, null, oHelper,
|
||||||
putTracker));
|
putTracker));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -70,7 +69,8 @@ public class TestS3ABlockOutputStream extends AbstractS3AMockTest {
|
||||||
S3AFileSystem s3a = mock(S3AFileSystem.class);
|
S3AFileSystem s3a = mock(S3AFileSystem.class);
|
||||||
when(s3a.getBucket()).thenReturn("bucket");
|
when(s3a.getBucket()).thenReturn("bucket");
|
||||||
WriteOperationHelper woh = new WriteOperationHelper(s3a,
|
WriteOperationHelper woh = new WriteOperationHelper(s3a,
|
||||||
new Configuration());
|
new Configuration(),
|
||||||
|
new EmptyS3AStatisticsContext());
|
||||||
ByteArrayInputStream inputStream = new ByteArrayInputStream(
|
ByteArrayInputStream inputStream = new ByteArrayInputStream(
|
||||||
"a".getBytes());
|
"a".getBytes());
|
||||||
// first one works
|
// first one works
|
||||||
|
|
|
@ -18,8 +18,6 @@
|
||||||
|
|
||||||
package org.apache.hadoop.fs.s3a;
|
package org.apache.hadoop.fs.s3a;
|
||||||
|
|
||||||
import java.util.concurrent.atomic.AtomicLong;
|
|
||||||
|
|
||||||
import com.amazonaws.AmazonServiceException;
|
import com.amazonaws.AmazonServiceException;
|
||||||
import com.amazonaws.SdkBaseException;
|
import com.amazonaws.SdkBaseException;
|
||||||
import com.amazonaws.services.s3.Headers;
|
import com.amazonaws.services.s3.Headers;
|
||||||
|
@ -36,6 +34,7 @@ import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.fs.PathIOException;
|
import org.apache.hadoop.fs.PathIOException;
|
||||||
import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy;
|
import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy;
|
||||||
import org.apache.hadoop.fs.s3a.impl.ChangeTracker;
|
import org.apache.hadoop.fs.s3a.impl.ChangeTracker;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.impl.CountingChangeTracker;
|
||||||
import org.apache.hadoop.test.HadoopTestBase;
|
import org.apache.hadoop.test.HadoopTestBase;
|
||||||
|
|
||||||
import static org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy.CHANGE_DETECTED;
|
import static org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy.CHANGE_DETECTED;
|
||||||
|
@ -359,7 +358,7 @@ public class TestStreamChangeTracker extends HadoopTestBase {
|
||||||
final ChangeTracker tracker,
|
final ChangeTracker tracker,
|
||||||
final int expectedCount) {
|
final int expectedCount) {
|
||||||
assertEquals("counter in tracker " + tracker,
|
assertEquals("counter in tracker " + tracker,
|
||||||
expectedCount, tracker.getVersionMismatches().get());
|
expectedCount, tracker.getVersionMismatches());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -386,7 +385,7 @@ public class TestStreamChangeTracker extends HadoopTestBase {
|
||||||
source,
|
source,
|
||||||
requireVersion);
|
requireVersion);
|
||||||
ChangeTracker tracker = new ChangeTracker(URI, policy,
|
ChangeTracker tracker = new ChangeTracker(URI, policy,
|
||||||
new AtomicLong(0), objectAttributes);
|
new CountingChangeTracker(), objectAttributes);
|
||||||
if (objectAttributes.getVersionId() == null
|
if (objectAttributes.getVersionId() == null
|
||||||
&& objectAttributes.getETag() == null) {
|
&& objectAttributes.getETag() == null) {
|
||||||
assertFalse("Tracker should not have applied constraints " + tracker,
|
assertFalse("Tracker should not have applied constraints " + tracker,
|
||||||
|
|
|
@ -51,6 +51,7 @@ import org.apache.hadoop.fs.s3a.commit.CommitOperations;
|
||||||
import org.apache.hadoop.fs.s3a.commit.files.PendingSet;
|
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.SinglePendingCommit;
|
||||||
import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool;
|
import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics;
|
||||||
|
|
||||||
import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
|
import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
|
||||||
import static org.apache.hadoop.fs.s3a.Constants.*;
|
import static org.apache.hadoop.fs.s3a.Constants.*;
|
||||||
|
@ -62,6 +63,7 @@ import static org.apache.hadoop.fs.s3a.auth.RolePolicies.*;
|
||||||
import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.forbidden;
|
import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.forbidden;
|
||||||
import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.newAssumedRoleConfig;
|
import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.newAssumedRoleConfig;
|
||||||
import static org.apache.hadoop.fs.s3a.s3guard.S3GuardToolTestHelper.exec;
|
import static org.apache.hadoop.fs.s3a.s3guard.S3GuardToolTestHelper.exec;
|
||||||
|
import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsSourceToString;
|
||||||
import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
|
import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
|
||||||
import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
|
import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
|
||||||
import static org.apache.hadoop.test.LambdaTestUtils.*;
|
import static org.apache.hadoop.test.LambdaTestUtils.*;
|
||||||
|
@ -573,8 +575,11 @@ public class ITestAssumeRole extends AbstractS3ATestBase {
|
||||||
.addResources(directory(writeableDir))
|
.addResources(directory(writeableDir))
|
||||||
);
|
);
|
||||||
roleFS = (S3AFileSystem) writeableDir.getFileSystem(conf);
|
roleFS = (S3AFileSystem) writeableDir.getFileSystem(conf);
|
||||||
CommitOperations fullOperations = new CommitOperations(fs);
|
CommitterStatistics committerStatistics = fs.newCommitterStatistics();
|
||||||
CommitOperations operations = new CommitOperations(roleFS);
|
CommitOperations fullOperations = new CommitOperations(fs,
|
||||||
|
committerStatistics);
|
||||||
|
CommitOperations operations = new CommitOperations(roleFS,
|
||||||
|
committerStatistics);
|
||||||
|
|
||||||
File localSrc = File.createTempFile("source", "");
|
File localSrc = File.createTempFile("source", "");
|
||||||
writeCSVData(localSrc);
|
writeCSVData(localSrc);
|
||||||
|
@ -608,7 +613,7 @@ public class ITestAssumeRole extends AbstractS3ATestBase {
|
||||||
name + CommitConstants.PENDING_SUFFIX), true);
|
name + CommitConstants.PENDING_SUFFIX), true);
|
||||||
assertTrue(src.delete());
|
assertTrue(src.delete());
|
||||||
}));
|
}));
|
||||||
progress.assertCount("Process counter is not expected",
|
progress.assertCount("progress counter is not expected",
|
||||||
range);
|
range);
|
||||||
|
|
||||||
try {
|
try {
|
||||||
|
@ -652,6 +657,8 @@ public class ITestAssumeRole extends AbstractS3ATestBase {
|
||||||
} finally {
|
} finally {
|
||||||
LOG.info("Cleanup");
|
LOG.info("Cleanup");
|
||||||
fullOperations.abortPendingUploadsUnderPath(readOnlyDir);
|
fullOperations.abortPendingUploadsUnderPath(readOnlyDir);
|
||||||
|
LOG.info("Committer statistics {}",
|
||||||
|
ioStatisticsSourceToString(committerStatistics));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -724,7 +724,7 @@ public class ITestRestrictedReadAccess extends AbstractS3ATestBase {
|
||||||
S3ATestUtils.MetricDiff getMetric = new S3ATestUtils.MetricDiff(fs,
|
S3ATestUtils.MetricDiff getMetric = new S3ATestUtils.MetricDiff(fs,
|
||||||
Statistic.OBJECT_METADATA_REQUESTS);
|
Statistic.OBJECT_METADATA_REQUESTS);
|
||||||
S3ATestUtils.MetricDiff listMetric = new S3ATestUtils.MetricDiff(fs,
|
S3ATestUtils.MetricDiff listMetric = new S3ATestUtils.MetricDiff(fs,
|
||||||
Statistic.OBJECT_LIST_REQUESTS);
|
Statistic.OBJECT_LIST_REQUEST);
|
||||||
FileStatus[] st;
|
FileStatus[] st;
|
||||||
try {
|
try {
|
||||||
st = filter == null
|
st = filter == null
|
||||||
|
|
|
@ -18,6 +18,8 @@
|
||||||
|
|
||||||
package org.apache.hadoop.fs.s3a.auth;
|
package org.apache.hadoop.fs.s3a.auth;
|
||||||
|
|
||||||
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
|
|
||||||
import org.apache.hadoop.util.Progressable;
|
import org.apache.hadoop.util.Progressable;
|
||||||
|
|
||||||
import static org.junit.Assert.assertEquals;
|
import static org.junit.Assert.assertEquals;
|
||||||
|
@ -27,14 +29,14 @@ import static org.junit.Assert.assertEquals;
|
||||||
*/
|
*/
|
||||||
public class ProgressCounter implements Progressable {
|
public class ProgressCounter implements Progressable {
|
||||||
|
|
||||||
private long count;
|
private final AtomicLong count = new AtomicLong();
|
||||||
|
|
||||||
public void progress() {
|
public void progress() {
|
||||||
count++;
|
count.incrementAndGet();
|
||||||
}
|
}
|
||||||
|
|
||||||
public long getCount() {
|
public long getCount() {
|
||||||
return count;
|
return count.get();
|
||||||
}
|
}
|
||||||
|
|
||||||
public void assertCount(String message, int expected) {
|
public void assertCount(String message, int expected) {
|
||||||
|
|
|
@ -44,6 +44,8 @@ import org.apache.hadoop.fs.s3a.S3AEncryptionMethods;
|
||||||
import org.apache.hadoop.fs.s3a.S3AFileSystem;
|
import org.apache.hadoop.fs.s3a.S3AFileSystem;
|
||||||
import org.apache.hadoop.fs.s3a.S3ATestUtils;
|
import org.apache.hadoop.fs.s3a.S3ATestUtils;
|
||||||
import org.apache.hadoop.fs.s3a.Statistic;
|
import org.apache.hadoop.fs.s3a.Statistic;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.StatisticsFromAwsSdk;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext;
|
||||||
import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher;
|
import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher;
|
||||||
import org.apache.hadoop.io.Text;
|
import org.apache.hadoop.io.Text;
|
||||||
import org.apache.hadoop.mapreduce.security.TokenCache;
|
import org.apache.hadoop.mapreduce.security.TokenCache;
|
||||||
|
@ -70,6 +72,7 @@ import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationTokenIOExceptio
|
||||||
import static org.apache.hadoop.fs.s3a.auth.delegation.MiniKerberizedHadoopCluster.ALICE;
|
import static org.apache.hadoop.fs.s3a.auth.delegation.MiniKerberizedHadoopCluster.ALICE;
|
||||||
import static org.apache.hadoop.fs.s3a.auth.delegation.MiniKerberizedHadoopCluster.assertSecurityEnabled;
|
import static org.apache.hadoop.fs.s3a.auth.delegation.MiniKerberizedHadoopCluster.assertSecurityEnabled;
|
||||||
import static org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens.lookupS3ADelegationToken;
|
import static org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens.lookupS3ADelegationToken;
|
||||||
|
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.AWS_SDK_METRICS_ENABLED;
|
||||||
import static org.apache.hadoop.test.LambdaTestUtils.doAs;
|
import static org.apache.hadoop.test.LambdaTestUtils.doAs;
|
||||||
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
|
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
|
||||||
import static org.hamcrest.Matchers.containsString;
|
import static org.hamcrest.Matchers.containsString;
|
||||||
|
@ -211,7 +214,7 @@ public class ITestSessionDelegationInFileystem extends AbstractDelegationIT {
|
||||||
S3ATestUtils.MetricDiff invocationDiff = new S3ATestUtils.MetricDiff(fs,
|
S3ATestUtils.MetricDiff invocationDiff = new S3ATestUtils.MetricDiff(fs,
|
||||||
Statistic.INVOCATION_GET_DELEGATION_TOKEN);
|
Statistic.INVOCATION_GET_DELEGATION_TOKEN);
|
||||||
S3ATestUtils.MetricDiff issueDiff = new S3ATestUtils.MetricDiff(fs,
|
S3ATestUtils.MetricDiff issueDiff = new S3ATestUtils.MetricDiff(fs,
|
||||||
Statistic.DELEGATION_TOKENS_ISSUED);
|
Statistic.DELEGATION_TOKEN_ISSUED);
|
||||||
Token<AbstractS3ATokenIdentifier> token =
|
Token<AbstractS3ATokenIdentifier> token =
|
||||||
requireNonNull(fs.getDelegationToken(""),
|
requireNonNull(fs.getDelegationToken(""),
|
||||||
"no token from filesystem " + fs);
|
"no token from filesystem " + fs);
|
||||||
|
@ -305,6 +308,9 @@ public class ITestSessionDelegationInFileystem extends AbstractDelegationIT {
|
||||||
describe("Delegation tokens can be passed to a new filesystem;"
|
describe("Delegation tokens can be passed to a new filesystem;"
|
||||||
+ " if role restricted, permissions are tightened.");
|
+ " if role restricted, permissions are tightened.");
|
||||||
S3AFileSystem fs = getFileSystem();
|
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);
|
readLandsatMetadata(fs);
|
||||||
|
|
||||||
URI uri = fs.getUri();
|
URI uri = fs.getUri();
|
||||||
|
@ -365,7 +371,7 @@ public class ITestSessionDelegationInFileystem extends AbstractDelegationIT {
|
||||||
|
|
||||||
S3ATestUtils.MetricDiff issueDiff = new S3ATestUtils.MetricDiff(
|
S3ATestUtils.MetricDiff issueDiff = new S3ATestUtils.MetricDiff(
|
||||||
delegatedFS,
|
delegatedFS,
|
||||||
Statistic.DELEGATION_TOKENS_ISSUED);
|
Statistic.DELEGATION_TOKEN_ISSUED);
|
||||||
|
|
||||||
// verify that the FS returns the existing token when asked
|
// verify that the FS returns the existing token when asked
|
||||||
// so that chained deployments will work
|
// so that chained deployments will work
|
||||||
|
@ -561,8 +567,13 @@ public class ITestSessionDelegationInFileystem extends AbstractDelegationIT {
|
||||||
conf.set(ENDPOINT, "");
|
conf.set(ENDPOINT, "");
|
||||||
factory.setConf(conf);
|
factory.setConf(conf);
|
||||||
String host = landsat.getHost();
|
String host = landsat.getHost();
|
||||||
|
StatisticsFromAwsSdk awsStats = null;
|
||||||
|
if (AWS_SDK_METRICS_ENABLED) {
|
||||||
|
awsStats = new EmptyS3AStatisticsContext()
|
||||||
|
.newStatisticsFromAwsSdk();
|
||||||
|
}
|
||||||
AmazonS3 s3 = factory.createS3Client(landsat, host, testing,
|
AmazonS3 s3 = factory.createS3Client(landsat, host, testing,
|
||||||
"ITestSessionDelegationInFileystem");
|
"ITestSessionDelegationInFileystem", awsStats);
|
||||||
|
|
||||||
return Invoker.once("HEAD", host,
|
return Invoker.once("HEAD", host,
|
||||||
() -> s3.getObjectMetadata(host, landsat.getPath().substring(1)));
|
() -> s3.getObjectMetadata(host, landsat.getPath().substring(1)));
|
||||||
|
|
|
@ -53,6 +53,7 @@ import static org.apache.hadoop.fs.s3a.Constants.*;
|
||||||
import static org.apache.hadoop.fs.s3a.MultipartTestUtils.listMultipartUploads;
|
import static org.apache.hadoop.fs.s3a.MultipartTestUtils.listMultipartUploads;
|
||||||
import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
|
import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
|
||||||
import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
|
import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
|
||||||
|
import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToString;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Base test suite for committer operations.
|
* Base test suite for committer operations.
|
||||||
|
@ -460,6 +461,8 @@ public abstract class AbstractCommitITest extends AbstractS3ATestBase {
|
||||||
commitDetails);
|
commitDetails);
|
||||||
LOG.info("Committer statistics: \n{}",
|
LOG.info("Committer statistics: \n{}",
|
||||||
successData.dumpMetrics(" ", " = ", "\n"));
|
successData.dumpMetrics(" ", " = ", "\n"));
|
||||||
|
LOG.info("Job IOStatistics: \n{}",
|
||||||
|
ioStatisticsToString(successData.getIOStatistics()));
|
||||||
LOG.info("Diagnostics\n{}",
|
LOG.info("Diagnostics\n{}",
|
||||||
successData.dumpDiagnostics(" ", " = ", "\n"));
|
successData.dumpDiagnostics(" ", " = ", "\n"));
|
||||||
if (!committerName.isEmpty()) {
|
if (!committerName.isEmpty()) {
|
||||||
|
@ -486,7 +489,7 @@ public abstract class AbstractCommitITest extends AbstractS3ATestBase {
|
||||||
* @throws IOException failure to find/load the file
|
* @throws IOException failure to find/load the file
|
||||||
* @throws AssertionError file is 0-bytes long,
|
* @throws AssertionError file is 0-bytes long,
|
||||||
*/
|
*/
|
||||||
public static SuccessData loadSuccessFile(final S3AFileSystem fs,
|
public static SuccessData loadSuccessFile(final FileSystem fs,
|
||||||
final Path outputPath, final String origin) throws IOException {
|
final Path outputPath, final String origin) throws IOException {
|
||||||
ContractTestUtils.assertPathExists(fs,
|
ContractTestUtils.assertPathExists(fs,
|
||||||
"Output directory " + outputPath
|
"Output directory " + outputPath
|
||||||
|
@ -505,7 +508,9 @@ public abstract class AbstractCommitITest extends AbstractS3ATestBase {
|
||||||
+ success + " from " + origin
|
+ success + " from " + origin
|
||||||
+ "; an S3A committer was not used",
|
+ "; an S3A committer was not used",
|
||||||
status.getLen() > 0);
|
status.getLen() > 0);
|
||||||
LOG.info("Loading committer success file {}", success);
|
String body = ContractTestUtils.readUTF8(fs, success, -1);
|
||||||
|
LOG.info("Loading committer success file {}. Actual contents=\n{}", success,
|
||||||
|
body);
|
||||||
return SuccessData.load(fs, success);
|
return SuccessData.load(fs, success);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -43,6 +43,7 @@ import org.apache.hadoop.fs.contract.ContractTestUtils;
|
||||||
import org.apache.hadoop.fs.s3a.S3AFileSystem;
|
import org.apache.hadoop.fs.s3a.S3AFileSystem;
|
||||||
import org.apache.hadoop.fs.s3a.commit.files.SuccessData;
|
import org.apache.hadoop.fs.s3a.commit.files.SuccessData;
|
||||||
import org.apache.hadoop.fs.s3a.commit.magic.MagicS3GuardCommitter;
|
import org.apache.hadoop.fs.s3a.commit.magic.MagicS3GuardCommitter;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
|
||||||
import org.apache.hadoop.io.IntWritable;
|
import org.apache.hadoop.io.IntWritable;
|
||||||
import org.apache.hadoop.io.LongWritable;
|
import org.apache.hadoop.io.LongWritable;
|
||||||
import org.apache.hadoop.io.MapFile;
|
import org.apache.hadoop.io.MapFile;
|
||||||
|
@ -77,6 +78,9 @@ import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.E_NO_SP
|
||||||
import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.FS_S3A_COMMITTER_UUID;
|
import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.FS_S3A_COMMITTER_UUID;
|
||||||
import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.FS_S3A_COMMITTER_UUID_SOURCE;
|
import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.FS_S3A_COMMITTER_UUID_SOURCE;
|
||||||
import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.SPARK_WRITE_UUID;
|
import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.SPARK_WRITE_UUID;
|
||||||
|
import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_TASKS_SUCCEEDED;
|
||||||
|
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticCounter;
|
||||||
|
import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsSourceToString;
|
||||||
import static org.apache.hadoop.test.LambdaTestUtils.*;
|
import static org.apache.hadoop.test.LambdaTestUtils.*;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -1339,9 +1343,28 @@ public abstract class AbstractITCommitProtocol extends AbstractCommitITest {
|
||||||
assertTrue("Committer does not have data to commit " + committer,
|
assertTrue("Committer does not have data to commit " + committer,
|
||||||
committer.needsTaskCommit(tContext));
|
committer.needsTaskCommit(tContext));
|
||||||
commitTask(committer, tContext);
|
commitTask(committer, tContext);
|
||||||
|
// at this point the committer tasks stats should be current.
|
||||||
|
IOStatisticsSnapshot snapshot = new IOStatisticsSnapshot(
|
||||||
|
committer.getIOStatistics());
|
||||||
|
String commitsCompleted = COMMITTER_TASKS_SUCCEEDED.getSymbol();
|
||||||
|
assertThatStatisticCounter(snapshot, commitsCompleted)
|
||||||
|
.describedAs("task commit count")
|
||||||
|
.isEqualTo(1L);
|
||||||
|
|
||||||
|
|
||||||
commitJob(committer, jContext);
|
commitJob(committer, jContext);
|
||||||
|
LOG.info("committer iostatistics {}",
|
||||||
|
ioStatisticsSourceToString(committer));
|
||||||
|
|
||||||
// validate output
|
// validate output
|
||||||
verifySuccessMarker(outDir, committer.getUUID());
|
SuccessData successData = verifySuccessMarker(outDir, committer.getUUID());
|
||||||
|
|
||||||
|
// the task commit count should get through the job commit
|
||||||
|
IOStatisticsSnapshot successStats = successData.getIOStatistics();
|
||||||
|
LOG.info("loaded statistics {}", successStats);
|
||||||
|
assertThatStatisticCounter(successStats, commitsCompleted)
|
||||||
|
.describedAs("task commit count")
|
||||||
|
.isEqualTo(1L);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -80,7 +80,6 @@ import static org.apache.hadoop.fs.s3a.commit.CommitConstants._SUCCESS;
|
||||||
import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.FS_S3A_COMMITTER_UUID;
|
import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.FS_S3A_COMMITTER_UUID;
|
||||||
import static org.apache.hadoop.fs.s3a.commit.staging.Paths.getMultipartUploadCommitsDirectory;
|
import static org.apache.hadoop.fs.s3a.commit.staging.Paths.getMultipartUploadCommitsDirectory;
|
||||||
import static org.apache.hadoop.fs.s3a.commit.staging.StagingCommitterConstants.STAGING_UPLOADS;
|
import static org.apache.hadoop.fs.s3a.commit.staging.StagingCommitterConstants.STAGING_UPLOADS;
|
||||||
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Test an MR Job with all the different committers.
|
* Test an MR Job with all the different committers.
|
||||||
|
|
|
@ -28,6 +28,7 @@ import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.fs.s3a.MockS3AFileSystem;
|
import org.apache.hadoop.fs.s3a.MockS3AFileSystem;
|
||||||
import org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase.ClientErrors;
|
import org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase.ClientErrors;
|
||||||
import org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase.ClientResults;
|
import org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase.ClientResults;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
|
||||||
import org.apache.hadoop.mapreduce.JobContext;
|
import org.apache.hadoop.mapreduce.JobContext;
|
||||||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||||
|
|
||||||
|
@ -76,7 +77,8 @@ class MockedStagingCommitter extends StagingCommitter {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void maybeCreateSuccessMarker(JobContext context,
|
protected void maybeCreateSuccessMarker(JobContext context,
|
||||||
List<String> filenames)
|
List<String> filenames,
|
||||||
|
final IOStatisticsSnapshot ioStatistics)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
//skipped
|
//skipped
|
||||||
}
|
}
|
||||||
|
|
|
@ -57,8 +57,9 @@ import static org.apache.hadoop.fs.s3a.S3ATestUtils.MetricDiff;
|
||||||
import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
|
import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
|
||||||
import static org.apache.hadoop.fs.s3a.S3AUtils.applyLocatedFiles;
|
import static org.apache.hadoop.fs.s3a.S3AUtils.applyLocatedFiles;
|
||||||
import static org.apache.hadoop.fs.s3a.Statistic.FILES_DELETE_REJECTED;
|
import static org.apache.hadoop.fs.s3a.Statistic.FILES_DELETE_REJECTED;
|
||||||
|
import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_BULK_DELETE_REQUEST;
|
||||||
import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_DELETE_OBJECTS;
|
import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_DELETE_OBJECTS;
|
||||||
import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_DELETE_REQUESTS;
|
import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_DELETE_REQUEST;
|
||||||
import static org.apache.hadoop.fs.s3a.auth.RoleModel.Effects;
|
import static org.apache.hadoop.fs.s3a.auth.RoleModel.Effects;
|
||||||
import static org.apache.hadoop.fs.s3a.auth.RoleModel.Statement;
|
import static org.apache.hadoop.fs.s3a.auth.RoleModel.Statement;
|
||||||
import static org.apache.hadoop.fs.s3a.auth.RoleModel.directory;
|
import static org.apache.hadoop.fs.s3a.auth.RoleModel.directory;
|
||||||
|
@ -664,7 +665,9 @@ public class ITestPartialRenamesDeletes extends AbstractS3ATestBase {
|
||||||
|
|
||||||
// this set can be deleted by the role FS
|
// this set can be deleted by the role FS
|
||||||
MetricDiff rejectionCount = new MetricDiff(roleFS, FILES_DELETE_REJECTED);
|
MetricDiff rejectionCount = new MetricDiff(roleFS, FILES_DELETE_REJECTED);
|
||||||
MetricDiff deleteVerbCount = new MetricDiff(roleFS, OBJECT_DELETE_REQUESTS);
|
MetricDiff deleteVerbCount = new MetricDiff(roleFS, OBJECT_DELETE_REQUEST);
|
||||||
|
MetricDiff bulkDeleteVerbCount = new MetricDiff(roleFS,
|
||||||
|
OBJECT_BULK_DELETE_REQUEST);
|
||||||
MetricDiff deleteObjectCount = new MetricDiff(roleFS,
|
MetricDiff deleteObjectCount = new MetricDiff(roleFS,
|
||||||
OBJECT_DELETE_OBJECTS);
|
OBJECT_DELETE_OBJECTS);
|
||||||
|
|
||||||
|
@ -673,7 +676,9 @@ public class ITestPartialRenamesDeletes extends AbstractS3ATestBase {
|
||||||
if (multiDelete) {
|
if (multiDelete) {
|
||||||
// multi-delete status checks
|
// multi-delete status checks
|
||||||
extractCause(MultiObjectDeleteException.class, ex);
|
extractCause(MultiObjectDeleteException.class, ex);
|
||||||
deleteVerbCount.assertDiffEquals("Wrong delete request count", 1);
|
deleteVerbCount.assertDiffEquals("Wrong delete request count", 0);
|
||||||
|
bulkDeleteVerbCount.assertDiffEquals("Wrong bulk delete request count",
|
||||||
|
1);
|
||||||
deleteObjectCount.assertDiffEquals("Number of keys in delete request",
|
deleteObjectCount.assertDiffEquals("Number of keys in delete request",
|
||||||
readOnlyFiles.size());
|
readOnlyFiles.size());
|
||||||
rejectionCount.assertDiffEquals("Wrong rejection count",
|
rejectionCount.assertDiffEquals("Wrong rejection count",
|
||||||
|
|
|
@ -0,0 +1,105 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.fs.s3a.impl;
|
||||||
|
|
||||||
|
import com.amazonaws.ClientConfiguration;
|
||||||
|
import com.amazonaws.client.builder.AwsClientBuilder;
|
||||||
|
import org.junit.Ignore;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import org.apache.hadoop.test.AbstractHadoopTestBase;
|
||||||
|
|
||||||
|
import static org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.createEndpointConfiguration;
|
||||||
|
import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.fixBucketRegion;
|
||||||
|
import static org.assertj.core.api.Assertions.assertThat;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Unit tests related to the {@link NetworkBinding} class.
|
||||||
|
*/
|
||||||
|
public class TestNetworkBinding extends AbstractHadoopTestBase {
|
||||||
|
|
||||||
|
private static final String US_EAST_1 = "us-east-1";
|
||||||
|
|
||||||
|
private static final String US_WEST_2 = "us-west-2";
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testUSEast() {
|
||||||
|
assertRegionFixup(US_EAST_1, US_EAST_1);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testUSWest() {
|
||||||
|
assertRegionFixup(US_WEST_2, US_WEST_2);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testRegionUStoUSEast() {
|
||||||
|
assertRegionFixup("US", US_EAST_1);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testRegionNullToUSEast() {
|
||||||
|
assertRegionFixup(null, US_EAST_1);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static void assertRegionFixup(String region, String expected) {
|
||||||
|
assertThat(fixBucketRegion(region))
|
||||||
|
.describedAs("Fixup of %s", region)
|
||||||
|
.isEqualTo(expected);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testNull() throws Throwable {
|
||||||
|
expectEndpoint("", true, "unused");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
@Ignore("disabled until endpoint logic works for S3 client builder API")
|
||||||
|
public void testUSEastEndpoint() throws Throwable {
|
||||||
|
expectEndpoint(US_EAST_1, false, US_EAST_1);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
@Ignore("disabled until endpoint logic works for S3 client builder API")
|
||||||
|
public void testUSWestEndpoint() throws Throwable {
|
||||||
|
expectEndpoint(US_WEST_2, false, US_WEST_2);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void expectEndpoint(final String src,
|
||||||
|
final boolean expectNull,
|
||||||
|
final String expectRegion) {
|
||||||
|
AwsClientBuilder.EndpointConfiguration epr =
|
||||||
|
createEndpointConfiguration(src, new ClientConfiguration());
|
||||||
|
String eprStr = epr == null
|
||||||
|
? "(empty)"
|
||||||
|
: ("(" + epr.getServiceEndpoint() + " " + epr.getSigningRegion());
|
||||||
|
if (expectNull) {
|
||||||
|
assertThat(epr)
|
||||||
|
.describedAs("Endpoint configuration of %s =",
|
||||||
|
src, eprStr)
|
||||||
|
.isNull();
|
||||||
|
} else {
|
||||||
|
assertThat(epr)
|
||||||
|
.describedAs("Endpoint configuration of %s =",
|
||||||
|
src, eprStr)
|
||||||
|
.hasFieldOrPropertyWithValue("serviceEndpoint", src)
|
||||||
|
.hasFieldOrPropertyWithValue("signingRegion", expectRegion);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -1,61 +0,0 @@
|
||||||
/*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
|
||||||
* or more contributor license agreements. See the NOTICE file
|
|
||||||
* distributed with this work for additional information
|
|
||||||
* regarding copyright ownership. The ASF licenses this file
|
|
||||||
* to you under the Apache License, Version 2.0 (the
|
|
||||||
* "License"); you may not use this file except in compliance
|
|
||||||
* with the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
|
|
||||||
package org.apache.hadoop.fs.s3a.impl;
|
|
||||||
|
|
||||||
import org.assertj.core.api.Assertions;
|
|
||||||
import org.junit.Test;
|
|
||||||
|
|
||||||
import org.apache.hadoop.test.HadoopTestBase;
|
|
||||||
|
|
||||||
import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.fixBucketRegion;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Unit tests related to the {@link NetworkBinding} class.
|
|
||||||
*/
|
|
||||||
public class TestNeworkBinding extends HadoopTestBase {
|
|
||||||
|
|
||||||
private static final String US_EAST_1 = "us-east-1";
|
|
||||||
private static final String US_WEST_2 = "us-west-2";
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testUSEast() {
|
|
||||||
assertRegionFixup(US_EAST_1, US_EAST_1);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testUSWest() {
|
|
||||||
assertRegionFixup(US_WEST_2, US_WEST_2);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testRegionUStoUSEast() {
|
|
||||||
assertRegionFixup("US", US_EAST_1);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testRegionNullToUSEast() {
|
|
||||||
assertRegionFixup(null, US_EAST_1);
|
|
||||||
}
|
|
||||||
|
|
||||||
private static void assertRegionFixup(String region, String expected) {
|
|
||||||
Assertions.assertThat(fixBucketRegion(region))
|
|
||||||
.describedAs("Fixup of %s", region)
|
|
||||||
.isEqualTo(expected);
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -20,6 +20,7 @@ package org.apache.hadoop.fs.s3a.performance;
|
||||||
|
|
||||||
import java.io.FileNotFoundException;
|
import java.io.FileNotFoundException;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.EnumSet;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.Callable;
|
import java.util.concurrent.Callable;
|
||||||
|
|
||||||
|
@ -31,16 +32,19 @@ import org.apache.hadoop.fs.FSDataOutputStreamBuilder;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.fs.contract.ContractTestUtils;
|
import org.apache.hadoop.fs.contract.ContractTestUtils;
|
||||||
import org.apache.hadoop.fs.s3a.AbstractS3ATestBase;
|
import org.apache.hadoop.fs.s3a.AbstractS3ATestBase;
|
||||||
|
import org.apache.hadoop.fs.s3a.Constants;
|
||||||
import org.apache.hadoop.fs.s3a.S3AFileStatus;
|
import org.apache.hadoop.fs.s3a.S3AFileStatus;
|
||||||
import org.apache.hadoop.fs.s3a.S3AFileSystem;
|
import org.apache.hadoop.fs.s3a.S3AFileSystem;
|
||||||
import org.apache.hadoop.fs.s3a.Statistic;
|
import org.apache.hadoop.fs.s3a.Statistic;
|
||||||
import org.apache.hadoop.fs.s3a.Tristate;
|
import org.apache.hadoop.fs.s3a.Tristate;
|
||||||
import org.apache.hadoop.fs.s3a.impl.DirectoryPolicy;
|
import org.apache.hadoop.fs.s3a.impl.DirectoryPolicy;
|
||||||
import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum;
|
import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.StatisticTypeEnum;
|
||||||
|
|
||||||
import static org.apache.hadoop.fs.s3a.Constants.*;
|
import static org.apache.hadoop.fs.s3a.Constants.*;
|
||||||
import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
|
import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
|
||||||
import static org.apache.hadoop.fs.s3a.Statistic.*;
|
import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_BULK_DELETE_REQUEST;
|
||||||
|
import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_DELETE_REQUEST;
|
||||||
import static org.apache.hadoop.fs.s3a.performance.OperationCost.*;
|
import static org.apache.hadoop.fs.s3a.performance.OperationCost.*;
|
||||||
import static org.apache.hadoop.fs.s3a.performance.OperationCostValidator.expect;
|
import static org.apache.hadoop.fs.s3a.performance.OperationCostValidator.expect;
|
||||||
import static org.apache.hadoop.fs.s3a.performance.OperationCostValidator.probe;
|
import static org.apache.hadoop.fs.s3a.performance.OperationCostValidator.probe;
|
||||||
|
@ -83,6 +87,18 @@ public class AbstractS3ACostTest extends AbstractS3ATestBase {
|
||||||
|
|
||||||
private OperationCostValidator costValidator;
|
private OperationCostValidator costValidator;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Is bulk deletion enabled?
|
||||||
|
*/
|
||||||
|
private boolean isBulkDelete;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Which statistic measures marker deletion?
|
||||||
|
* this is the bulk delete statistic by default;
|
||||||
|
* if that is disabled it becomes the single delete counter.
|
||||||
|
*/
|
||||||
|
private Statistic deleteMarkerStatistic;
|
||||||
|
|
||||||
public AbstractS3ACostTest(
|
public AbstractS3ACostTest(
|
||||||
final boolean s3guard,
|
final boolean s3guard,
|
||||||
final boolean keepMarkers,
|
final boolean keepMarkers,
|
||||||
|
@ -149,23 +165,24 @@ public class AbstractS3ACostTest extends AbstractS3ATestBase {
|
||||||
.isEqualTo(isKeepingMarkers()
|
.isEqualTo(isKeepingMarkers()
|
||||||
? DirectoryPolicy.MarkerPolicy.Keep
|
? DirectoryPolicy.MarkerPolicy.Keep
|
||||||
: DirectoryPolicy.MarkerPolicy.Delete);
|
: DirectoryPolicy.MarkerPolicy.Delete);
|
||||||
|
// All counter statistics of the filesystem are added as metrics.
|
||||||
|
// Durations too, as they have counters of success and failure.
|
||||||
|
OperationCostValidator.Builder builder = OperationCostValidator.builder(
|
||||||
|
getFileSystem());
|
||||||
|
EnumSet.allOf(Statistic.class).stream()
|
||||||
|
.filter(s ->
|
||||||
|
s.getType() == StatisticTypeEnum.TYPE_COUNTER
|
||||||
|
|| s.getType() == StatisticTypeEnum.TYPE_DURATION)
|
||||||
|
.forEach(s -> builder.withMetric(s));
|
||||||
|
costValidator = builder.build();
|
||||||
|
|
||||||
// insert new metrics so as to keep the list sorted
|
// determine bulk delete settings
|
||||||
costValidator = OperationCostValidator.builder(getFileSystem())
|
final Configuration fsConf = getFileSystem().getConf();
|
||||||
.withMetrics(
|
isBulkDelete = fsConf.getBoolean(Constants.ENABLE_MULTI_DELETE,
|
||||||
DIRECTORIES_CREATED,
|
true);
|
||||||
DIRECTORIES_DELETED,
|
deleteMarkerStatistic = isBulkDelete()
|
||||||
FAKE_DIRECTORIES_DELETED,
|
? OBJECT_BULK_DELETE_REQUEST
|
||||||
FILES_DELETED,
|
: OBJECT_DELETE_REQUEST;
|
||||||
INVOCATION_COPY_FROM_LOCAL_FILE,
|
|
||||||
OBJECT_COPY_REQUESTS,
|
|
||||||
OBJECT_DELETE_REQUESTS,
|
|
||||||
OBJECT_DELETE_OBJECTS,
|
|
||||||
OBJECT_LIST_REQUESTS,
|
|
||||||
OBJECT_METADATA_REQUESTS,
|
|
||||||
OBJECT_PUT_BYTES,
|
|
||||||
OBJECT_PUT_REQUESTS)
|
|
||||||
.build();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public void assumeUnguarded() {
|
public void assumeUnguarded() {
|
||||||
|
@ -369,8 +386,7 @@ public class AbstractS3ACostTest extends AbstractS3ATestBase {
|
||||||
* Execute a closure expecting an exception.
|
* Execute a closure expecting an exception.
|
||||||
* @param clazz type of exception
|
* @param clazz type of exception
|
||||||
* @param text text to look for in exception (optional)
|
* @param text text to look for in exception (optional)
|
||||||
* @param head expected head request count.
|
* @param cost expected cost declaration.
|
||||||
* @param list expected list request count.
|
|
||||||
* @param eval closure to evaluate
|
* @param eval closure to evaluate
|
||||||
* @param <T> return type of closure
|
* @param <T> return type of closure
|
||||||
* @param <E> exception type
|
* @param <E> exception type
|
||||||
|
@ -647,4 +663,20 @@ public class AbstractS3ACostTest extends AbstractS3ATestBase {
|
||||||
getFileSystem(), status.getPath())))
|
getFileSystem(), status.getPath())))
|
||||||
.isEqualTo(expected);
|
.isEqualTo(expected);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Is bulk deletion enabled?
|
||||||
|
*/
|
||||||
|
protected boolean isBulkDelete() {
|
||||||
|
return isBulkDelete;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Which statistic measures marker deletion?
|
||||||
|
* this is the bulk delete statistic by default;
|
||||||
|
* if that is disabled it becomes the single delete counter.
|
||||||
|
*/
|
||||||
|
protected Statistic getDeleteMarkerStatistic() {
|
||||||
|
return deleteMarkerStatistic;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -47,6 +47,10 @@ import static org.apache.hadoop.test.LambdaTestUtils.intercept;
|
||||||
* Use metrics to assert about the cost of file API calls.
|
* Use metrics to assert about the cost of file API calls.
|
||||||
* <p></p>
|
* <p></p>
|
||||||
* Parameterized on guarded vs raw. and directory marker keep vs delete.
|
* Parameterized on guarded vs raw. and directory marker keep vs delete.
|
||||||
|
* There's extra complexity related to bulk/non-bulk delete calls.
|
||||||
|
* If bulk deletes are disabled, many more requests are made to delete
|
||||||
|
* parent directories. The counters of objects deleted are constant
|
||||||
|
* irrespective of the delete mode.
|
||||||
*/
|
*/
|
||||||
@RunWith(Parameterized.class)
|
@RunWith(Parameterized.class)
|
||||||
public class ITestS3ADeleteCost extends AbstractS3ACostTest {
|
public class ITestS3ADeleteCost extends AbstractS3ACostTest {
|
||||||
|
@ -112,18 +116,19 @@ public class ITestS3ADeleteCost extends AbstractS3ACostTest {
|
||||||
// if deleting markers, look for the parent too
|
// if deleting markers, look for the parent too
|
||||||
probe(rawAndDeleting, OBJECT_METADATA_REQUESTS,
|
probe(rawAndDeleting, OBJECT_METADATA_REQUESTS,
|
||||||
FILESTATUS_FILE_PROBE_H + FILESTATUS_DIR_PROBE_H),
|
FILESTATUS_FILE_PROBE_H + FILESTATUS_DIR_PROBE_H),
|
||||||
withWhenRaw(OBJECT_LIST_REQUESTS,
|
withWhenRaw(OBJECT_LIST_REQUEST,
|
||||||
FILESTATUS_FILE_PROBE_L + FILESTATUS_DIR_PROBE_L),
|
FILESTATUS_FILE_PROBE_L + FILESTATUS_DIR_PROBE_L),
|
||||||
with(DIRECTORIES_DELETED, 0),
|
with(DIRECTORIES_DELETED, 0),
|
||||||
with(FILES_DELETED, 1),
|
with(FILES_DELETED, 1),
|
||||||
|
|
||||||
// keeping: create no parent dirs or delete parents
|
// keeping: create no parent dirs or delete parents
|
||||||
withWhenKeeping(DIRECTORIES_CREATED, 0),
|
withWhenKeeping(DIRECTORIES_CREATED, 0),
|
||||||
withWhenKeeping(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST),
|
withWhenKeeping(OBJECT_DELETE_OBJECTS, DELETE_OBJECT_REQUEST),
|
||||||
|
|
||||||
// deleting: create a parent and delete any of its parents
|
// deleting: create a parent and delete any of its parents
|
||||||
withWhenDeleting(DIRECTORIES_CREATED, 1),
|
withWhenDeleting(DIRECTORIES_CREATED, 1),
|
||||||
withWhenDeleting(OBJECT_DELETE_REQUESTS,
|
// two objects will be deleted
|
||||||
|
withWhenDeleting(OBJECT_DELETE_OBJECTS,
|
||||||
DELETE_OBJECT_REQUEST
|
DELETE_OBJECT_REQUEST
|
||||||
+ DELETE_MARKER_REQUEST)
|
+ DELETE_MARKER_REQUEST)
|
||||||
);
|
);
|
||||||
|
@ -160,7 +165,7 @@ public class ITestS3ADeleteCost extends AbstractS3ACostTest {
|
||||||
// if deleting markers, look for the parent too
|
// if deleting markers, look for the parent too
|
||||||
probe(rawAndDeleting, OBJECT_METADATA_REQUESTS,
|
probe(rawAndDeleting, OBJECT_METADATA_REQUESTS,
|
||||||
FILESTATUS_FILE_PROBE_H + FILESTATUS_DIR_PROBE_H),
|
FILESTATUS_FILE_PROBE_H + FILESTATUS_DIR_PROBE_H),
|
||||||
withWhenRaw(OBJECT_LIST_REQUESTS,
|
withWhenRaw(OBJECT_LIST_REQUEST,
|
||||||
FILESTATUS_FILE_PROBE_L + FILESTATUS_DIR_PROBE_L),
|
FILESTATUS_FILE_PROBE_L + FILESTATUS_DIR_PROBE_L),
|
||||||
with(DIRECTORIES_DELETED, 0),
|
with(DIRECTORIES_DELETED, 0),
|
||||||
with(FILES_DELETED, 1),
|
with(FILES_DELETED, 1),
|
||||||
|
@ -169,10 +174,10 @@ public class ITestS3ADeleteCost extends AbstractS3ACostTest {
|
||||||
with(DIRECTORIES_CREATED, 0),
|
with(DIRECTORIES_CREATED, 0),
|
||||||
|
|
||||||
// keeping: create no parent dirs or delete parents
|
// keeping: create no parent dirs or delete parents
|
||||||
withWhenKeeping(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST),
|
withWhenKeeping(OBJECT_DELETE_REQUEST, DELETE_OBJECT_REQUEST),
|
||||||
|
|
||||||
// deleting: create a parent and delete any of its parents
|
// deleting: create a parent and delete any of its parents
|
||||||
withWhenDeleting(OBJECT_DELETE_REQUESTS,
|
withWhenDeleting(OBJECT_DELETE_REQUEST,
|
||||||
DELETE_OBJECT_REQUEST));
|
DELETE_OBJECT_REQUEST));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -199,22 +204,20 @@ public class ITestS3ADeleteCost extends AbstractS3ACostTest {
|
||||||
|
|
||||||
LOG.info("creating sub directory {}", subDir);
|
LOG.info("creating sub directory {}", subDir);
|
||||||
// one dir created, possibly a parent removed
|
// one dir created, possibly a parent removed
|
||||||
|
final int fakeDirectoriesToDelete = directoriesInPath(subDir) - 1;
|
||||||
verifyMetrics(() -> {
|
verifyMetrics(() -> {
|
||||||
mkdirs(subDir);
|
mkdirs(subDir);
|
||||||
return "after mkdir(subDir) " + getMetricSummary();
|
return "after mkdir(subDir) " + getMetricSummary();
|
||||||
},
|
},
|
||||||
with(DIRECTORIES_CREATED, 1),
|
with(DIRECTORIES_CREATED, 1),
|
||||||
with(DIRECTORIES_DELETED, 0),
|
with(DIRECTORIES_DELETED, 0),
|
||||||
withWhenKeeping(OBJECT_DELETE_REQUESTS, 0),
|
withWhenKeeping(getDeleteMarkerStatistic(), 0),
|
||||||
withWhenKeeping(FAKE_DIRECTORIES_DELETED, 0),
|
withWhenKeeping(FAKE_DIRECTORIES_DELETED, 0),
|
||||||
withWhenDeleting(OBJECT_DELETE_REQUESTS, DELETE_MARKER_REQUEST),
|
withWhenDeleting(getDeleteMarkerStatistic(),
|
||||||
|
isBulkDelete() ? DELETE_MARKER_REQUEST : fakeDirectoriesToDelete),
|
||||||
// delete all possible fake dirs above the subdirectory
|
// delete all possible fake dirs above the subdirectory
|
||||||
withWhenDeleting(FAKE_DIRECTORIES_DELETED,
|
withWhenDeleting(FAKE_DIRECTORIES_DELETED,
|
||||||
directoriesInPath(subDir) - 1));
|
fakeDirectoriesToDelete));
|
||||||
|
|
||||||
int dirDeleteRequests = 1;
|
|
||||||
int fileDeleteRequests = 0;
|
|
||||||
int totalDeleteRequests = dirDeleteRequests + fileDeleteRequests;
|
|
||||||
|
|
||||||
LOG.info("About to delete {}", parent);
|
LOG.info("About to delete {}", parent);
|
||||||
// now delete the deep tree.
|
// now delete the deep tree.
|
||||||
|
@ -223,8 +226,6 @@ public class ITestS3ADeleteCost extends AbstractS3ACostTest {
|
||||||
return "deleting parent dir " + parent + " " + getMetricSummary();
|
return "deleting parent dir " + parent + " " + getMetricSummary();
|
||||||
},
|
},
|
||||||
|
|
||||||
// two directory markers will be deleted in a single request
|
|
||||||
with(OBJECT_DELETE_REQUESTS, totalDeleteRequests),
|
|
||||||
// keeping: the parent dir marker needs deletion alongside
|
// keeping: the parent dir marker needs deletion alongside
|
||||||
// the subdir one.
|
// the subdir one.
|
||||||
withWhenKeeping(OBJECT_DELETE_OBJECTS, dirsCreated),
|
withWhenKeeping(OBJECT_DELETE_OBJECTS, dirsCreated),
|
||||||
|
@ -265,6 +266,7 @@ public class ITestS3ADeleteCost extends AbstractS3ACostTest {
|
||||||
// creating a file should trigger demise of the src dir marker
|
// creating a file should trigger demise of the src dir marker
|
||||||
// unless markers are being kept
|
// unless markers are being kept
|
||||||
|
|
||||||
|
final int directories = directoriesInPath(srcDir);
|
||||||
verifyMetrics(() -> {
|
verifyMetrics(() -> {
|
||||||
file(new Path(srcDir, "source.txt"));
|
file(new Path(srcDir, "source.txt"));
|
||||||
LOG.info("Metrics: {}\n{}", getMetricSummary(), getFileSystem());
|
LOG.info("Metrics: {}\n{}", getMetricSummary(), getFileSystem());
|
||||||
|
@ -273,12 +275,13 @@ public class ITestS3ADeleteCost extends AbstractS3ACostTest {
|
||||||
with(DIRECTORIES_CREATED, 0),
|
with(DIRECTORIES_CREATED, 0),
|
||||||
with(DIRECTORIES_DELETED, 0),
|
with(DIRECTORIES_DELETED, 0),
|
||||||
// keeping: no delete operations.
|
// keeping: no delete operations.
|
||||||
withWhenKeeping(OBJECT_DELETE_REQUESTS, 0),
|
withWhenKeeping(getDeleteMarkerStatistic(), 0),
|
||||||
withWhenKeeping(FAKE_DIRECTORIES_DELETED, 0),
|
withWhenKeeping(FAKE_DIRECTORIES_DELETED, 0),
|
||||||
// delete all possible fake dirs above the file
|
// delete all possible fake dirs above the file
|
||||||
withWhenDeleting(OBJECT_DELETE_REQUESTS, 1),
|
withWhenDeleting(getDeleteMarkerStatistic(),
|
||||||
|
isBulkDelete() ? 1: directories),
|
||||||
withWhenDeleting(FAKE_DIRECTORIES_DELETED,
|
withWhenDeleting(FAKE_DIRECTORIES_DELETED,
|
||||||
directoriesInPath(srcDir)));
|
directories));
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -35,6 +35,7 @@ import org.apache.hadoop.fs.s3a.S3AFileSystem;
|
||||||
|
|
||||||
import static org.apache.hadoop.fs.s3a.Statistic.*;
|
import static org.apache.hadoop.fs.s3a.Statistic.*;
|
||||||
import static org.apache.hadoop.fs.s3a.performance.OperationCost.*;
|
import static org.apache.hadoop.fs.s3a.performance.OperationCost.*;
|
||||||
|
import static org.apache.hadoop.fs.s3a.performance.OperationCostValidator.probe;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Use metrics to assert about the cost of file API calls.
|
* Use metrics to assert about the cost of file API calls.
|
||||||
|
@ -97,19 +98,34 @@ public class ITestS3ARenameCost extends AbstractS3ACostTest {
|
||||||
// rename the source file to the destination file.
|
// rename the source file to the destination file.
|
||||||
// this tests file rename, not dir rename
|
// this tests file rename, not dir rename
|
||||||
// as srcFile2 exists, the parent dir of srcFilePath must not be created.
|
// as srcFile2 exists, the parent dir of srcFilePath must not be created.
|
||||||
|
final int directoriesInPath = directoriesInPath(destDir);
|
||||||
verifyMetrics(() ->
|
verifyMetrics(() ->
|
||||||
execRename(srcFilePath, destFilePath),
|
execRename(srcFilePath, destFilePath),
|
||||||
whenRaw(RENAME_SINGLE_FILE_DIFFERENT_DIR),
|
whenRaw(RENAME_SINGLE_FILE_DIFFERENT_DIR),
|
||||||
with(DIRECTORIES_CREATED, 0),
|
with(DIRECTORIES_CREATED, 0),
|
||||||
with(DIRECTORIES_DELETED, 0),
|
with(DIRECTORIES_DELETED, 0),
|
||||||
// keeping: only the core delete operation is issued.
|
// keeping: only the core delete operation is issued.
|
||||||
withWhenKeeping(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST),
|
withWhenKeeping(OBJECT_DELETE_REQUEST, DELETE_OBJECT_REQUEST),
|
||||||
withWhenKeeping(FAKE_DIRECTORIES_DELETED, 0),
|
withWhenKeeping(FAKE_DIRECTORIES_DELETED, 0),
|
||||||
|
withWhenKeeping(OBJECT_DELETE_OBJECTS, 1),
|
||||||
|
|
||||||
// deleting: delete any fake marker above the destination.
|
// deleting: delete any fake marker above the destination.
|
||||||
withWhenDeleting(OBJECT_DELETE_REQUESTS,
|
// the actual request count depends on whether bulk delete is
|
||||||
DELETE_OBJECT_REQUEST + DELETE_MARKER_REQUEST),
|
// enabled or not
|
||||||
|
|
||||||
|
// no bulk delete: multiple marker calls
|
||||||
|
probe(isDeleting() && !isBulkDelete(), OBJECT_DELETE_REQUEST,
|
||||||
|
DELETE_OBJECT_REQUEST + directoriesInPath),
|
||||||
|
|
||||||
|
// bulk delete: split up
|
||||||
|
probe(isDeleting() && isBulkDelete(), OBJECT_DELETE_REQUEST,
|
||||||
|
DELETE_OBJECT_REQUEST),
|
||||||
|
probe(isDeleting() && isBulkDelete(), OBJECT_BULK_DELETE_REQUEST,
|
||||||
|
DELETE_MARKER_REQUEST),
|
||||||
withWhenDeleting(FAKE_DIRECTORIES_DELETED,
|
withWhenDeleting(FAKE_DIRECTORIES_DELETED,
|
||||||
directoriesInPath(destDir)));
|
directoriesInPath),
|
||||||
|
withWhenDeleting(OBJECT_DELETE_OBJECTS,
|
||||||
|
directoriesInPath + 1));
|
||||||
|
|
||||||
assertIsFile(destFilePath);
|
assertIsFile(destFilePath);
|
||||||
assertIsDirectory(srcDir);
|
assertIsDirectory(srcDir);
|
||||||
|
@ -139,7 +155,7 @@ public class ITestS3ARenameCost extends AbstractS3ACostTest {
|
||||||
whenRaw(RENAME_SINGLE_FILE_SAME_DIR),
|
whenRaw(RENAME_SINGLE_FILE_SAME_DIR),
|
||||||
with(OBJECT_COPY_REQUESTS, 1),
|
with(OBJECT_COPY_REQUESTS, 1),
|
||||||
with(DIRECTORIES_CREATED, 0),
|
with(DIRECTORIES_CREATED, 0),
|
||||||
with(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST),
|
with(OBJECT_DELETE_REQUEST, DELETE_OBJECT_REQUEST),
|
||||||
with(FAKE_DIRECTORIES_DELETED, 0));
|
with(FAKE_DIRECTORIES_DELETED, 0));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -164,7 +180,7 @@ public class ITestS3ARenameCost extends AbstractS3ACostTest {
|
||||||
// here we expect there to be no fake directories
|
// here we expect there to be no fake directories
|
||||||
with(DIRECTORIES_CREATED, 0),
|
with(DIRECTORIES_CREATED, 0),
|
||||||
// one for the renamed file only
|
// one for the renamed file only
|
||||||
with(OBJECT_DELETE_REQUESTS,
|
with(OBJECT_DELETE_REQUEST,
|
||||||
DELETE_OBJECT_REQUEST),
|
DELETE_OBJECT_REQUEST),
|
||||||
// no directories are deleted: This is root
|
// no directories are deleted: This is root
|
||||||
with(DIRECTORIES_DELETED, 0),
|
with(DIRECTORIES_DELETED, 0),
|
||||||
|
@ -196,7 +212,7 @@ public class ITestS3ARenameCost extends AbstractS3ACostTest {
|
||||||
with(DIRECTORIES_DELETED, 0),
|
with(DIRECTORIES_DELETED, 0),
|
||||||
with(FAKE_DIRECTORIES_DELETED, 0),
|
with(FAKE_DIRECTORIES_DELETED, 0),
|
||||||
with(FILES_DELETED, 1),
|
with(FILES_DELETED, 1),
|
||||||
with(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST),
|
with(OBJECT_DELETE_REQUEST, DELETE_OBJECT_REQUEST),
|
||||||
whenRaw(FILE_STATUS_FILE_PROBE)); /* no need to look at parent. */
|
whenRaw(FILE_STATUS_FILE_PROBE)); /* no need to look at parent. */
|
||||||
|
|
||||||
} finally {
|
} finally {
|
||||||
|
|
|
@ -51,6 +51,8 @@ public final class OperationCost {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Delete cost when deleting a marker.
|
* Delete cost when deleting a marker.
|
||||||
|
* Note: if bulk delete is disabled, this changes to being
|
||||||
|
* the number of directories deleted.
|
||||||
*/
|
*/
|
||||||
public static final int DELETE_MARKER_REQUEST = DELETE_OBJECT_REQUEST;
|
public static final int DELETE_MARKER_REQUEST = DELETE_OBJECT_REQUEST;
|
||||||
|
|
||||||
|
|
|
@ -31,11 +31,15 @@ import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import org.apache.hadoop.fs.s3a.S3AFileSystem;
|
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.S3ATestUtils;
|
||||||
import org.apache.hadoop.fs.s3a.Statistic;
|
import org.apache.hadoop.fs.s3a.Statistic;
|
||||||
|
import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
|
||||||
|
import org.apache.hadoop.metrics2.lib.MutableCounter;
|
||||||
|
import org.apache.hadoop.metrics2.lib.MutableMetric;
|
||||||
|
|
||||||
import static java.util.Objects.requireNonNull;
|
import static java.util.Objects.requireNonNull;
|
||||||
import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_LIST_REQUESTS;
|
import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_LIST_REQUEST;
|
||||||
import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_METADATA_REQUESTS;
|
import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_METADATA_REQUESTS;
|
||||||
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
|
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
|
||||||
|
|
||||||
|
@ -91,15 +95,30 @@ public final class OperationCostValidator {
|
||||||
private final Map<String, S3ATestUtils.MetricDiff> metricDiffs
|
private final Map<String, S3ATestUtils.MetricDiff> metricDiffs
|
||||||
= new TreeMap<>();
|
= new TreeMap<>();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Instrumentation's IO Statistics.
|
||||||
|
*/
|
||||||
|
private final IOStatisticsStore ioStatistics;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Build the instance.
|
* Build the instance.
|
||||||
* @param builder builder containing all options.
|
* @param builder builder containing all options.
|
||||||
*/
|
*/
|
||||||
private OperationCostValidator(Builder builder) {
|
private OperationCostValidator(Builder builder) {
|
||||||
builder.metrics.forEach(stat ->
|
S3AFileSystem fs = builder.filesystem;
|
||||||
metricDiffs.put(stat.getSymbol(),
|
S3AInstrumentation instrumentation = fs.getInstrumentation();
|
||||||
new S3ATestUtils.MetricDiff(builder.filesystem, stat)));
|
for (Statistic stat : builder.metrics) {
|
||||||
|
String symbol = stat.getSymbol();
|
||||||
|
MutableMetric metric = instrumentation.lookupMetric(symbol);
|
||||||
|
if (metric instanceof MutableCounter) {
|
||||||
|
// only counters are used in the cost tracking;
|
||||||
|
// other statistics are ignored.
|
||||||
|
metricDiffs.put(symbol,
|
||||||
|
new S3ATestUtils.MetricDiff(fs, stat));
|
||||||
|
}
|
||||||
|
}
|
||||||
builder.metrics.clear();
|
builder.metrics.clear();
|
||||||
|
ioStatistics = instrumentation.getIOStatistics();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -137,6 +156,7 @@ public final class OperationCostValidator {
|
||||||
ExpectedProbe... expectedA) throws Exception {
|
ExpectedProbe... expectedA) throws Exception {
|
||||||
List<ExpectedProbe> expected = Arrays.asList(expectedA);
|
List<ExpectedProbe> expected = Arrays.asList(expectedA);
|
||||||
resetMetricDiffs();
|
resetMetricDiffs();
|
||||||
|
|
||||||
// verify that 1+ probe is enabled
|
// verify that 1+ probe is enabled
|
||||||
assumeProbesEnabled(expected);
|
assumeProbesEnabled(expected);
|
||||||
// if we get here, then yes.
|
// if we get here, then yes.
|
||||||
|
@ -147,8 +167,9 @@ public final class OperationCostValidator {
|
||||||
"operation returning "
|
"operation returning "
|
||||||
+ (r != null ? r.toString() : "null");
|
+ (r != null ? r.toString() : "null");
|
||||||
LOG.info("{}", text);
|
LOG.info("{}", text);
|
||||||
LOG.info("state {}", this);
|
LOG.info("state {}", this.toString());
|
||||||
LOG.info("probes {}", expected);
|
LOG.info("probes {}", expected);
|
||||||
|
LOG.info("IOStatistics {}", ioStatistics);
|
||||||
for (ExpectedProbe ed : expected) {
|
for (ExpectedProbe ed : expected) {
|
||||||
ed.verify(this, text);
|
ed.verify(this, text);
|
||||||
}
|
}
|
||||||
|
@ -246,7 +267,8 @@ public final class OperationCostValidator {
|
||||||
* @return this
|
* @return this
|
||||||
*/
|
*/
|
||||||
public Builder withMetric(Statistic statistic) {
|
public Builder withMetric(Statistic statistic) {
|
||||||
return withMetric(statistic);
|
metrics.add(statistic);
|
||||||
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -330,7 +352,7 @@ public final class OperationCostValidator {
|
||||||
boolean enabled, OperationCost cost) {
|
boolean enabled, OperationCost cost) {
|
||||||
return probes(enabled,
|
return probes(enabled,
|
||||||
probe(OBJECT_METADATA_REQUESTS, cost.head()),
|
probe(OBJECT_METADATA_REQUESTS, cost.head()),
|
||||||
probe(OBJECT_LIST_REQUESTS, cost.list()));
|
probe(OBJECT_LIST_REQUEST, cost.list()));
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -54,7 +54,7 @@ import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL;
|
||||||
import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume;
|
import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume;
|
||||||
import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
|
import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
|
||||||
import static org.apache.hadoop.fs.s3a.S3AUtils.applyLocatedFiles;
|
import static org.apache.hadoop.fs.s3a.S3AUtils.applyLocatedFiles;
|
||||||
import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_LIST_REQUESTS;
|
import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_LIST_REQUEST;
|
||||||
import static org.apache.hadoop.fs.s3a.Statistic.S3GUARD_METADATASTORE_AUTHORITATIVE_DIRECTORIES_UPDATED;
|
import static org.apache.hadoop.fs.s3a.Statistic.S3GUARD_METADATASTORE_AUTHORITATIVE_DIRECTORIES_UPDATED;
|
||||||
import static org.apache.hadoop.fs.s3a.Statistic.S3GUARD_METADATASTORE_RECORD_WRITES;
|
import static org.apache.hadoop.fs.s3a.Statistic.S3GUARD_METADATASTORE_RECORD_WRITES;
|
||||||
import static org.apache.hadoop.fs.s3a.s3guard.AuthoritativeAuditOperation.ERROR_PATH_NOT_AUTH_IN_FS;
|
import static org.apache.hadoop.fs.s3a.s3guard.AuthoritativeAuditOperation.ERROR_PATH_NOT_AUTH_IN_FS;
|
||||||
|
@ -330,7 +330,7 @@ public class ITestDynamoDBMetadataStoreAuthoritativeMode
|
||||||
touchFile(nestedFile2);
|
touchFile(nestedFile2);
|
||||||
|
|
||||||
S3ATestUtils.MetricDiff objListRequests =
|
S3ATestUtils.MetricDiff objListRequests =
|
||||||
new S3ATestUtils.MetricDiff(authFS, OBJECT_LIST_REQUESTS);
|
new S3ATestUtils.MetricDiff(authFS, OBJECT_LIST_REQUEST);
|
||||||
|
|
||||||
RemoteIterator<LocatedFileStatus> statusIterator =
|
RemoteIterator<LocatedFileStatus> statusIterator =
|
||||||
authFS.listFiles(dir, true);
|
authFS.listFiles(dir, true);
|
||||||
|
@ -372,7 +372,7 @@ public class ITestDynamoDBMetadataStoreAuthoritativeMode
|
||||||
touchFile(nestedFile2);
|
touchFile(nestedFile2);
|
||||||
|
|
||||||
S3ATestUtils.MetricDiff objListRequests =
|
S3ATestUtils.MetricDiff objListRequests =
|
||||||
new S3ATestUtils.MetricDiff(authFS, OBJECT_LIST_REQUESTS);
|
new S3ATestUtils.MetricDiff(authFS, OBJECT_LIST_REQUEST);
|
||||||
|
|
||||||
RemoteIterator<LocatedFileStatus> statusIterator =
|
RemoteIterator<LocatedFileStatus> statusIterator =
|
||||||
authFS.listFiles(dir, true);
|
authFS.listFiles(dir, true);
|
||||||
|
@ -766,7 +766,7 @@ public class ITestDynamoDBMetadataStoreAuthoritativeMode
|
||||||
S3ATestUtils.MetricDiff authDirsMarked = new S3ATestUtils.MetricDiff(authFS,
|
S3ATestUtils.MetricDiff authDirsMarked = new S3ATestUtils.MetricDiff(authFS,
|
||||||
S3GUARD_METADATASTORE_AUTHORITATIVE_DIRECTORIES_UPDATED);
|
S3GUARD_METADATASTORE_AUTHORITATIVE_DIRECTORIES_UPDATED);
|
||||||
S3ATestUtils.MetricDiff listRequests = new S3ATestUtils.MetricDiff(authFS,
|
S3ATestUtils.MetricDiff listRequests = new S3ATestUtils.MetricDiff(authFS,
|
||||||
OBJECT_LIST_REQUESTS);
|
OBJECT_LIST_REQUEST);
|
||||||
final T call = fn.call();
|
final T call = fn.call();
|
||||||
authDirsMarked.assertDiffEquals(updates);
|
authDirsMarked.assertDiffEquals(updates);
|
||||||
listRequests.assertDiffEquals(lists);
|
listRequests.assertDiffEquals(lists);
|
||||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.hadoop.fs.s3a.s3guard;
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.io.FileNotFoundException;
|
import java.io.FileNotFoundException;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.io.UncheckedIOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
@ -49,7 +50,6 @@ import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.fs.StorageStatistics;
|
import org.apache.hadoop.fs.StorageStatistics;
|
||||||
import org.apache.hadoop.fs.contract.ContractTestUtils;
|
import org.apache.hadoop.fs.contract.ContractTestUtils;
|
||||||
import org.apache.hadoop.fs.impl.WrappedIOException;
|
|
||||||
import org.apache.hadoop.fs.s3a.AWSServiceThrottledException;
|
import org.apache.hadoop.fs.s3a.AWSServiceThrottledException;
|
||||||
import org.apache.hadoop.fs.s3a.Invoker;
|
import org.apache.hadoop.fs.s3a.Invoker;
|
||||||
import org.apache.hadoop.fs.s3a.S3AFileStatus;
|
import org.apache.hadoop.fs.s3a.S3AFileStatus;
|
||||||
|
@ -532,7 +532,7 @@ public class ITestDynamoDBMetadataStoreScale
|
||||||
LOG.info("Deleting {}", p);
|
LOG.info("Deleting {}", p);
|
||||||
list.add(p);
|
list.add(p);
|
||||||
});
|
});
|
||||||
} catch (WrappedIOException e) {
|
} catch (UncheckedIOException e) {
|
||||||
// the iterator may have overloaded; swallow if so.
|
// the iterator may have overloaded; swallow if so.
|
||||||
if (!(e.getCause() instanceof AWSServiceThrottledException)) {
|
if (!(e.getCause() instanceof AWSServiceThrottledException)) {
|
||||||
throw e;
|
throw e;
|
||||||
|
|
|
@ -25,8 +25,7 @@ import java.util.concurrent.atomic.AtomicLong;
|
||||||
import com.amazonaws.event.ProgressEvent;
|
import com.amazonaws.event.ProgressEvent;
|
||||||
import com.amazonaws.event.ProgressEventType;
|
import com.amazonaws.event.ProgressEventType;
|
||||||
import com.amazonaws.event.ProgressListener;
|
import com.amazonaws.event.ProgressListener;
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
import org.assertj.core.api.Assertions;
|
||||||
import org.apache.hadoop.fs.s3a.S3ATestUtils;
|
|
||||||
import org.junit.FixMethodOrder;
|
import org.junit.FixMethodOrder;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.junit.runners.MethodSorters;
|
import org.junit.runners.MethodSorters;
|
||||||
|
@ -36,17 +35,24 @@ import org.slf4j.LoggerFactory;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.FSDataInputStream;
|
import org.apache.hadoop.fs.FSDataInputStream;
|
||||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||||
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.fs.StorageStatistics;
|
import org.apache.hadoop.fs.StorageStatistics;
|
||||||
import org.apache.hadoop.fs.contract.ContractTestUtils;
|
import org.apache.hadoop.fs.contract.ContractTestUtils;
|
||||||
import org.apache.hadoop.fs.s3a.S3AFileSystem;
|
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.Statistic;
|
||||||
|
import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics;
|
||||||
|
import org.apache.hadoop.fs.statistics.IOStatistics;
|
||||||
import org.apache.hadoop.util.Progressable;
|
import org.apache.hadoop.util.Progressable;
|
||||||
|
|
||||||
import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
|
import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
|
||||||
import static org.apache.hadoop.fs.s3a.Constants.*;
|
import static org.apache.hadoop.fs.s3a.Constants.*;
|
||||||
import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
|
import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
|
||||||
|
import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.demandStringifyIOStatistics;
|
||||||
|
import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsSourceToString;
|
||||||
|
import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics;
|
||||||
|
import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.snapshotIOStatistics;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Scale test which creates a huge file.
|
* Scale test which creates a huge file.
|
||||||
|
@ -170,7 +176,7 @@ public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
|
||||||
Statistic putBytesPending = Statistic.OBJECT_PUT_BYTES_PENDING;
|
Statistic putBytesPending = Statistic.OBJECT_PUT_BYTES_PENDING;
|
||||||
|
|
||||||
ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
|
ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
|
||||||
S3AInstrumentation.OutputStreamStatistics streamStatistics;
|
BlockOutputStreamStatistics streamStatistics;
|
||||||
long blocksPer10MB = blocksPerMB * 10;
|
long blocksPer10MB = blocksPerMB * 10;
|
||||||
ProgressCallback progress = new ProgressCallback(timer);
|
ProgressCallback progress = new ProgressCallback(timer);
|
||||||
try (FSDataOutputStream out = fs.create(fileToCreate,
|
try (FSDataOutputStream out = fs.create(fileToCreate,
|
||||||
|
@ -221,8 +227,20 @@ public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
|
||||||
logFSState();
|
logFSState();
|
||||||
bandwidth(timer, filesize);
|
bandwidth(timer, filesize);
|
||||||
LOG.info("Statistics after stream closed: {}", streamStatistics);
|
LOG.info("Statistics after stream closed: {}", streamStatistics);
|
||||||
|
IOStatistics iostats = snapshotIOStatistics(
|
||||||
|
retrieveIOStatistics(getFileSystem()));
|
||||||
|
LOG.info("IOStatistics after upload: {}",
|
||||||
|
demandStringifyIOStatistics(iostats));
|
||||||
long putRequestCount = storageStatistics.getLong(putRequests);
|
long putRequestCount = storageStatistics.getLong(putRequests);
|
||||||
Long putByteCount = storageStatistics.getLong(putBytes);
|
Long putByteCount = storageStatistics.getLong(putBytes);
|
||||||
|
Assertions.assertThat(putRequestCount)
|
||||||
|
.describedAs("Put request count from filesystem stats %s",
|
||||||
|
iostats)
|
||||||
|
.isGreaterThan(0);
|
||||||
|
Assertions.assertThat(putByteCount)
|
||||||
|
.describedAs("putByteCount count from filesystem stats %s",
|
||||||
|
iostats)
|
||||||
|
.isGreaterThan(0);
|
||||||
LOG.info("PUT {} bytes in {} operations; {} MB/operation",
|
LOG.info("PUT {} bytes in {} operations; {} MB/operation",
|
||||||
putByteCount, putRequestCount,
|
putByteCount, putRequestCount,
|
||||||
putByteCount / (putRequestCount * _1MB));
|
putByteCount / (putRequestCount * _1MB));
|
||||||
|
@ -234,7 +252,7 @@ public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
|
||||||
"Put file " + fileToCreate + " of size " + filesize);
|
"Put file " + fileToCreate + " of size " + filesize);
|
||||||
if (streamStatistics != null) {
|
if (streamStatistics != null) {
|
||||||
assertEquals("actively allocated blocks in " + streamStatistics,
|
assertEquals("actively allocated blocks in " + streamStatistics,
|
||||||
0, streamStatistics.blocksActivelyAllocated());
|
0, streamStatistics.getBlocksActivelyAllocated());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -541,12 +559,7 @@ public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
public void test_900_dumpStats() {
|
public void test_900_dumpStats() {
|
||||||
StringBuilder sb = new StringBuilder();
|
LOG.info("Statistics\n{}", ioStatisticsSourceToString(getFileSystem()));
|
||||||
|
|
||||||
getFileSystem().getStorageStatistics()
|
|
||||||
.forEach(kv -> sb.append(kv.toString()).append("\n"));
|
|
||||||
|
|
||||||
LOG.info("Statistics\n{}", sb);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
protected void deleteHugeFile() throws IOException {
|
protected void deleteHugeFile() throws IOException {
|
||||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue