From 4e7707df4e0c108d3f2798d30489cc2aaffe8d4d Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 8 Feb 2021 18:06:31 +0000 Subject: [PATCH 1/8] HADOOP-13551. Wire up AWS SDK metrics to IOStatistics Moves to the builder API for AWS S3 client creation, and offers a similar-style API to the S3AFS and tests, hiding the details of what options are client, what are AWS Conf, and doing the wiring up of S3A stats interfaces to the AWS internals. This will break HBase's HBoss test suite, but I couldn't see a way to cleanly address this. I'll need to work on that code separately Change-Id: I4fb05f4a54eece8fac96abf8d6af9c889392a6c1 --- .../org/apache/hadoop/fs/s3a/Constants.java | 27 +- .../hadoop/fs/s3a/DefaultS3ClientFactory.java | 161 ++++-------- .../fs/s3a/InconsistentS3ClientFactory.java | 30 +-- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 26 +- .../apache/hadoop/fs/s3a/S3ClientFactory.java | 232 +++++++++++++++++- .../hadoop/fs/s3a/impl/InternalConstants.java | 6 - .../hadoop/fs/s3a/MockS3ClientFactory.java | 11 +- .../hadoop/fs/s3a/auth/ITestCustomSigner.java | 75 ++++-- .../ITestSessionDelegationInFileystem.java | 24 +- .../fs/s3a/commit/ITestCommitOperations.java | 1 + .../ITestAWSStatisticCollection.java | 82 +++++++ 11 files changed, 462 insertions(+), 213 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestAWSStatisticCollection.java diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java index d14a82e5c304c..c4b8f6e3c4683 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java @@ -160,14 +160,33 @@ private Constants() { DEFAULT_SSL_CHANNEL_MODE = DelegatingSSLSocketFactory.SSLChannelMode.Default_JSSE; - //use a custom endpoint? + /** + * Endpoint. For v4 signing and/or better performance, + * this should be the specific endpoint of the region + * in which the bucket is hosted. + */ public static final String ENDPOINT = "fs.s3a.endpoint"; /** - * Default value of s3 endpoint. If not set explicitly using - * {@code AmazonS3#setEndpoint()}, this is used. + * Default value of s3 endpoint: {@value}. + * It tells the AWS client to work it out by asking the central + * endpoint where the bucket lives; caching that + * value in the client for the life of the process. + *

+ * Note: previously this constant was defined as + * {@link #CENTRAL_ENDPOINT}, however the actual + * S3A client code used "" as the default when + * {@link #ENDPOINT} was unset. + * As core-default.xml also set the endpoint to "", + * the empty string has long been the real + * default value. + */ + public static final String DEFAULT_ENDPOINT = ""; + + /** + * The central endpoint :{@value}. */ - public static final String DEFAULT_ENDPOINT = "s3.amazonaws.com"; + public static final String CENTRAL_ENDPOINT = "s3.amazonaws.com"; //Enable path style access? Overrides default virtual hosting public static final String PATH_STYLE_ACCESS = "fs.s3a.path.style.access"; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java index 96d16e8b1b8e0..6efde1d9158fe 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java @@ -24,7 +24,7 @@ import com.amazonaws.ClientConfiguration; import com.amazonaws.auth.AWSCredentialsProvider; import com.amazonaws.client.builder.AwsClientBuilder; -import com.amazonaws.metrics.RequestMetricCollector; +import com.amazonaws.handlers.RequestHandler2; import com.amazonaws.services.s3.AmazonS3; import com.amazonaws.services.s3.AmazonS3Client; import com.amazonaws.services.s3.AmazonS3ClientBuilder; @@ -41,13 +41,10 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; -import org.apache.hadoop.fs.s3a.statistics.StatisticsFromAwsSdk; import org.apache.hadoop.fs.s3a.statistics.impl.AwsStatisticsCollector; import static org.apache.hadoop.fs.s3a.Constants.EXPERIMENTAL_AWS_INTERNAL_THROTTLING; -import static org.apache.hadoop.fs.s3a.Constants.ENDPOINT; import static org.apache.hadoop.fs.s3a.Constants.EXPERIMENTAL_AWS_INTERNAL_THROTTLING_DEFAULT; -import static org.apache.hadoop.fs.s3a.Constants.PATH_STYLE_ACCESS; /** * The default {@link S3ClientFactory} implementation. @@ -70,22 +67,24 @@ public class DefaultS3ClientFactory extends Configured LoggerFactory.getLogger(DefaultS3ClientFactory.class); /** - * Create the client. + * Create the client by preparing the AwsConf configuration + * and then invoking {@link #buildAmazonS3Client(AWSCredentialsProvider, ClientConfiguration, S3ClientCreationParameters, String, boolean)} *

* If the AWS stats are not null then a {@link AwsStatisticsCollector}. * is created to bind to the two. - * Important: until this binding works properly across regions, - * this should be null. */ @Override - public AmazonS3 createS3Client(URI name, - final String bucket, - final AWSCredentialsProvider credentials, - final String userAgentSuffix, - final StatisticsFromAwsSdk statisticsFromAwsSdk) throws IOException { + public AmazonS3 createS3Client( + final URI uri, + final S3ClientCreationParameters parameters) throws IOException { Configuration conf = getConf(); final ClientConfiguration awsConf = S3AUtils - .createAwsConf(conf, bucket, Constants.AWS_SERVICE_IDENTIFIER_S3); + .createAwsConf(conf, + uri.getHost(), + Constants.AWS_SERVICE_IDENTIFIER_S3); + // add any headers + parameters.getHeaders().forEach((h, v) -> + awsConf.addHeader(h, v)); // When EXPERIMENTAL_AWS_INTERNAL_THROTTLING is false // throttling is explicitly disabled on the S3 client so that @@ -96,111 +95,62 @@ public AmazonS3 createS3Client(URI name, conf.getBoolean(EXPERIMENTAL_AWS_INTERNAL_THROTTLING, EXPERIMENTAL_AWS_INTERNAL_THROTTLING_DEFAULT)); - if (!StringUtils.isEmpty(userAgentSuffix)) { - awsConf.setUserAgentSuffix(userAgentSuffix); + if (!StringUtils.isEmpty(parameters.getUserAgentSuffix())) { + awsConf.setUserAgentSuffix(parameters.getUserAgentSuffix()); } - // optional metrics - RequestMetricCollector metrics = statisticsFromAwsSdk != null - ? new AwsStatisticsCollector(statisticsFromAwsSdk) - : null; - return newAmazonS3Client( - credentials, + return buildAmazonS3Client( awsConf, - metrics, - conf.getTrimmed(ENDPOINT, ""), - conf.getBoolean(PATH_STYLE_ACCESS, false)); + parameters); } /** - * 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. + * Use the Builder API to create a an AWS S3 client. *

- * This has a more complex endpoint configuration in a - * way which does not yet work in this code in a way - * which doesn't trigger regressions. So it is only used - * when SDK metrics are supplied. - * @param credentials credentials to use + * This has a more complex endpoint configuration mechanism + * which initially caused problems; the + * {@code withForceGlobalBucketAccessEnabled(true)} + * command is critical here. * @param awsConf AWS configuration - * @param metrics metrics collector or null - * @param endpoint endpoint string; may be "" - * @param pathStyleAccess enable path style access? + * @param parameters parameters * @return new AmazonS3 client */ - private AmazonS3 buildAmazonS3Client( - final AWSCredentialsProvider credentials, + protected AmazonS3 buildAmazonS3Client( final ClientConfiguration awsConf, - final RequestMetricCollector metrics, - final String endpoint, - final boolean pathStyleAccess) { + final S3ClientCreationParameters parameters) { AmazonS3ClientBuilder b = AmazonS3Client.builder(); - b.withCredentials(credentials); + b.withCredentials(parameters.getCredentialSet()); b.withClientConfiguration(awsConf); - b.withPathStyleAccessEnabled(pathStyleAccess); - if (metrics != null) { - b.withMetricsCollector(metrics); + b.withPathStyleAccessEnabled(parameters.isPathStyleAccess()); + + if (parameters.getMetrics() != null) { + b.withMetricsCollector( + new AwsStatisticsCollector(parameters.getMetrics())); + } + if (parameters.getRequestHandlers() != null) { + b.withRequestHandlers( + parameters.getRequestHandlers().toArray(new RequestHandler2[0])); + } + if (parameters.getMonitoringListener() != null) { + b.withMonitoringListener(parameters.getMonitoringListener()); } // endpoint set up is a PITA - // client.setEndpoint("") is no longer available AwsClientBuilder.EndpointConfiguration epr - = createEndpointConfiguration(endpoint, awsConf); + = createEndpointConfiguration(parameters.getEndpoint(), + awsConf); if (epr != null) { // an endpoint binding was constructed: use it. b.withEndpointConfiguration(epr); + } else { + // no idea what the endpoint is, so tell the SDK + // to work it out at the cost of an extra HEAD request + b.withForceGlobalBucketAccessEnabled(true); } final AmazonS3 client = b.build(); return client; } - /** - * Wrapper around constructor for {@link AmazonS3} client. - * Override this to provide an extended version of the client. - *

- * This uses a deprecated constructor -it is currently - * the only one which works for us. - * @param credentials credentials to use - * @param awsConf AWS configuration - * @param endpoint endpoint string; may be "" - * @param pathStyleAccess enable path style access? - * @return new AmazonS3 client - */ - @SuppressWarnings("deprecation") - private AmazonS3 classicAmazonS3Client( - AWSCredentialsProvider credentials, - ClientConfiguration awsConf, - final String endpoint, - final boolean pathStyleAccess) { - final AmazonS3 client = new AmazonS3Client(credentials, awsConf); - return configureAmazonS3Client(client, endpoint, pathStyleAccess); - } - /** * Configure classic S3 client. *

@@ -226,31 +176,6 @@ protected static AmazonS3 configureAmazonS3Client(AmazonS3 s3, throw new IllegalArgumentException(msg, e); } } - return applyS3ClientOptions(s3, pathStyleAccess); - } - - /** - * Perform any tuning of the {@code S3ClientOptions} settings based on - * the Hadoop configuration. - * This is different from the general AWS configuration creation as - * it is unique to S3 connections. - *

- * The {@link Constants#PATH_STYLE_ACCESS} option enables path-style access - * to S3 buckets if configured. By default, the - * behavior is to use virtual hosted-style access with URIs of the form - * {@code http://bucketname.s3.amazonaws.com} - *

- * Enabling path-style access and a - * region-specific endpoint switches the behavior to use URIs of the form - * {@code http://s3-eu-west-1.amazonaws.com/bucketname}. - * It is common to use this when connecting to private S3 servers, as it - * avoids the need to play with DNS entries. - * @param s3 S3 client - * @param pathStyleAccess enable path style access? - * @return the S3 client - */ - protected static AmazonS3 applyS3ClientOptions(AmazonS3 s3, - final boolean pathStyleAccess) { if (pathStyleAccess) { LOG.debug("Enabling path style access!"); s3.setS3ClientOptions(S3ClientOptions.builder() diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java index ddc492235dba8..c11581f1d5d78 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java @@ -19,8 +19,6 @@ package org.apache.hadoop.fs.s3a; import com.amazonaws.ClientConfiguration; -import com.amazonaws.auth.AWSCredentialsProvider; -import com.amazonaws.metrics.RequestMetricCollector; import com.amazonaws.services.s3.AmazonS3; import org.apache.hadoop.classification.InterfaceAudience; @@ -31,31 +29,25 @@ * This client is for testing only; it is in the production * {@code hadoop-aws} module to enable integration tests to use this * just by editing the Hadoop configuration used to bring up the client. + * + * The factory uses the older constructor-based instantiation/configuration + * of the client, so does not wire up metrics, handlers etc. */ @InterfaceAudience.Private @InterfaceStability.Unstable public class InconsistentS3ClientFactory extends DefaultS3ClientFactory { - /** - * Create the inconsistent client. - * Logs a warning that this is being done. - * @param credentials credentials to use - * @param awsConf AWS configuration - * @param metrics metric collector - * @param endpoint AWS endpoint - * @param pathStyleAccess should path style access be supported? - * @return an inconsistent client. - */ @Override - protected AmazonS3 newAmazonS3Client(AWSCredentialsProvider credentials, - ClientConfiguration awsConf, - final RequestMetricCollector metrics, - final String endpoint, - final boolean pathStyleAccess) { + protected AmazonS3 buildAmazonS3Client( + final ClientConfiguration awsConf, + final S3ClientCreationParameters parameters) { LOG.warn("** FAILURE INJECTION ENABLED. Do not run in production! **"); InconsistentAmazonS3Client s3 - = new InconsistentAmazonS3Client(credentials, awsConf, getConf()); - configureAmazonS3Client(s3, endpoint, pathStyleAccess); + = new InconsistentAmazonS3Client( + parameters.getCredentialSet(), awsConf, getConf()); + configureAmazonS3Client(s3, + parameters.getEndpoint(), + parameters.isPathStyleAccess()); return s3; } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index f625346957f4a..8db5d51def84e 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -69,7 +69,6 @@ import com.amazonaws.services.s3.model.ObjectMetadata; import com.amazonaws.services.s3.model.PutObjectRequest; import com.amazonaws.services.s3.model.PutObjectResult; - import com.amazonaws.services.s3.model.SSEAwsKeyManagementParams; import com.amazonaws.services.s3.model.SSECustomerKey; import com.amazonaws.services.s3.model.UploadPartRequest; @@ -83,7 +82,6 @@ import com.amazonaws.event.ProgressListener; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; -import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListeningExecutorService; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -166,7 +164,6 @@ import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics; import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics; import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext; -import org.apache.hadoop.fs.s3a.statistics.StatisticsFromAwsSdk; import org.apache.hadoop.fs.s3a.statistics.impl.BondedS3AStatisticsContext; import org.apache.hadoop.fs.s3native.S3xLoginHelper; import org.apache.hadoop.io.retry.RetryPolicies; @@ -198,7 +195,6 @@ import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletionIgnoringExceptions; import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isObjectNotFound; import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isUnknownBucket; -import static org.apache.hadoop.fs.s3a.impl.InternalConstants.AWS_SDK_METRICS_ENABLED; import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_404; import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.fixBucketRegion; import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.logDnsLookup; @@ -376,6 +372,11 @@ public void initialize(URI name, Configuration originalConf) LOG.debug("Initializing S3AFileSystem for {}", bucket); // clone the configuration into one with propagated bucket options Configuration conf = propagateBucketOptions(originalConf, bucket); + // fix up the classloader of the configuration to be whatever + // classloader loaded this filesystem. + // See: HADOOP-17372 + conf.setClassLoader(this.getClass().getClassLoader()); + // patch the Hadoop security providers patchSecurityCredentialProviders(conf); // look for delegation token support early. @@ -740,16 +741,17 @@ private void bindAWSClient(URI name, boolean dtEnabled) throws IOException { S3_CLIENT_FACTORY_IMPL, DEFAULT_S3_CLIENT_FACTORY_IMPL, S3ClientFactory.class); - StatisticsFromAwsSdk awsStats = null; - // TODO: HADOOP-16830 when the S3 client building code works - // with different regions, - // then non-null stats can be passed in here. - if (AWS_SDK_METRICS_ENABLED) { - awsStats = statisticsContext.newStatisticsFromAwsSdk(); - } + S3ClientFactory.S3ClientCreationParameters parameters = null; + parameters = new S3ClientFactory.S3ClientCreationParameters() + .withCredentialSet(credentials) + .withEndpoint(conf.getTrimmed(ENDPOINT, DEFAULT_ENDPOINT)) + .withMetrics(statisticsContext.newStatisticsFromAwsSdk()) + .withPathStyleAccess(conf.getBoolean(PATH_STYLE_ACCESS, false)) + .withUserAgentSuffix(uaSuffix); s3 = ReflectionUtils.newInstance(s3ClientFactoryClass, conf) - .createS3Client(getUri(), bucket, credentials, uaSuffix, awsStats); + .createS3Client(getUri(), + parameters); } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java index e04d3b5cbd269..dbb39fb662408 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java @@ -18,38 +18,246 @@ package org.apache.hadoop.fs.s3a; +import javax.annotation.Nullable; import java.io.IOException; import java.net.URI; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.handlers.RequestHandler2; +import com.amazonaws.monitoring.MonitoringListener; import com.amazonaws.services.s3.AmazonS3; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.s3a.statistics.StatisticsFromAwsSdk; +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_ENDPOINT; + /** * Factory for creation of {@link AmazonS3} client instances. + * Important: HBase's HBoss module implements this interface in its + * tests. + * Take care when updating this interface to ensure that a client + * implementing only the deprecated method will work. + * See https://github.com/apache/hbase-filesystem + * */ -@InterfaceAudience.Private -@InterfaceStability.Unstable +@InterfaceAudience.LimitedPrivate("HBoss") +@InterfaceStability.Evolving public interface S3ClientFactory { /** * Creates a new {@link AmazonS3} client. * - * @param name raw input S3A file system URI - * @param bucket Optional bucket to use to look up per-bucket proxy secrets - * @param credentialSet credentials to use - * @param userAgentSuffix optional suffix for the UA field. - * @param statisticsFromAwsSdk binding for AWS stats - may be null + * @param uri S3A file system URI + * @param parameters parameter object * @return S3 client * @throws IOException IO problem */ - AmazonS3 createS3Client(URI name, - String bucket, - AWSCredentialsProvider credentialSet, - String userAgentSuffix, - StatisticsFromAwsSdk statisticsFromAwsSdk) throws IOException; + AmazonS3 createS3Client(URI uri, + S3ClientCreationParameters parameters) throws IOException; + + /** + * Settings for the S3 Client. + * Implemented as a class to pass in so that adding + * new parameters does not break the binding of + * external implementations of the factory. + */ + final class S3ClientCreationParameters { + + /** + * Credentials. + */ + private AWSCredentialsProvider credentialSet; + + /** + * Endpoint. + */ + private String endpoint = DEFAULT_ENDPOINT; + + /** + * Custom Headers. + */ + private final Map headers = new HashMap<>(); + + /** + * Monitoring listener. + */ + private MonitoringListener monitoringListener; + + /** + * RequestMetricCollector metrics...if not-null will be wrapped + * with an {@code AwsStatisticsCollector} and passed to + * the client. + */ + private StatisticsFromAwsSdk metrics; + + /** + * Use (deprecated) path style access. + */ + private boolean pathStyleAccess; + + /** + * This is in the settings awaiting wiring up and testing. + */ + private boolean requesterPays; + + /** + * Request handlers; used for auditing, X-Ray etc. + */ + private List requestHandlers; + + /** + * Suffix to UA. + */ + private String userAgentSuffix = ""; + + public List getRequestHandlers() { + return requestHandlers; + } + + /** + * List of request handlers. + * @param handlers handler list. + * @return this object + */ + public S3ClientCreationParameters withRequestHandlers( + @Nullable final List handlers) { + requestHandlers = handlers; + return this; + } + + public MonitoringListener getMonitoringListener() { + return monitoringListener; + } + + /** + * listener for AWS monitoring events. + * @param listener listener + * @return this object + */ + public S3ClientCreationParameters withMonitoringListener( + @Nullable final MonitoringListener listener) { + monitoringListener = listener; + return this; + } + + public StatisticsFromAwsSdk getMetrics() { + return metrics; + } + + /** + * Metrics binding. This is the S3A-level + * statistics interface, which will be wired + * up to the AWS callbacks. + * @param statistics statistics implementation + * @return this object + */ + public S3ClientCreationParameters withMetrics( + @Nullable final StatisticsFromAwsSdk statistics) { + metrics = statistics; + return this; + } + + /** + * Requester pays option. Not yet wired up. + * @param value new value + * @return the builder + */ + public S3ClientCreationParameters withRequesterPays( + final boolean value) { + requesterPays = value; + return this; + } + + public boolean isRequesterPays() { + return requesterPays; + } + + public AWSCredentialsProvider getCredentialSet() { + return credentialSet; + } + + /** + * Set credentials. + * @param value new value + * @return the builder + */ + + public S3ClientCreationParameters withCredentialSet( + final AWSCredentialsProvider value) { + credentialSet = value; + return this; + } + + public String getUserAgentSuffix() { + return userAgentSuffix; + } + + /** + * Set UA suffix. + * @param value new value + * @return the builder + */ + + public S3ClientCreationParameters withUserAgentSuffix( + final String value) { + userAgentSuffix = value; + return this; + } + + public String getEndpoint() { + return endpoint; + } + + /** + * Set endpoint. + * @param value new value + * @return the builder + */ + + public S3ClientCreationParameters withEndpoint( + final String value) { + endpoint = value; + return this; + } + + public boolean isPathStyleAccess() { + return pathStyleAccess; + } + + /** + * Set path access option. + * @param value new value + * @return the builder + */ + public S3ClientCreationParameters withPathStyleAccess( + final boolean value) { + pathStyleAccess = value; + return this; + } + + /** + * Add a custom header. + * @param header header name + * @param value new value + * @return the builder + */ + public S3ClientCreationParameters withHeader( + String header, String value) { + headers.put(header, value); + return this; + } + /** + * Get the map of headers. + * @return (mutable) header map + */ + public Map getHeaders() { + return headers; + } + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java index 358ec261ccaae..a5ce1f68ad3fc 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java @@ -111,10 +111,4 @@ private InternalConstants() { */ 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 = true; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java index 868ec2c36cd03..bd121ba2728eb 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java @@ -23,13 +23,10 @@ import java.net.URI; import java.util.ArrayList; -import com.amazonaws.auth.AWSCredentialsProvider; import com.amazonaws.services.s3.AmazonS3; import com.amazonaws.services.s3.model.MultipartUploadListing; import com.amazonaws.services.s3.model.Region; -import org.apache.hadoop.fs.s3a.statistics.StatisticsFromAwsSdk; - /** * An {@link S3ClientFactory} that returns Mockito mocks of the {@link AmazonS3} * interface suitable for unit testing. @@ -37,12 +34,10 @@ public class MockS3ClientFactory implements S3ClientFactory { @Override - public AmazonS3 createS3Client(URI name, - final String bucket, - final AWSCredentialsProvider credentialSet, - final String userAgentSuffix, - final StatisticsFromAwsSdk statisticsFromAwsSdks) { + public AmazonS3 createS3Client(URI uri, + final S3ClientCreationParameters parameters) { AmazonS3 s3 = mock(AmazonS3.class); + String bucket = uri.getHost(); when(s3.doesBucketExist(bucket)).thenReturn(true); when(s3.doesBucketExistV2(bucket)).thenReturn(true); // this listing is used in startup if purging is enabled, so diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestCustomSigner.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestCustomSigner.java index d9cb1d97bf5fc..499107bc7002a 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestCustomSigner.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestCustomSigner.java @@ -23,12 +23,11 @@ import java.util.HashMap; import java.util.Map; import java.util.Objects; +import java.util.concurrent.atomic.AtomicInteger; import com.amazonaws.SignableRequest; import com.amazonaws.auth.AWSCredentials; import com.amazonaws.auth.Signer; -import com.amazonaws.services.s3.AmazonS3; -import com.amazonaws.services.s3.AmazonS3ClientBuilder; import com.amazonaws.services.s3.internal.AWSS3V4Signer; import org.assertj.core.api.Assertions; import org.junit.Test; @@ -40,14 +39,15 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.AbstractS3ATestBase; -import org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider; +import org.apache.hadoop.fs.s3a.Constants; +import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.auth.ITestCustomSigner.CustomSignerInitializer.StoreValue; import org.apache.hadoop.fs.s3a.auth.delegation.DelegationTokenProvider; import org.apache.hadoop.security.UserGroupInformation; import static org.apache.hadoop.fs.s3a.Constants.CUSTOM_SIGNERS; import static org.apache.hadoop.fs.s3a.Constants.SIGNING_ALGORITHM_S3; -import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.fixBucketRegion; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching; /** * Tests for custom Signers and SignerInitializers. @@ -62,23 +62,32 @@ public class ITestCustomSigner extends AbstractS3ATestBase { private String regionName; + private String endpoint; + @Override public void setup() throws Exception { super.setup(); - regionName = determineRegion(getFileSystem().getBucket()); + final S3AFileSystem fs = getFileSystem(); + regionName = determineRegion(fs.getBucket()); LOG.info("Determined region name to be [{}] for bucket [{}]", regionName, - getFileSystem().getBucket()); + fs.getBucket()); + endpoint = fs.getConf() + .get(Constants.ENDPOINT, Constants.CENTRAL_ENDPOINT); + LOG.info("Test endpoint is {}", endpoint); } @Test public void testCustomSignerAndInitializer() throws IOException, InterruptedException { + final Path basePath = path(getMethodName()); UserGroupInformation ugi1 = UserGroupInformation.createRemoteUser("user1"); - FileSystem fs1 = runMkDirAndVerify(ugi1, "/customsignerpath1", "id1"); + FileSystem fs1 = runMkDirAndVerify(ugi1, + new Path(basePath, "customsignerpath1"), "id1"); UserGroupInformation ugi2 = UserGroupInformation.createRemoteUser("user2"); - FileSystem fs2 = runMkDirAndVerify(ugi2, "/customsignerpath2", "id2"); + FileSystem fs2 = runMkDirAndVerify(ugi2, + new Path(basePath, "customsignerpath2"), "id2"); Assertions.assertThat(CustomSignerInitializer.knownStores.size()) .as("Num registered stores mismatch").isEqualTo(2); @@ -91,20 +100,19 @@ public void testCustomSignerAndInitializer() } private FileSystem runMkDirAndVerify(UserGroupInformation ugi, - String pathString, String identifier) + Path finalPath, String identifier) throws IOException, InterruptedException { Configuration conf = createTestConfig(identifier); - Path path = new Path(pathString); - path = path.makeQualified(getFileSystem().getUri(), - getFileSystem().getWorkingDirectory()); - - Path finalPath = path; return ugi.doAs((PrivilegedExceptionAction) () -> { - int invocationCount = CustomSigner.invocationCount; + int instantiationCount = CustomSigner.getInstantiationCount(); + int invocationCount = CustomSigner.getInvocationCount(); FileSystem fs = finalPath.getFileSystem(conf); fs.mkdirs(finalPath); - Assertions.assertThat(CustomSigner.invocationCount) - .as("Invocation count lower than expected") + Assertions.assertThat(CustomSigner.getInstantiationCount()) + .as("CustomSigner Instantiation count lower than expected") + .isGreaterThan(instantiationCount); + Assertions.assertThat(CustomSigner.getInvocationCount()) + .as("CustomSigner Invocation count lower than expected") .isGreaterThan(invocationCount); Assertions.assertThat(CustomSigner.lastStoreValue) @@ -118,6 +126,12 @@ private FileSystem runMkDirAndVerify(UserGroupInformation ugi, }); } + /** + * Create a test conf with the custom signer; fixes up + * endpoint to be that of the test FS. + * @param identifier test key. + * @return a configuration for a filesystem. + */ private Configuration createTestConfig(String identifier) { Configuration conf = createConfiguration(); @@ -128,24 +142,35 @@ private Configuration createTestConfig(String identifier) { conf.set(TEST_ID_KEY, identifier); conf.set(TEST_REGION_KEY, regionName); + conf.set(Constants.ENDPOINT, endpoint); + // make absolutely sure there is no caching. + disableFilesystemCaching(conf); return conf; } private String determineRegion(String bucketName) throws IOException { - String region = getFileSystem().getBucketLocation(bucketName); - return fixBucketRegion(region); + return getFileSystem().getBucketLocation(bucketName); } @Private public static final class CustomSigner implements Signer { - private static int invocationCount = 0; + + private static AtomicInteger instantiationCount = new AtomicInteger(0); + private static AtomicInteger invocationCount = new AtomicInteger(0); private static StoreValue lastStoreValue; + public CustomSigner() { + int c = instantiationCount.incrementAndGet(); + LOG.info("Creating Signer #{}", c); + } + @Override public void sign(SignableRequest request, AWSCredentials credentials) { - invocationCount++; + int c = invocationCount.incrementAndGet(); + LOG.info("Signing request #{}", c); + String host = request.getEndpoint().getHost(); String bucketName = host.split("\\.")[0]; try { @@ -159,6 +184,14 @@ public void sign(SignableRequest request, AWSCredentials credentials) { realSigner.setRegionName(lastStoreValue.conf.get(TEST_REGION_KEY)); realSigner.sign(request, credentials); } + + public static int getInstantiationCount() { + return instantiationCount.get(); + } + + public static int getInvocationCount() { + return invocationCount.get(); + } } @Private diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationInFileystem.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationInFileystem.java index f5562bdf32e5e..26655de9d4417 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationInFileystem.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationInFileystem.java @@ -43,8 +43,8 @@ import org.apache.hadoop.fs.s3a.S3AEncryptionMethods; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3ATestUtils; +import org.apache.hadoop.fs.s3a.S3ClientFactory; 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.io.Text; @@ -72,7 +72,6 @@ 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.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.intercept; import static org.hamcrest.Matchers.containsString; @@ -557,23 +556,22 @@ public void testDelegationBindingMismatch2() throws Throwable { */ protected ObjectMetadata readLandsatMetadata(final S3AFileSystem delegatedFS) throws Exception { - AWSCredentialProviderList testing + AWSCredentialProviderList testingCreds = delegatedFS.shareCredentials("testing"); URI landsat = new URI(DEFAULT_CSVTEST_FILE); DefaultS3ClientFactory factory = new DefaultS3ClientFactory(); - Configuration conf = new Configuration(delegatedFS.getConf()); - conf.set(ENDPOINT, ""); - factory.setConf(conf); + factory.setConf(new Configuration(delegatedFS.getConf())); String host = landsat.getHost(); - StatisticsFromAwsSdk awsStats = null; - if (AWS_SDK_METRICS_ENABLED) { - awsStats = new EmptyS3AStatisticsContext() - .newStatisticsFromAwsSdk(); - } - AmazonS3 s3 = factory.createS3Client(landsat, host, testing, - "ITestSessionDelegationInFileystem", awsStats); + S3ClientFactory.S3ClientCreationParameters parameters = null; + parameters = new S3ClientFactory.S3ClientCreationParameters() + .withCredentialSet(testingCreds) + .withEndpoint(DEFAULT_ENDPOINT) + .withMetrics(new EmptyS3AStatisticsContext() + .newStatisticsFromAwsSdk()) + .withUserAgentSuffix("ITestSessionDelegationInFileystem"); + AmazonS3 s3 = factory.createS3Client(landsat, parameters); return Invoker.once("HEAD", host, () -> s3.getObjectMetadata(host, landsat.getPath().substring(1))); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperations.java index b025f6f0969fe..4d7f81d019b74 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperations.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperations.java @@ -175,6 +175,7 @@ public void testCreateAbortEmptyFile() throws Throwable { Path destFile = methodPath(filename); Path pendingFilePath = makeMagic(destFile); touch(fs, pendingFilePath); + waitForConsistency(); validateIntermediateAndFinalPaths(pendingFilePath, destFile); Path pendingDataPath = validatePendingCommitData(filename, pendingFilePath); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestAWSStatisticCollection.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestAWSStatisticCollection.java new file mode 100644 index 0000000000000..bf1253c385adf --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestAWSStatisticCollection.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.statistics; + +import org.junit.Test; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.AbstractS3ATestBase; +import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.fs.statistics.IOStatistics; + +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_ENDPOINT; +import static org.apache.hadoop.fs.s3a.Constants.ENDPOINT; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.getLandsatCSVPath; +import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_REQUEST; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticCounter; + +/** + * Verify that AWS SDK statistics are wired up. + * This test tries to read data from US-east-1 and us-west-2 buckets + * so as to be confident that the nuances of region mapping + * are handed correctly (HADOOP-13551). + * The statistics are probed to verify that the wiring up is complete. + */ +public class ITestAWSStatisticCollection extends AbstractS3ATestBase { + + private static final Path COMMON_CRAWL_PATH + = new Path("s3a://osm-pds/planet/planet-latest.orc"); + + @Test + public void testLandsatStatistics() throws Throwable { + final Configuration conf = getConfiguration(); + // skips the tests if the lansdat path isn't the default. + Path path = getLandsatCSVPath(conf); + conf.set(ENDPOINT, DEFAULT_ENDPOINT); + conf.unset("fs.s3a.bucket.landsat-pds.endpoint"); + + try (S3AFileSystem fs = (S3AFileSystem) path.getFileSystem(conf)) { + fs.getObjectMetadata(path); + IOStatistics iostats = fs.getIOStatistics(); + assertThatStatisticCounter(iostats, + STORE_IO_REQUEST.getSymbol()) + .isGreaterThanOrEqualTo(1); + } + } + + @Test + public void testCommonCrawlStatistics() throws Throwable { + final Configuration conf = getConfiguration(); + // skips the tests if the lansdat path isn't the default. + Path landsatPath = getLandsatCSVPath(conf); + + Path path = COMMON_CRAWL_PATH; + conf.set(ENDPOINT, ""); + + try (S3AFileSystem fs = (S3AFileSystem) path.getFileSystem(conf)) { + fs.getObjectMetadata(path); + IOStatistics iostats = fs.getIOStatistics(); + assertThatStatisticCounter(iostats, + STORE_IO_REQUEST.getSymbol()) + .isGreaterThanOrEqualTo(1); + } + } + +} From 5402c40e8ad5b2537c89e1a4f6b8046306c2d26c Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 18 Mar 2021 15:15:48 +0000 Subject: [PATCH 2/8] HADOOP-13351. Mehakmeet's review comments Change-Id: I9ea831f5fec48ebcc7d08fe0e5d6918f8dfa2786 --- .../hadoop/fs/s3a/DefaultS3ClientFactory.java | 12 +++--------- .../hadoop/fs/s3a/auth/ITestCustomSigner.java | 15 +++++++++------ .../statistics/ITestAWSStatisticCollection.java | 8 ++++---- 3 files changed, 16 insertions(+), 19 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java index 6efde1d9158fe..e93218f6f355c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java @@ -22,7 +22,6 @@ import java.net.URI; import com.amazonaws.ClientConfiguration; -import com.amazonaws.auth.AWSCredentialsProvider; import com.amazonaws.client.builder.AwsClientBuilder; import com.amazonaws.handlers.RequestHandler2; import com.amazonaws.services.s3.AmazonS3; @@ -49,7 +48,7 @@ /** * The default {@link S3ClientFactory} implementation. * This calls the AWS SDK to configure and create an - * {@link AmazonS3Client} that communicates with the S3 service. + * {@code AmazonS3Client} that communicates with the S3 service. */ @InterfaceAudience.Private @InterfaceStability.Unstable @@ -57,8 +56,6 @@ public class DefaultS3ClientFactory extends Configured implements S3ClientFactory { 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. @@ -68,10 +65,7 @@ public class DefaultS3ClientFactory extends Configured /** * Create the client by preparing the AwsConf configuration - * and then invoking {@link #buildAmazonS3Client(AWSCredentialsProvider, ClientConfiguration, S3ClientCreationParameters, String, boolean)} - *

- * If the AWS stats are not null then a {@link AwsStatisticsCollector}. - * is created to bind to the two. + * and then invoking {@code buildAmazonS3Client()} */ @Override public AmazonS3 createS3Client( @@ -105,7 +99,7 @@ public AmazonS3 createS3Client( } /** - * Use the Builder API to create a an AWS S3 client. + * Use the Builder API to create an AWS S3 client. *

* This has a more complex endpoint configuration mechanism * which initially caused problems; the diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestCustomSigner.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestCustomSigner.java index 499107bc7002a..72af1752b1253 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestCustomSigner.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestCustomSigner.java @@ -157,18 +157,21 @@ private String determineRegion(String bucketName) throws IOException { public static final class CustomSigner implements Signer { - private static AtomicInteger instantiationCount = new AtomicInteger(0); - private static AtomicInteger invocationCount = new AtomicInteger(0); + private static final AtomicInteger INSTANTIATION_COUNT = + new AtomicInteger(0); + private static final AtomicInteger INVOCATION_COUNT = + new AtomicInteger(0); + private static StoreValue lastStoreValue; public CustomSigner() { - int c = instantiationCount.incrementAndGet(); + int c = INSTANTIATION_COUNT.incrementAndGet(); LOG.info("Creating Signer #{}", c); } @Override public void sign(SignableRequest request, AWSCredentials credentials) { - int c = invocationCount.incrementAndGet(); + int c = INVOCATION_COUNT.incrementAndGet(); LOG.info("Signing request #{}", c); String host = request.getEndpoint().getHost(); @@ -186,11 +189,11 @@ public void sign(SignableRequest request, AWSCredentials credentials) { } public static int getInstantiationCount() { - return instantiationCount.get(); + return INSTANTIATION_COUNT.get(); } public static int getInvocationCount() { - return invocationCount.get(); + return INVOCATION_COUNT.get(); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestAWSStatisticCollection.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestAWSStatisticCollection.java index bf1253c385adf..e7696996dbd1a 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestAWSStatisticCollection.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestAWSStatisticCollection.java @@ -47,7 +47,7 @@ public class ITestAWSStatisticCollection extends AbstractS3ATestBase { @Test public void testLandsatStatistics() throws Throwable { final Configuration conf = getConfiguration(); - // skips the tests if the lansdat path isn't the default. + // skips the tests if the landsat path isn't the default. Path path = getLandsatCSVPath(conf); conf.set(ENDPOINT, DEFAULT_ENDPOINT); conf.unset("fs.s3a.bucket.landsat-pds.endpoint"); @@ -64,11 +64,11 @@ public void testLandsatStatistics() throws Throwable { @Test public void testCommonCrawlStatistics() throws Throwable { final Configuration conf = getConfiguration(); - // skips the tests if the lansdat path isn't the default. - Path landsatPath = getLandsatCSVPath(conf); + // skips the tests if the landsat path isn't the default. + getLandsatCSVPath(conf); Path path = COMMON_CRAWL_PATH; - conf.set(ENDPOINT, ""); + conf.set(ENDPOINT, DEFAULT_ENDPOINT); try (S3AFileSystem fs = (S3AFileSystem) path.getFileSystem(conf)) { fs.getObjectMetadata(path); From 5d7be34d94f4a99c73378c7f88644bc08a2c2692 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 2 Feb 2021 16:25:06 +0000 Subject: [PATCH 3/8] HADOOP-17511. Add audit logging to S3A connector Notion of AuditSpan which is created for a given operation; goal is to pass it along everywhere. It's thread local per FS-instance; store operations pick this up in their constructor from the StoreContext. The entryPoint() method in S3A FS has been enhanced to initiate the spans. For this to work, internal code SHALL NOT call those entry points (Done) and all public API points MUST be declared as entry points. Tricky given the lack of layering in this branch. The auditing is intended to be a plugin point; currently there is one which logs at Info (yes, noisy but good for development) Change-Id: Idddbbd3138e52b40e97f9c2383453190a02cd8ec HADOOP-15711. Audit Log. - compiles - activate() is no-op if current thread audit span is the one to activate. - input streams pick up context - some audit-driven tuning of test teardown ! running tests: key omitted coverage is commit operations. One thing to (seriously) consider is creating a WriteOperationsHelper for each writing op, storing the context in it, then activating on each operation. Need to consider full lifecycle. Change-Id: I2998c4980fce3c65984fd728ec98e3299d4c329d HADOOP-17511. Audit Log * mkdir working again * review changed code and tune slightly * strip out some surplus getFileStatus calls from contract test utils. Change-Id: I5124d6e9579abbe650fe53e2531aff50dc2fedcd HADOOP-15711. Auditing Listing is (probably) wired up IOStats integration too Change-Id: I4293c99d7a67a05d659ec3c690daeab5c883a5fb HADOOP-15711. Auditing * Dynamic loading of audit class * IOStatistics integration is better * Fewer (no?) activation of active span in current thread * log audit uses timestamp of creation as partial ID; and unique counter per instance (for better multi-fs logging) * thread pools for s3 buckets include bucket name (gets into log4j logs) Change-Id: Id2f977354851f3dd96c005450d46b8181dbf567e HADOOP-15711. S3A Auditing All FS entry points instrumented for auditing * WriteOperationHelper is no longer a singleton and picks up the active span when created. This lets us use it in places like commit logic without them needing changed everywhere * various helper classes of S3A now extend AbstractStoreOperation so pick up their span automatically. With the WriteOperationHelper code it's *less* critical, but... * before almost every S3 API call, the active span gets to "prepare" the request. That's where headers can be set, etc. Even without header manipulation -this is where checks for calls without a valid span can go. * which I'm trying to do in the logging code, including incrementing the .failed counter. Doesn't seem to be live tho Yes, this is a fairly large patch, but not through complexity, just that every entry point needs to become a span. Life would be much, much easier with an MVC-style structure as its ~impossible to be confident that various public S3AFS API calls aren't being made externally. Change-Id: I22dfe6f5bdb7bae4c4ba14f2a77e460f12ca1f14 HADOOP-17511. Auditing * adds referer header to AWS requests -this gets into the logs (todo: optional) * can declare that lack of a span for an s3 request is a failure (this doesn't get picked up in copy ops BTW) * passes span through submitted callbacks * moves most of the span management logic into ActiveAuditManager * getWriteOperationHelper() now dynamically creates a helper within the active span, rather than a long-lived static instance. Change-Id: Iecb9911a84421cec0e33011db60159c7c6ee79f3 HADOOP-15711. Auditing: scope/coverage running all the tests with auditing set to fail if any call is made unaudited highlights scope for improvement. Production * Critical: nested scopes * Committers: scoping * S3GuardTool spans around lower-level operations. * Multipart listing to become a spanned entry point. Test * create spans before all tests doing low-level things Change-Id: Ic3577681181942af1a0807dc78ff6393bb3d325f HADOOP-17511. audit elements as ?k=v params; MR/job ID. * multiple params go in as key=value * includes UGI principal * and the MR or spark job which created the FS instances/auditor For MR/Distcp, JobID will work. For spark, it's not enough. Assuming that the FS instances are cached across tasks (they should be for performance reasons) the scope is incorrect. We'd really need task setup patching in a Context scope somewhere Change-Id: Idc4e7eec29e0137e002be4ae56efed8798b8e337 HADOOP-17511. auditing ongoing * CommonContext for JobID from spark/MR jobs * RequestFactory from the HADOOP-16848 refactoring -preparation takes place there * new s3a.api package for the new API stuff Change-Id: Ib11e53304506897f2366672b268257e668a403ed HADOOP-15711. Auditing: use RequestFactory everywhere and wrap invocation as appropriate Change-Id: Ibabb5828c9db018cd9262ae814d37a1a1a15fb02 HADOOP-17511. Auditing: request preparation now happens in AWS SDK Change-Id: Ib2be23510db72b6152658523ae7c79117f0e4391 HADOOP-17511 Auditing; with HADOOP-13551 metrics. * Get the new builder working, with metric wireup the test * AuditSpan prepareRequest() ops are called through S3 SDK * List iterators are cleaned up in s3a code and FS Shell; this closes the spans. * Starting on a document Change-Id: I728fa071f81da5dbc52fd517b537004107cdf65b HADOOP-15711. Auditing -trying to get COPY To audit Wiring up through request header to try to attach a context to all requests. But: the context list is *not* passed through the xfer manager Next attempt: state change callbacks. This may work for one thread, but as many blocks are copied across different threads, it lacks complete coverage. Plan: give up on trying to get an audit log entry for copy/rename. The only next step would be to move off xfer manager and do it ourselves, which isn't necessarily bad, but it is extra work. Or: somehow the thread pool/executor could be made "clever", with every thread submitted through the transfer manager (created uniquely for each copy) to pick up the span Change-Id: I424675228a0fd9b45c0b1ce8de24b3c20d9cfdfe HADOOP-15711. Auditing. * lots more of logging and useful stuff at that * Input stream correctly creating requests through the system. This includes moving the input stream away from direct use of the s3 client to through an interface -the S3AFS impl includes the audit span and sets it up. Change-Id: Ic8969e4444333cf557247aff8c1bcaca64ed9ebf HADOOP-15711. Move ~all setup into AuditManager * AuditManager is CompositeService; wraps active audit Service * Unit tests for span lifecycle showed more work was needed there in order to have the "reset span" (naming?) always avaiable, active... * logging auditor moves very low level logs to trace; preparation -> debug Change-Id: I0ef81dd3d26fe2b8eee943c73808e4791c1a1dd7 HADOOP-15711. Auditing: mkdir working; listing getting complex All the mkdir logic is moved into MkdirOperation with callbacks for required FS calls; tests isolate this and verify cost. All within the same audit span. Listing: * the span to use is passed down and cached in the object listing iterator, where it is passed in to the listing callbacks. * the close() of the span is no longer performed when the object listing completes. The complication here is that the delete and rename calls also use the same operations; they don't want their spans closed prematurely. Change-Id: I773e98694c4403857b48ddaa529303498c8d752b HADOOP-15711. Auditing for multipart listing; other failures Find/fix failures triggered by ops happening outside spans, either genuine (multipart, getBucketLocation()) or from test cases which bypass the public entry points. Change-Id: I88aba852161320c9391541522eaba73e4a5f708d Proposed: test suite teardown to reset any span HADOOP-15711. Auditing: redefine AuditSpan lifecycle declare that AuditSpans last until GC'd, moving to making sure that every use deactivates after. Lets me move all of the span knowledge from S3ABlockOutputStream into WriteOperationHelper...should see if the same can be done for other bits. Change-Id: I28627438d2ba9f0a1118f05f3a4a4d7191830db2 HADOOP-15711. Auditing: invocation coverage and enhancement - mkdirs and getFilesStatus are duration; the tracking of that is integrated with the auditing. - StoreContext provides access to the RequestFactory for anything which wants to make AWS requests. -ExecutingStoreOperation extends CallableRaisingIOE. This allows an operation to be passed straight into a closure/executor! Change-Id: Id24c7d3b8025a6d058d0195fea6def45bcfbe9a7 HADOOP-15711. Auditing: S3AFS integrate audit and duration invocations Makes for cleaner code which also measures API duration. Change-Id: I7586b39630972007491d4229d9e28cbc7e39d074 --- .../org/apache/hadoop/fs/shell/Command.java | 3 + .../org/apache/hadoop/fs/shell/PathData.java | 18 +- .../fs/statistics/StoreStatisticNames.java | 7 + .../impl/EmptyIOStatisticsStore.java | 182 +++ .../statistics/impl/IOStatisticsBinding.java | 42 + .../util/functional/RemoteIterators.java | 34 +- .../hadoop/fs/contract/ContractTestUtils.java | 10 +- .../org/apache/hadoop/fs/s3a/Invoker.java | 38 +- .../org/apache/hadoop/fs/s3a/Listing.java | 119 +- .../apache/hadoop/fs/s3a/MultipartUtils.java | 110 +- .../hadoop/fs/s3a/S3ABlockOutputStream.java | 10 + .../apache/hadoop/fs/s3a/S3AFileSystem.java | 1131 ++++++++++------- .../apache/hadoop/fs/s3a/S3AInputStream.java | 45 +- .../hadoop/fs/s3a/S3AReadOpContext.java | 18 +- .../org/apache/hadoop/fs/s3a/S3AUtils.java | 30 +- .../org/apache/hadoop/fs/s3a/Statistic.java | 49 +- .../hadoop/fs/s3a/WriteOperationHelper.java | 288 +++-- .../apache/hadoop/fs/s3a/WriteOperations.java | 18 +- .../hadoop/fs/s3a/api/RequestFactory.java | 282 ++++ .../hadoop/fs/s3a/api/package-info.java | 34 + .../fs/s3a/audit/AWSRequestAnalyzer.java | 256 ++++ .../fs/s3a/audit/AbstractAuditSpanImpl.java | 30 + .../s3a/audit/AbstractOperationAuditor.java | 102 ++ .../fs/s3a/audit/ActiveAuditManager.java | 454 +++++++ .../hadoop/fs/s3a/audit/AuditConstants.java | 108 ++ .../fs/s3a/audit/AuditFailureException.java | 43 + .../hadoop/fs/s3a/audit/AuditIntegration.java | 73 ++ .../fs/s3a/audit/AuditInternalConstants.java | 42 + .../hadoop/fs/s3a/audit/AuditManager.java | 65 + .../apache/hadoop/fs/s3a/audit/AuditSpan.java | 67 + .../fs/s3a/audit/AuditSpanCallbacks.java | 91 ++ .../hadoop/fs/s3a/audit/AuditSpanSource.java | 45 + .../hadoop/fs/s3a/audit/AwsS3Verbs.java | 50 + .../fs/s3a/audit/CommonAuditContext.java | 147 +++ .../fs/s3a/audit/HttpReferrerAuditEntry.java | 336 +++++ .../hadoop/fs/s3a/audit/LoggingAuditor.java | 366 ++++++ .../hadoop/fs/s3a/audit/NoopAuditManager.java | 72 ++ .../hadoop/fs/s3a/audit/NoopAuditor.java | 62 + .../apache/hadoop/fs/s3a/audit/NoopSpan.java | 61 + .../hadoop/fs/s3a/audit/NoopSpanSource.java | 44 + .../hadoop/fs/s3a/audit/OperationAuditor.java | 36 + .../hadoop/fs/s3a/audit/package-info.java | 41 + .../fs/s3a/commit/AbstractS3ACommitter.java | 66 +- .../fs/s3a/commit/CommitOperations.java | 31 +- .../hadoop/fs/s3a/commit/CommitUtils.java | 24 + .../commit/InternalCommitterConstants.java | 6 + .../fs/s3a/commit/MagicCommitIntegration.java | 16 +- .../s3a/commit/magic/MagicCommitTracker.java | 2 +- .../s3a/commit/staging/StagingCommitter.java | 2 + .../fs/s3a/impl/AbstractStoreOperation.java | 40 +- .../hadoop/fs/s3a/impl/CallableSupplier.java | 54 +- .../hadoop/fs/s3a/impl/ContextAccessors.java | 15 + .../hadoop/fs/s3a/impl/DeleteOperation.java | 2 +- .../fs/s3a/impl/ExecutingStoreOperation.java | 38 +- .../hadoop/fs/s3a/impl/HeaderProcessing.java | 18 +- .../hadoop/fs/s3a/impl/InternalConstants.java | 10 + .../s3a/impl/ListingOperationCallbacks.java | 19 +- .../hadoop/fs/s3a/impl/LogExactlyOnce.java | 10 + .../hadoop/fs/s3a/impl/MkdirOperation.java | 184 +++ .../hadoop/fs/s3a/impl/RenameOperation.java | 4 +- .../fs/s3a/impl/RequestFactoryImpl.java | 651 ++++++++++ .../hadoop/fs/s3a/impl/StoreContext.java | 25 +- .../s3a/s3guard/DumpS3GuardDynamoTable.java | 32 +- .../fs/s3a/s3guard/DynamoDBMetadataStore.java | 4 +- .../hadoop/fs/s3a/s3guard/S3GuardTool.java | 22 +- .../hadoop/fs/s3a/select/SelectBinding.java | 14 +- .../hadoop/fs/s3a/tools/MarkerTool.java | 17 +- .../site/markdown/tools/hadoop-aws/audit.png | Bin 0 -> 49705 bytes .../markdown/tools/hadoop-aws/auditing.md | 270 ++++ .../fs/contract/s3a/ITestS3AContractSeek.java | 2 +- .../hadoop/fs/s3a/AbstractS3ATestBase.java | 55 + .../hadoop/fs/s3a/ITestS3AEmptyDirectory.java | 7 +- .../fs/s3a/ITestS3AFailureHandling.java | 13 +- .../hadoop/fs/s3a/ITestS3AMiscOperations.java | 23 +- .../hadoop/fs/s3a/ITestS3AMultipartUtils.java | 4 +- .../fs/s3a/ITestS3ARemoteFileChanged.java | 2 +- .../hadoop/fs/s3a/ITestS3GuardEmptyDirs.java | 44 +- .../hadoop/fs/s3a/MockS3AFileSystem.java | 44 +- .../hadoop/fs/s3a/MultipartTestUtils.java | 57 +- .../apache/hadoop/fs/s3a/S3ATestUtils.java | 46 - .../fs/s3a/TestS3ABlockOutputStream.java | 11 +- .../s3a/audit/TestAuditManagerLifecycle.java | 174 +++ .../fs/s3a/audit/TestAuditOperations.java | 43 + .../hadoop/fs/s3a/auth/ITestAssumeRole.java | 3 +- .../fs/s3a/commit/AbstractCommitITest.java | 12 +- .../s3a/commit/AbstractITCommitProtocol.java | 3 +- .../fs/s3a/commit/ITestCommitOperations.java | 3 +- .../hadoop/fs/s3a/impl/ITestXAttrCost.java | 3 +- .../fs/s3a/impl/TestHeaderProcessing.java | 15 + .../s3a/impl/TestPartialDeleteFailures.java | 15 + .../s3a/performance/AbstractS3ACostTest.java | 56 +- .../ITestDirectoryMarkerListing.java | 13 +- .../fs/s3a/performance/ITestS3AMkdirCost.java | 190 +++ .../fs/s3a/s3guard/ITestS3GuardFsck.java | 2 - .../fs/s3a/s3guard/ITestS3GuardToolLocal.java | 48 +- .../s3a/scale/AbstractSTestS3AHugeFiles.java | 3 +- .../scale/ITestS3ADirectoryPerformance.java | 120 +- .../MinimalListingOperationCallbacks.java | 7 +- .../src/test/resources/core-site.xml | 7 + .../src/test/resources/log4j.properties | 6 + 100 files changed, 6637 insertions(+), 1029 deletions(-) create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/EmptyIOStatisticsStore.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/package-info.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AWSRequestAnalyzer.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AbstractAuditSpanImpl.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AbstractOperationAuditor.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/ActiveAuditManager.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditConstants.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditFailureException.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditIntegration.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditInternalConstants.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditManager.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditSpan.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditSpanCallbacks.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditSpanSource.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AwsS3Verbs.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/CommonAuditContext.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/HttpReferrerAuditEntry.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/LoggingAuditor.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/NoopAuditManager.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/NoopAuditor.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/NoopSpan.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/NoopSpanSource.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/OperationAuditor.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/package-info.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MkdirOperation.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java create mode 100644 hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/audit.png create mode 100644 hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/auditing.md create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditManagerLifecycle.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditOperations.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AMkdirCost.java diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Command.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Command.java index c81825776a613..0bdb47730a929 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Command.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Command.java @@ -38,6 +38,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.hadoop.util.functional.RemoteIterators.cleanupRemoteIterator; + /** * An abstract class for the execution of a file system command */ @@ -361,6 +363,7 @@ protected void processPaths(PathData parent, } } } + cleanupRemoteIterator(itemsIterator); } private void processPathInternal(PathData item) throws IOException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PathData.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PathData.java index dad54ea07bdf1..1ff8d8f0494a1 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PathData.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PathData.java @@ -39,6 +39,8 @@ import org.apache.hadoop.fs.PathNotFoundException; import org.apache.hadoop.fs.RemoteIterator; +import static org.apache.hadoop.util.functional.RemoteIterators.mappingRemoteIterator; + /** * Encapsulates a Path (path), its FileStatus (stat), and its FileSystem (fs). * PathData ensures that the returned path string will be the same as the @@ -287,20 +289,8 @@ public RemoteIterator getDirectoryContentsIterator() throws IOException { checkIfExists(FileTypeRequirement.SHOULD_BE_DIRECTORY); final RemoteIterator stats = this.fs.listStatusIterator(path); - return new RemoteIterator() { - - @Override - public boolean hasNext() throws IOException { - return stats.hasNext(); - } - - @Override - public PathData next() throws IOException { - FileStatus file = stats.next(); - String child = getStringForChildPath(file.getPath()); - return new PathData(fs, child, file); - } - }; + return mappingRemoteIterator(stats, + file -> new PathData(fs, getStringForChildPath(file.getPath()), file)); } /** diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java index ef04feca6917f..5f792a640ee1a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java @@ -155,6 +155,10 @@ public final class StoreStatisticNames { public static final String DELEGATION_TOKENS_ISSUED = "delegation_tokens_issued"; + /** Probe for store existing: {@value}. */ + public static final String STORE_EXISTS_PROBE + = "store_exists_probe"; + /** Requests throttled and retried: {@value}. */ public static final String STORE_IO_THROTTLED = "store_io_throttled"; @@ -349,6 +353,9 @@ public final class StoreStatisticNames { public static final String MULTIPART_UPLOAD_STARTED = "multipart_upload_started"; + public static final String MULTIPART_UPLOAD_LIST + = "multipart_upload_list"; + private StoreStatisticNames() { } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/EmptyIOStatisticsStore.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/EmptyIOStatisticsStore.java new file mode 100644 index 0000000000000..c970546e6dcb8 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/EmptyIOStatisticsStore.java @@ -0,0 +1,182 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.statistics.impl; + +import javax.annotation.Nullable; +import java.time.Duration; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.MeanStatistic; + +import static java.util.Collections.emptyMap; + +/** + * An Empty IOStatisticsStore implementation. + */ +final class EmptyIOStatisticsStore implements IOStatisticsStore { + + /** + * The sole instance of this class. + */ + private static final EmptyIOStatisticsStore INSTANCE = + new EmptyIOStatisticsStore(); + /** + * Get the single instance of this class. + * @return a shared, empty instance. + */ + static IOStatisticsStore getInstance() { + return INSTANCE; + } + + private EmptyIOStatisticsStore() { + } + + @Override + public Map counters() { + return emptyMap(); + } + + @Override + public Map gauges() { + return emptyMap(); + } + + @Override + public Map minimums() { + return emptyMap(); + } + + @Override + public Map maximums() { + return emptyMap(); + } + + @Override + public Map meanStatistics() { + return emptyMap(); + } + + @Override + public boolean aggregate(@Nullable final IOStatistics statistics) { + return false; + } + + @Override + public long incrementCounter(final String key, final long value) { + return 0; + } + + @Override + public void setCounter(final String key, final long value) { + + } + + @Override + public void setGauge(final String key, final long value) { + + } + + @Override + public long incrementGauge(final String key, final long value) { + return 0; + } + + @Override + public void setMaximum(final String key, final long value) { + + } + + @Override + public long incrementMaximum(final String key, final long value) { + return 0; + } + + @Override + public void setMinimum(final String key, final long value) { + + } + + @Override + public long incrementMinimum(final String key, final long value) { + return 0; + } + + @Override + public void addMinimumSample(final String key, final long value) { + + } + + @Override + public void addMaximumSample(final String key, final long value) { + + } + + @Override + public void setMeanStatistic(final String key, final MeanStatistic value) { + + } + + @Override + public void addMeanStatisticSample(final String key, final long value) { + + } + + @Override + public void reset() { + + } + + @Override + public AtomicLong getCounterReference(final String key) { + return null; + } + + @Override + public AtomicLong getMaximumReference(final String key) { + return null; + } + + @Override + public AtomicLong getMinimumReference(final String key) { + return null; + } + + @Override + public AtomicLong getGaugeReference(final String key) { + return null; + } + + @Override + public MeanStatistic getMeanStatistic(final String key) { + return null; + } + + @Override + public void addTimedOperation(final String prefix, + final long durationMillis) { + + } + + @Override + public void addTimedOperation(final String prefix, final Duration duration) { + + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java index c3507dbc73ef4..1aece01e2fe31 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java @@ -28,6 +28,7 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.function.BiFunction; import java.util.function.Function; +import java.util.function.Supplier; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; @@ -97,6 +98,15 @@ public static IOStatistics emptyStatistics() { return EmptyIOStatistics.getInstance(); } + /** + * Get the shared instance of the immutable empty statistics + * store. + * @return an empty statistics object. + */ + public static IOStatisticsStore emptyStatisticsStore() { + return EmptyIOStatisticsStore.getInstance(); + } + /** * Take an IOStatistics instance and wrap it in a source. * @param statistics statistics. @@ -573,6 +583,38 @@ public static Callable trackDurationOfCallable( }; } + /** + * Given a Java supplier, evaluate it while + * tracking the duration of the operation and success/failure. + * @param factory factory of duration trackers + * @param statistic statistic key + * @param input input callable. + * @param return type. + * @return the output of the supplier. + */ + public static B trackDurationOfSupplier( + @Nullable DurationTrackerFactory factory, + String statistic, + Supplier input) { + // create the tracker outside try-with-resources so + // that failures can be set in the catcher. + DurationTracker tracker = createTracker(factory, statistic); + try { + // exec the input function and return its value + return input.get(); + } catch (RuntimeException 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(); + } + } + /** * Create the tracker. If the factory is null, a stub * tracker is returned. diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RemoteIterators.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RemoteIterators.java index 3ac0fced1493d..5fdea4f5b747a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RemoteIterators.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RemoteIterators.java @@ -189,6 +189,7 @@ public static RemoteIterator closingRemoteIterator( /** * Build a list from a RemoteIterator. + * @param source source iterator * @param type * @return a list of the values. * @throws IOException if the source RemoteIterator raises it. @@ -202,12 +203,17 @@ public static List toList(RemoteIterator source) /** * Build an array from a RemoteIterator. + * @param source source iterator + * @param a destination array; if too small a new array + * of the same type is created * @param type * @return an array of the values. * @throws IOException if the source RemoteIterator raises it. */ - public static T[] toArray(RemoteIterator source) throws IOException { - return (T[]) toList(source).toArray(); + public static T[] toArray(RemoteIterator source, + T[] a) throws IOException { + List list = toList(source); + return list.toArray(a); } /** @@ -240,18 +246,28 @@ public static long foreach( consumer.accept(source.next()); } - // maybe log the results - logIOStatisticsAtDebug(LOG, "RemoteIterator Statistics: {}", source); } finally { - if (source instanceof Closeable) { - // source is closeable, so close. - IOUtils.cleanupWithLogger(LOG, (Closeable) source); - } + cleanupRemoteIterator(source); } - return count; } + /** + * Clean up after an iteration. + * If the log is at debug, calculate and log the IOStatistics. + * If the iterator is closeable, cast and then cleanup the iterator + * @param source iterator source + * @param type of source + */ + public static void cleanupRemoteIterator(RemoteIterator source) { + // maybe log the results + logIOStatisticsAtDebug(LOG, "RemoteIterator Statistics: {}", source); + if (source instanceof Closeable) { + /* source is closeable, so close.*/ + IOUtils.cleanupWithLogger(LOG, (Closeable) source); + } + } + /** * A remote iterator from a singleton. It has a single next() * value, after which hasNext() returns false and next() fails. diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java index 35193fa2dc712..e13a49ca10e70 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java @@ -399,9 +399,7 @@ public static boolean rm(FileSystem fileSystem, IOException { if (fileSystem != null) { rejectRootOperation(path, allowRootDelete); - if (fileSystem.exists(path)) { - return fileSystem.delete(path, recursive); - } + return fileSystem.delete(path, recursive); } return false; @@ -728,8 +726,10 @@ public static void assertDeleted(FileSystem fs, assertPathExists(fs, "about to be deleted file", file); } boolean deleted = fs.delete(file, recursive); - String dir = ls(fs, file.getParent()); - assertTrue("Delete failed on " + file + ": " + dir, deleted); + if (!deleted) { + String dir = ls(fs, file.getParent()); + assertTrue("Delete failed on " + file + ": " + dir, deleted); + } assertPathDoesNotExist(fs, "Deleted file", file); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java index 19cd6c985b531..a851f0fbac687 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java @@ -34,6 +34,7 @@ import org.apache.hadoop.io.retry.RetryPolicy; import org.apache.hadoop.util.DurationInfo; import org.apache.hadoop.util.functional.CallableRaisingIOE; +import org.apache.hadoop.util.functional.InvocationRaisingIOE; /** * Class to provide lambda expression invocation of AWS operations. @@ -43,7 +44,8 @@ * the other {@code retry() and retryUntranslated()} calls are wrappers. * * The static {@link #once(String, String, CallableRaisingIOE)} and - * {@link #once(String, String, VoidOperation)} calls take an operation and + * {@link #once(String, String, InvocationRaisingIOE)} calls take an + * operation and * return it with AWS exceptions translated to IOEs of some form. * * The retry logic on a failure is defined by the retry policy passed in @@ -57,7 +59,7 @@ * but before the sleep. * These callbacks can be used for reporting and incrementing statistics. * - * The static {@link #quietly(String, String, VoidOperation)} and + * The static {@link #quietly(String, String, InvocationRaisingIOE)} and * {@link #quietlyEval(String, String, CallableRaisingIOE)} calls exist to * take any operation and quietly catch and log at debug. * The return value of {@link #quietlyEval(String, String, CallableRaisingIOE)} @@ -126,11 +128,11 @@ public static T once(String action, String path, * @throws IOException any IOE raised, or translated exception */ @Retries.OnceTranslated - public static void once(String action, String path, VoidOperation operation) - throws IOException { + public static void once(String action, String path, + InvocationRaisingIOE operation) throws IOException { once(action, path, () -> { - operation.execute(); + operation.apply(); return null; }); } @@ -171,10 +173,10 @@ public static void ignoreIOExceptions( Logger log, String action, String path, - VoidOperation operation) { + InvocationRaisingIOE operation) { ignoreIOExceptions(log, action, path, () -> { - operation.execute(); + operation.apply(); return null; }); } @@ -194,11 +196,11 @@ public void retry(String action, String path, boolean idempotent, Retried retrying, - VoidOperation operation) + InvocationRaisingIOE operation) throws IOException { retry(action, path, idempotent, retrying, () -> { - operation.execute(); + operation.apply(); return null; }); } @@ -221,11 +223,11 @@ public void maybeRetry(boolean doRetry, String path, boolean idempotent, Retried retrying, - VoidOperation operation) + InvocationRaisingIOE operation) throws IOException { maybeRetry(doRetry, action, path, idempotent, retrying, () -> { - operation.execute(); + operation.apply(); return null; }); } @@ -243,7 +245,7 @@ public void maybeRetry(boolean doRetry, public void retry(String action, String path, boolean idempotent, - VoidOperation operation) + InvocationRaisingIOE operation) throws IOException { retry(action, path, idempotent, retryCallback, operation); } @@ -265,7 +267,7 @@ public void maybeRetry( String action, String path, boolean idempotent, - VoidOperation operation) + InvocationRaisingIOE operation) throws IOException { maybeRetry(doRetry, action, path, idempotent, retryCallback, operation); } @@ -475,7 +477,7 @@ public T retryUntranslated( */ public static void quietly(String action, String path, - VoidOperation operation) { + InvocationRaisingIOE operation) { try { once(action, path, operation); } catch (Exception e) { @@ -515,14 +517,6 @@ private static String toDescription(String action, @Nullable String path) { (StringUtils.isNotEmpty(path) ? (" on " + path) : ""); } - /** - * Void operation which may raise an IOException. - */ - @FunctionalInterface - public interface VoidOperation { - void execute() throws IOException; - } - /** * Callback for retry and notification operations. * Even if the interface is throwing up "raw" exceptions, this handler diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java index 3cb3d5d832df3..1a8551246f767 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java @@ -22,6 +22,8 @@ import com.amazonaws.AmazonClientException; import com.amazonaws.services.s3.model.S3ObjectSummary; + +import org.apache.hadoop.fs.s3a.audit.AuditSpan; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.commons.lang3.tuple.Triple; @@ -45,6 +47,7 @@ import org.slf4j.Logger; +import java.io.Closeable; import java.io.FileNotFoundException; import java.io.IOException; import java.time.Instant; @@ -79,6 +82,9 @@ /** * Place for the S3A listing classes; keeps all the small classes under control. + * + * Spans passed in are attached to the listing iterators returned, but are not + * closed at the end of the iteration. This is because the same span */ @InterfaceAudience.Private public class Listing extends AbstractStoreOperation { @@ -137,16 +143,19 @@ public static RemoteIterator toProvidedFileStatusIterator( * @param filter the filter on which paths to accept * @param acceptor the class/predicate to decide which entries to accept * in the listing based on the full file status. + * @param span audit span for this iterator * @return the iterator * @throws IOException IO Problems */ + @Retries.RetryRaw public FileStatusListingIterator createFileStatusListingIterator( Path listPath, S3ListRequest request, PathFilter filter, - Listing.FileStatusAcceptor acceptor) throws IOException { + Listing.FileStatusAcceptor acceptor, + AuditSpan span) throws IOException { return createFileStatusListingIterator(listPath, request, filter, acceptor, - null); + null, span); } /** @@ -159,6 +168,7 @@ public FileStatusListingIterator createFileStatusListingIterator( * in the listing based on the full file status. * @param providedStatus the provided list of file status, which may contain * items that are not listed from source. + * @param span audit span for this iterator * @return the iterator * @throws IOException IO Problems */ @@ -168,9 +178,10 @@ public FileStatusListingIterator createFileStatusListingIterator( S3ListRequest request, PathFilter filter, Listing.FileStatusAcceptor acceptor, - RemoteIterator providedStatus) throws IOException { + RemoteIterator providedStatus, + AuditSpan span) throws IOException { return new FileStatusListingIterator( - createObjectListingIterator(listPath, request), + createObjectListingIterator(listPath, request, span), filter, acceptor, providedStatus); @@ -181,14 +192,16 @@ public FileStatusListingIterator createFileStatusListingIterator( * list object request. * @param listPath path of the listing * @param request initial request to make + * @param span audit span for this iterator * @return the iterator * @throws IOException IO Problems */ @Retries.RetryRaw - public ObjectListingIterator createObjectListingIterator( + private ObjectListingIterator createObjectListingIterator( final Path listPath, - final S3ListRequest request) throws IOException { - return new ObjectListingIterator(listPath, request); + final S3ListRequest request, + final AuditSpan span) throws IOException { + return new ObjectListingIterator(listPath, request, span); } /** @@ -245,6 +258,7 @@ public RemoteIterator createSingleStatusIterator( * @param forceNonAuthoritativeMS forces metadata store to act like non * authoritative. This is useful when * listFiles output is used by import tool. + * @param span audit span for this iterator * @return an iterator over listing. * @throws IOException any exception. */ @@ -252,7 +266,8 @@ public RemoteIterator getListFilesAssumingDir( Path path, boolean recursive, Listing.FileStatusAcceptor acceptor, boolean collectTombstones, - boolean forceNonAuthoritativeMS) throws IOException { + boolean forceNonAuthoritativeMS, + AuditSpan span) throws IOException { String key = maybeAddTrailingSlash(pathToKey(path)); String delimiter = recursive ? null : "/"; @@ -328,7 +343,8 @@ public RemoteIterator getListFilesAssumingDir( .createListObjectsRequest(key, delimiter), ACCEPT_ALL, acceptor, - cachedFilesIterator)), + cachedFilesIterator, + span)), collectTombstones ? tombstones : null); } @@ -337,11 +353,13 @@ public RemoteIterator getListFilesAssumingDir( * Also performing tombstone reconciliation for guarded directories. * @param dir directory to check. * @param filter a path filter. + * @param span audit span for this iterator * @return an iterator that traverses statuses of the given dir. * @throws IOException in case of failure. */ public RemoteIterator getLocatedFileStatusIteratorForDir( - Path dir, PathFilter filter) throws IOException { + Path dir, PathFilter filter, AuditSpan span) throws IOException { + span.activate(); final String key = maybeAddTrailingSlash(pathToKey(dir)); final Listing.FileStatusAcceptor acceptor = new Listing.AcceptAllButSelfAndS3nDirs(dir); @@ -353,39 +371,56 @@ public RemoteIterator getLocatedFileStatusIteratorForDir( listingOperationCallbacks .getUpdatedTtlTimeProvider(), allowAuthoritative); - Set tombstones = meta != null - ? meta.listTombstones() - : null; - final RemoteIterator cachedFileStatusIterator = - createProvidedFileStatusIterator( - S3Guard.dirMetaToStatuses(meta), filter, acceptor); - return (allowAuthoritative && meta != null - && meta.isAuthoritative()) - ? createLocatedFileStatusIterator( - cachedFileStatusIterator) - : createTombstoneReconcilingIterator( + if (meta != null) { + // there's metadata + // convert to an iterator + final RemoteIterator cachedFileStatusIterator = + createProvidedFileStatusIterator( + S3Guard.dirMetaToStatuses(meta), filter, acceptor); + + // if the dir is authoritative and the data considers itself + // to be authorititative. + if (allowAuthoritative && meta.isAuthoritative()) { + // return the list + return createLocatedFileStatusIterator(cachedFileStatusIterator); + } else { + // merge the datasets + return createTombstoneReconcilingIterator( createLocatedFileStatusIterator( - createFileStatusListingIterator(dir, - listingOperationCallbacks - .createListObjectsRequest(key, "/"), - filter, - acceptor, - cachedFileStatusIterator)), - tombstones); + createFileStatusListingIterator(dir, + listingOperationCallbacks + .createListObjectsRequest(key, "/"), + filter, + acceptor, + cachedFileStatusIterator, + span)), + meta.listTombstones()); + } + } else { + // Unguarded + return createLocatedFileStatusIterator( + createFileStatusListingIterator(dir, + listingOperationCallbacks + .createListObjectsRequest(key, "/"), + filter, + acceptor, + span)); + } } /** * Calculate list of file statuses assuming path * to be a non-empty directory. * @param path input path. + * @param span audit span for this iterator + * @return Triple of file statuses, metaData, auth flag. * @throws IOException Any IO problems. */ public Triple, DirListingMetadata, Boolean> - getFileStatusesAssumingNonEmptyDir(Path path) + getFileStatusesAssumingNonEmptyDir(Path path, final AuditSpan span) throws IOException { String key = pathToKey(path); - List result; if (!key.isEmpty()) { key = key + '/'; } @@ -415,7 +450,8 @@ public RemoteIterator getLocatedFileStatusIteratorForDir( path, request, ACCEPT_ALL, - new Listing.AcceptAllButSelfAndS3nDirs(path)); + new Listing.AcceptAllButSelfAndS3nDirs(path), + span); // return the results obtained from s3. return Triple.of( @@ -730,11 +766,13 @@ public String toString() { * Thread safety: none. */ class ObjectListingIterator implements RemoteIterator, - IOStatisticsSource { + IOStatisticsSource, Closeable { /** The path listed. */ private final Path listPath; + private final AuditSpan span; + /** The most recent listing results. */ private S3ListResult objects; @@ -772,12 +810,14 @@ class ObjectListingIterator implements RemoteIterator, * initial set of results/fail if there was a problem talking to the bucket. * @param listPath path of the listing * @param request initial request to make + * @param span audit span for this iterator. * @throws IOException if listObjects raises one. */ @Retries.RetryRaw ObjectListingIterator( Path listPath, - S3ListRequest request) throws IOException { + S3ListRequest request, + AuditSpan span) throws IOException { this.listPath = listPath; this.maxKeys = listingOperationCallbacks.getMaxKeys(); this.request = request; @@ -786,8 +826,9 @@ class ObjectListingIterator implements RemoteIterator, .withDurationTracking(OBJECT_LIST_REQUEST) .withDurationTracking(OBJECT_CONTINUE_LIST_REQUEST) .build(); + this.span = span; this.s3ListResultFuture = listingOperationCallbacks - .listObjectsAsync(request, iostats); + .listObjectsAsync(request, iostats, span); } /** @@ -851,7 +892,7 @@ private void fetchNextBatchAsyncIfPresent() throws IOException { LOG.debug("[{}], Requesting next {} objects under {}", listingCount, maxKeys, listPath); s3ListResultFuture = listingOperationCallbacks - .continueListObjectsAsync(request, objects, iostats); + .continueListObjectsAsync(request, objects, iostats, span); } } @@ -883,6 +924,14 @@ public Path getListPath() { public int getListingCount() { return listingCount; } + + /** + * Close, if actually called, will close the span + * this listing was created with. + */ + @Override + public void close() { + } } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/MultipartUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/MultipartUtils.java index 6eb490f2df482..de7f6705844a3 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/MultipartUtils.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/MultipartUtils.java @@ -31,11 +31,20 @@ import org.slf4j.LoggerFactory; import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.s3a.api.RequestFactory; +import org.apache.hadoop.fs.s3a.audit.AuditSpan; +import org.apache.hadoop.fs.s3a.impl.StoreContext; + +import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_LIST; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfOperation; /** * MultipartUtils upload-specific functions for use by S3AFileSystem and Hadoop * CLI. + * The Audit span active when + * {@link #listMultipartUploads(StoreContext, AmazonS3, String, int)} + * was invoked is retained for all subsequent operations. */ public final class MultipartUtils { @@ -48,33 +57,47 @@ private MultipartUtils() { } /** * List outstanding multipart uploads. * Package private: S3AFileSystem and tests are the users of this. + * + * @param storeContext store context * @param s3 AmazonS3 client to use. - * @param bucketName name of S3 bucket to use. - * @param maxKeys maximum batch size to request at a time from S3. * @param prefix optional key prefix to narrow search. If null then whole * bucket will be searched. + * @param maxKeys maximum batch size to request at a time from S3. * @return an iterator of matching uploads */ - static MultipartUtils.UploadIterator listMultipartUploads(AmazonS3 s3, - Invoker invoker, String bucketName, int maxKeys, @Nullable String prefix) + static MultipartUtils.UploadIterator listMultipartUploads( + final StoreContext storeContext, + AmazonS3 s3, + @Nullable String prefix, + int maxKeys) throws IOException { - return new MultipartUtils.UploadIterator(s3, invoker, bucketName, maxKeys, + return new MultipartUtils.UploadIterator(storeContext, + s3, + maxKeys, prefix); } /** * Simple RemoteIterator wrapper for AWS `listMultipartUpload` API. * Iterates over batches of multipart upload metadata listings. + * All requests are in the StoreContext's active span + * at the time the iterator was constructed. */ static class ListingIterator implements RemoteIterator { - private final String bucketName; private final String prefix; + + private final RequestFactory requestFactory; + private final int maxKeys; private final AmazonS3 s3; private final Invoker invoker; + private final AuditSpan auditSpan; + + private final StoreContext storeContext; + /** * Most recent listing results. */ @@ -85,16 +108,24 @@ static class ListingIterator implements */ private boolean firstListing = true; - private int listCount = 1; + /** + * Count of list calls made. + */ + private int listCount = 0; - ListingIterator(AmazonS3 s3, Invoker invoker, String bucketName, - int maxKeys, @Nullable String prefix) throws IOException { + ListingIterator(final StoreContext storeContext, + AmazonS3 s3, + @Nullable String prefix, + int maxKeys) throws IOException { + this.storeContext = storeContext; this.s3 = s3; - this.bucketName = bucketName; + this.requestFactory = storeContext.getRequestFactory(); this.maxKeys = maxKeys; this.prefix = prefix; - this.invoker = invoker; + this.invoker = storeContext.getInvoker(); + this.auditSpan = storeContext.getActiveAuditSpan(); + // request the first listing. requestNextBatch(); } @@ -138,31 +169,36 @@ public MultipartUploadListing next() throws IOException { @Override public String toString() { - return "Upload iterator: prefix " + prefix + "; list count " + - listCount + "; isTruncated=" + listing.isTruncated(); + return "Upload iterator: prefix " + prefix + + "; list count " + listCount + + "; upload count " + listing.getMultipartUploads().size() + + "; isTruncated=" + listing.isTruncated(); } @Retries.RetryTranslated private void requestNextBatch() throws IOException { - ListMultipartUploadsRequest req = - new ListMultipartUploadsRequest(bucketName); - if (prefix != null) { - req.setPrefix(prefix); - } - if (!firstListing) { - req.setKeyMarker(listing.getNextKeyMarker()); - req.setUploadIdMarker(listing.getNextUploadIdMarker()); - } - req.setMaxUploads(listCount); + try (final AuditSpan span = auditSpan.activate()) { + ListMultipartUploadsRequest req = requestFactory + .newListMultipartUploadsRequest(prefix); + if (!firstListing) { + req.setKeyMarker(listing.getNextKeyMarker()); + req.setUploadIdMarker(listing.getNextUploadIdMarker()); + } + req.setMaxUploads(maxKeys); - LOG.debug("[{}], Requesting next {} uploads prefix {}, " + - "next key {}, next upload id {}", listCount, maxKeys, prefix, - req.getKeyMarker(), req.getUploadIdMarker()); - listCount++; + LOG.debug("[{}], Requesting next {} uploads prefix {}, " + + "next key {}, next upload id {}", listCount, maxKeys, prefix, + req.getKeyMarker(), req.getUploadIdMarker()); + listCount++; - listing = invoker.retry("listMultipartUploads", prefix, true, - () -> s3.listMultipartUploads(req)); - LOG.debug("New listing state: {}", this); + listing = invoker.retry("listMultipartUploads", prefix, true, + trackDurationOfOperation(storeContext.getInstrumentation(), + MULTIPART_UPLOAD_LIST.getSymbol(), + () -> s3.listMultipartUploads(req))); + LOG.debug("Listing found {} upload(s)", + listing.getMultipartUploads().size()); + LOG.debug("New listing state: {}", this); + } } } @@ -174,6 +210,10 @@ private void requestNextBatch() throws IOException { public static class UploadIterator implements RemoteIterator { + /** + * Iterator for issuing new upload list requests from + * where the previous one ended. + */ private ListingIterator lister; /** Current listing: the last upload listing we fetched. */ private MultipartUploadListing listing; @@ -181,11 +221,15 @@ public static class UploadIterator private ListIterator batchIterator; @Retries.RetryTranslated - public UploadIterator(AmazonS3 s3, Invoker invoker, String bucketName, - int maxKeys, @Nullable String prefix) + public UploadIterator( + final StoreContext storeContext, + AmazonS3 s3, + int maxKeys, + @Nullable String prefix) throws IOException { - lister = new ListingIterator(s3, invoker, bucketName, maxKeys, prefix); + lister = new ListingIterator(storeContext, s3, prefix, + maxKeys); requestNextBatch(); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java index 4f06981bc2d45..1bc67a481bde0 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java @@ -31,6 +31,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import com.amazonaws.SdkBaseException; import com.amazonaws.event.ProgressEvent; import com.amazonaws.event.ProgressEventType; import com.amazonaws.event.ProgressListener; @@ -135,6 +136,8 @@ class S3ABlockOutputStream extends OutputStream implements /** * Write operation helper; encapsulation of the filesystem operations. + * This contains the audit span for the operation, and activates/deactivates + * it within calls. */ private final WriteOperationHelper writeOperationHelper; @@ -408,6 +411,7 @@ public void close() throws IOException { final List partETags = multiPartUpload.waitForAllPartUploads(); bytes = bytesSubmitted; + // then complete the operation if (putTracker.aboutToComplete(multiPartUpload.getUploadId(), partETags, @@ -773,6 +777,12 @@ private void uploadBlockAsync(final S3ADataBlocks.DataBlock block) uploadData.getUploadStream(), uploadData.getFile(), 0L); + } catch (SdkBaseException aws) { + // catch and translate + IOException e = translateException("upload", key, aws); + // failure to start the upload. + noteUploadFailure(e); + throw e; } catch (IOException e) { // failure to start the upload. noteUploadFailure(e); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index 8db5d51def84e..763f2548c13af 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -21,7 +21,6 @@ import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; -import java.io.InputStream; import java.io.InterruptedIOException; import java.net.URI; import java.nio.file.AccessDeniedException; @@ -51,14 +50,15 @@ import com.amazonaws.AmazonClientException; import com.amazonaws.AmazonServiceException; +import com.amazonaws.AmazonWebServiceRequest; import com.amazonaws.SdkBaseException; import com.amazonaws.services.s3.AmazonS3; -import com.amazonaws.services.s3.model.AbortMultipartUploadRequest; import com.amazonaws.services.s3.model.CannedAccessControlList; import com.amazonaws.services.s3.model.CopyObjectRequest; import com.amazonaws.services.s3.model.DeleteObjectsRequest; import com.amazonaws.services.s3.model.DeleteObjectsResult; import com.amazonaws.services.s3.model.GetObjectMetadataRequest; +import com.amazonaws.services.s3.model.GetObjectRequest; import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest; import com.amazonaws.services.s3.model.InitiateMultipartUploadResult; import com.amazonaws.services.s3.model.ListMultipartUploadsRequest; @@ -69,6 +69,7 @@ import com.amazonaws.services.s3.model.ObjectMetadata; import com.amazonaws.services.s3.model.PutObjectRequest; import com.amazonaws.services.s3.model.PutObjectResult; +import com.amazonaws.services.s3.model.S3Object; import com.amazonaws.services.s3.model.SSEAwsKeyManagementParams; import com.amazonaws.services.s3.model.SSECustomerKey; import com.amazonaws.services.s3.model.UploadPartRequest; @@ -96,6 +97,11 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Globber; import org.apache.hadoop.fs.impl.OpenFileParameters; +import org.apache.hadoop.fs.s3a.api.RequestFactory; +import org.apache.hadoop.fs.s3a.audit.AuditManager; +import org.apache.hadoop.fs.s3a.audit.AuditSpan; +import org.apache.hadoop.fs.s3a.audit.AuditSpanSource; +import org.apache.hadoop.fs.s3a.audit.ActiveAuditManager; import org.apache.hadoop.fs.s3a.auth.SignerManager; import org.apache.hadoop.fs.s3a.auth.delegation.DelegationOperations; import org.apache.hadoop.fs.s3a.auth.delegation.DelegationTokenProvider; @@ -109,9 +115,11 @@ import org.apache.hadoop.fs.s3a.impl.HeaderProcessing; import org.apache.hadoop.fs.s3a.impl.InternalConstants; import org.apache.hadoop.fs.s3a.impl.ListingOperationCallbacks; +import org.apache.hadoop.fs.s3a.impl.MkdirOperation; import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport; import org.apache.hadoop.fs.s3a.impl.OperationCallbacks; import org.apache.hadoop.fs.s3a.impl.RenameOperation; +import org.apache.hadoop.fs.s3a.impl.RequestFactoryImpl; import org.apache.hadoop.fs.s3a.impl.S3AMultipartUploaderBuilder; import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum; import org.apache.hadoop.fs.s3a.impl.StoreContext; @@ -146,7 +154,6 @@ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.s3a.auth.RoleModel; import org.apache.hadoop.fs.s3a.auth.delegation.AWSPolicyProvider; -import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecretOperations; import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets; import org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens; import org.apache.hadoop.fs.s3a.auth.delegation.AbstractS3ATokenIdentifier; @@ -175,6 +182,7 @@ import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.SemaphoredDelegatingExecutor; import org.apache.hadoop.util.concurrent.HadoopExecutors; +import org.apache.hadoop.util.functional.CallableRaisingIOE; import static java.util.Objects.requireNonNull; import static org.apache.hadoop.fs.impl.AbstractFSBuilderImpl.rejectUnknownMandatoryKeys; @@ -184,7 +192,7 @@ import static org.apache.hadoop.fs.s3a.Listing.toLocatedFileStatusIterator; import static org.apache.hadoop.fs.s3a.S3AUtils.*; import static org.apache.hadoop.fs.s3a.Statistic.*; -import static org.apache.commons.lang3.StringUtils.isNotEmpty; +import static org.apache.hadoop.fs.s3a.audit.AuditConstants.INITIALIZE_SPAN; import static org.apache.hadoop.fs.s3a.auth.RolePolicies.STATEMENT_ALLOW_SSE_KMS_RW; import static org.apache.hadoop.fs.s3a.auth.RolePolicies.allowS3Operations; import static org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens.TokenIssuingPolicy.NoTokensAvailable; @@ -195,15 +203,20 @@ import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletionIgnoringExceptions; import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isObjectNotFound; import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isUnknownBucket; +import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DEFAULT_UPLOAD_PART_COUNT_LIMIT; +import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DELETE_CONSIDERED_IDEMPOTENT; import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_404; +import static org.apache.hadoop.fs.s3a.impl.InternalConstants.UPLOAD_PART_COUNT_LIMIT; import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.fixBucketRegion; import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.logDnsLookup; import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.dirMetaToStatuses; import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_CONTINUE_LIST_REQUEST; import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_LIST_REQUEST; import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.pairedTrackerFactory; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration; 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.fs.statistics.impl.IOStatisticsBinding.trackDurationOfSupplier; import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; import static org.apache.hadoop.util.functional.RemoteIterators.typeCastingRemoteIterator; @@ -223,22 +236,14 @@ @InterfaceAudience.Private @InterfaceStability.Evolving public class S3AFileSystem extends FileSystem implements StreamCapabilities, - AWSPolicyProvider, DelegationTokenProvider, IOStatisticsSource { + AWSPolicyProvider, DelegationTokenProvider, IOStatisticsSource, + AuditSpanSource { + /** * Default blocksize as used in blocksize and FS status queries. */ public static final int DEFAULT_BLOCKSIZE = 32 * 1024 * 1024; - /** - * This declared delete as idempotent. - * This is an "interesting" topic in past Hadoop FS work. - * Essentially: with a single caller, DELETE is idempotent - * but in a shared filesystem, it is is very much not so. - * Here, on the basis that isn't a filesystem with consistency guarantees, - * retryable results in files being deleted. - */ - public static final boolean DELETE_CONSIDERED_IDEMPOTENT = true; - private URI uri; private Path workingDir; private String username; @@ -338,6 +343,21 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, */ private final ContextAccessors contextAccessors = new ContextAccessorsImpl(); + /** + * Factory for AWS requests. + */ + private RequestFactory requestFactory; + + /** + * Audit manager (service lifecycle). + * Creates the audit service and manages the binding of different audit spans + * to different threads. + * Initially this is a no-op manager; once the service is initialized it will + * be replaced with a configured one. + */ + private AuditManager auditManager = + ActiveAuditManager.stubAuditManager(); + /** Add any deprecated keys. */ @SuppressWarnings("deprecation") private static void addDeprecatedKeys() { @@ -368,6 +388,7 @@ public void initialize(URI name, Configuration originalConf) throws IOException { // get the host; this is guaranteed to be non-null, non-empty bucket = name.getHost(); + AuditSpan span = null; try { LOG.debug("Initializing S3AFileSystem for {}", bucket); // clone the configuration into one with propagated bucket options @@ -408,8 +429,6 @@ public void initialize(URI name, Configuration originalConf) s3guardInvoker = new Invoker(new S3GuardExistsRetryPolicy(getConf()), onRetry); - writeHelper = new WriteOperationHelper(this, getConf(), - statisticsContext); failOnMetadataWriteError = conf.getBoolean(FAIL_ON_METADATA_WRITE_ERROR, FAIL_ON_METADATA_WRITE_ERROR_DEFAULT); @@ -439,6 +458,21 @@ public void initialize(URI name, Configuration originalConf) signerManager = new SignerManager(bucket, this, conf, owner); signerManager.initCustomSigners(); + // start auditing + initializeAuditService(); + + // create the requestFactory. + // requires the audit manager to be initialized. + requestFactory = createRequestFactory(); + + // create the static write operation helper. + // this doesn't have a short-lived span; auditors which + // require one may reject usages. + writeHelper = createWriteOperationHelper(getActiveAuditSpan()); + + // create an initial span for all other operations. + span = createSpan(INITIALIZE_SPAN, bucket, null); + // creates the AWS client, including overriding auth chain if // the FS came with a DT // this may do some patching of the configuration (e.g. setting @@ -516,10 +550,12 @@ public void initialize(URI name, Configuration originalConf) listing = new Listing(listingOperationCallbacks, createStoreContext()); } catch (AmazonClientException e) { // amazon client exception: stop all services then throw the translation + cleanupWithLogger(LOG, span); stopAllServices(); throw translateException("initializing ", new Path(name), e); } catch (IOException | RuntimeException e) { // other exceptions: stop the services. + cleanupWithLogger(LOG, span); stopAllServices(); throw e; } @@ -599,6 +635,7 @@ public Statistics getInstanceStatistics() { * @param conf configuration. */ private void initThreadPools(Configuration conf) { + final String name = "s3a-transfer-" + getBucket(); int maxThreads = conf.getInt(MAX_THREADS, DEFAULT_MAX_THREADS); if (maxThreads < 2) { LOG.warn(MAX_THREADS + " must be at least 2: forcing to 2."); @@ -612,13 +649,13 @@ private void initThreadPools(Configuration conf) { maxThreads, maxThreads + totalTasks, keepAliveTime, TimeUnit.SECONDS, - "s3a-transfer-shared"); + name + "-bounded"); unboundedThreadPool = new ThreadPoolExecutor( maxThreads, Integer.MAX_VALUE, keepAliveTime, TimeUnit.SECONDS, new LinkedBlockingQueue<>(), BlockingThreadPoolExecutorService.newDaemonThreadFactory( - "s3a-transfer-unbounded")); + name + "-unbounded")); unboundedThreadPool.allowCoreThreadTimeOut(true); executorCapacity = intOption(conf, EXECUTOR_CAPACITY, DEFAULT_EXECUTOR_CAPACITY, 1); @@ -648,7 +685,9 @@ protected static S3AStorageStatistics createStorageStatistics( protected void verifyBucketExists() throws UnknownStoreException, IOException { if (!invoker.retry("doesBucketExist", bucket, true, - () -> s3.doesBucketExist(bucket))) { + trackDurationOfOperation(getDurationTrackerFactory(), + STORE_EXISTS_PROBE.getSymbol(), + () -> s3.doesBucketExist(bucket)))) { throw new UnknownStoreException("Bucket " + bucket + " does not exist"); } } @@ -664,7 +703,9 @@ protected void verifyBucketExists() protected void verifyBucketExistsV2() throws UnknownStoreException, IOException { if (!invoker.retry("doesBucketExistV2", bucket, true, - () -> s3.doesBucketExistV2(bucket))) { + trackDurationOfOperation(getDurationTrackerFactory(), + STORE_EXISTS_PROBE.getSymbol(), + () -> s3.doesBucketExistV2(bucket)))) { throw new UnknownStoreException("Bucket " + bucket + " does not exist"); } } @@ -754,6 +795,109 @@ private void bindAWSClient(URI name, boolean dtEnabled) throws IOException { parameters); } + /** + * Initialize and launch the audit manager and service. + * As this takes the FS IOStatistics store, it must be invoked + * after instrumentation is initialized. + * @throws IOException failure to instantiate/initialize. + */ + protected void initializeAuditService() throws IOException { + auditManager = ActiveAuditManager.createAuditManager( + getConf(), + instrumentation.getIOStatistics()); + } + + /** + * The audit manager. + * @return the audit manager + */ + protected AuditManager getAuditManager() { + return auditManager; + } + + /** + * Get the active audit span. + * @return the span. + */ + @InterfaceAudience.Private + public AuditSpan getActiveAuditSpan() { + return getAuditManager().getActiveThreadSpan(); + } + + /** + * Preflight preparation of AWS request. + * @param type + * @return prepared entry. + */ + @InterfaceAudience.Private + @Retries.OnceRaw + private T prepareRequest(T t) { + return getActiveAuditSpan().beforeExecution(t); + } + + /** + * Get the audit span source; allows for components like the committers + * to have a source of spans without being hard coded to the FS only. + * @return the source of spans -base implementation is this instance. + */ + @InterfaceAudience.Private + public AuditSpanSource getAuditSpanSource() { + return this; + } + + /** + * Start an operation; this informs the audit service of the event + * and then sets it as the active span. + * @param name operation name. + * @param path1 first path of operation + * @param path2 second path of operation + * @return a span for the audit + * @throws IOException failure + */ + public AuditSpan createSpan(String name, + @Nullable String path1, + @Nullable String path2) + throws IOException { + + return getAuditManager().createSpan(name, path1, path2); + } + + /** + * Build the request factory. + * MUST be called after reading encryption secrets from settings/ + * delegation token. + * Protected, in case test/mock classes want to implement their + * own variants. + * @return request factory. + */ + protected RequestFactory createRequestFactory() { + long partCountLimit = longOption(getConf(), + UPLOAD_PART_COUNT_LIMIT, + DEFAULT_UPLOAD_PART_COUNT_LIMIT, + 1); + if (partCountLimit != DEFAULT_UPLOAD_PART_COUNT_LIMIT) { + LOG.warn("Configuration property {} shouldn't be overridden by client", + UPLOAD_PART_COUNT_LIMIT); + } + + return RequestFactoryImpl.builder() + .withBucket(requireNonNull(bucket)) + .withCannedACL(getCannedACL()) + .withEncryptionSecrets(requireNonNull(encryptionSecrets)) + .withMultipartPartCountLimit(partCountLimit) + .withRequestPreparer(getAuditManager()::requestCreated) + .build(); + } + + /** + * Get the request factory which uses this store's audit span. + * @return the request factory. + */ + @VisibleForTesting + public RequestFactory getRequestFactory() { + return requestFactory; + } + /** * Implementation of all operations used by delegation tokens. */ @@ -780,6 +924,9 @@ public DelegationOperations createDelegationOperations() { */ protected void setEncryptionSecrets(final EncryptionSecrets secrets) { this.encryptionSecrets = secrets; + if (requestFactory != null) { + requestFactory.setEncryptionSecrets(secrets); + } } /** @@ -933,11 +1080,13 @@ protected void setAmazonS3Client(AmazonS3 client) { /** * Get the region of a bucket. + * Invoked from StoreContext; consider an entry point. * @return the region in which a bucket is located * @throws AccessDeniedException if the caller lacks permission. * @throws IOException on any failure. */ @Retries.RetryTranslated + @InterfaceAudience.LimitedPrivate("diagnostics") public String getBucketLocation() throws IOException { return getBucketLocation(bucket); } @@ -956,8 +1105,10 @@ public String getBucketLocation() throws IOException { @VisibleForTesting @Retries.RetryTranslated public String getBucketLocation(String bucketName) throws IOException { - final String region = invoker.retry("getBucketLocation()", bucketName, true, - () -> s3.getBucketLocation(bucketName)); + final String region = trackDurationAndSpan( + STORE_EXISTS_PROBE, bucketName, null, () -> + invoker.retry("getBucketLocation()", bucketName, true, () -> + s3.getBucketLocation(bucketName))); return fixBucketRegion(region); } @@ -1195,8 +1346,9 @@ private FSDataInputStream open( final Optional providedStatus) throws IOException { - entryPoint(INVOCATION_OPEN); final Path path = qualify(file); + // this span is passed into the stream. + final AuditSpan auditSpan = entryPoint(INVOCATION_OPEN, path); S3AFileStatus fileStatus = extractOrFetchSimpleFileStatus(path, providedStatus); @@ -1214,13 +1366,15 @@ private FSDataInputStream open( fileStatus, policy, changeDetectionPolicy, - readAheadRange2); + readAheadRange2, + auditSpan); } else { readContext = createReadContext( fileStatus, inputPolicy, changeDetectionPolicy, - readAhead); + readAhead, + auditSpan); } LOG.debug("Opening '{}'", readContext); @@ -1228,7 +1382,60 @@ private FSDataInputStream open( new S3AInputStream( readContext, createObjectAttributes(fileStatus), - s3)); + createInputStreamCallbacks(auditSpan))); + } + + /** + * Overrride point: create the callbacks for S3AInputStream. + * @return an implementation of the InputStreamCallbacks, + */ + private S3AInputStream.InputStreamCallbacks createInputStreamCallbacks( + final AuditSpan auditSpan) { + return new InputStreamCallbacksImpl(auditSpan); + } + + /** + * Operations needed by S3AInputStream to read data. + */ + private final class InputStreamCallbacksImpl implements + S3AInputStream.InputStreamCallbacks { + + /** + * Audit span to activate before each call. + */ + private final AuditSpan auditSpan; + + /** + * Create. + * @param auditSpan Audit span to activate before each call. + */ + private InputStreamCallbacksImpl(final AuditSpan auditSpan) { + this.auditSpan = requireNonNull(auditSpan); + } + + /** + * Closes the audit span. + */ + @Override + public void close() { + auditSpan.close(); + } + + @Override + public GetObjectRequest newGetRequest(final String key) { + // active the audit span used for the operation + try (AuditSpan span = auditSpan.activate()) { + return getRequestFactory().newGetObjectRequest(key); + } + } + + @Override + public S3Object getObject(GetObjectRequest request) { + // active the audit span used for the operation + try (AuditSpan span = auditSpan.activate()) { + return s3.getObject(request); + } + } } /** @@ -1243,7 +1450,8 @@ private S3AReadOpContext createReadContext( final FileStatus fileStatus, final S3AInputPolicy seekPolicy, final ChangeDetectionPolicy changePolicy, - final long readAheadRange) { + final long readAheadRange, + final AuditSpan auditSpan) { return new S3AReadOpContext(fileStatus.getPath(), hasMetadataStore(), invoker, @@ -1253,7 +1461,8 @@ private S3AReadOpContext createReadContext( fileStatus, seekPolicy, changePolicy, - readAheadRange); + readAheadRange, + auditSpan); } /** @@ -1314,8 +1523,38 @@ private S3ObjectAttributes createObjectAttributes( public FSDataOutputStream create(Path f, FsPermission permission, boolean overwrite, int bufferSize, short replication, long blockSize, Progressable progress) throws IOException { - entryPoint(INVOCATION_CREATE); final Path path = qualify(f); + // the span will be picked up inside the output stream + return trackDurationAndSpan(INVOCATION_CREATE, path, () -> + innerCreateFile(path, permission, overwrite, bufferSize, replication, + blockSize, progress)); + } + + /** + * Create an FSDataOutputStream at the indicated Path with write-progress + * reporting; in the active span. + * Retry policy: retrying, translated on the getFileStatus() probe. + * No data is uploaded to S3 in this call, so no retry issues related to that. + * @param path the file name to open + * @param permission the permission to set. + * @param overwrite if a file with this name already exists, then if true, + * the file will be overwritten, and if false an error will be thrown. + * @param bufferSize the size of the buffer to be used. + * @param replication required block replication for the file. + * @param blockSize the requested block size. + * @param progress the progress reporter. + * @throws IOException in the event of IO related errors. + * @see #setPermission(Path, FsPermission) + */ + @SuppressWarnings("IOResourceOpenedButNotSafelyClosed") + @Retries.RetryTranslated + private FSDataOutputStream innerCreateFile(Path path, + FsPermission permission, + boolean overwrite, + int bufferSize, + short replication, + long blockSize, + Progressable progress) throws IOException { String key = pathToKey(path); FileStatus status = null; try { @@ -1360,13 +1599,15 @@ public FSDataOutputStream create(Path f, FsPermission permission, partSize, blockFactory, outputStreamStatistics, - getWriteOperationHelper(), + createWriteOperationHelper(getActiveAuditSpan()), putTracker), null); } /** - * Get a {@code WriteOperationHelper} instance. + * Create a Write Operation Helper with the current active span. + * All operations made through this helper will activate the + * span before execution. * * This class permits other low-level operations against the store. * It is unstable and @@ -1376,7 +1617,23 @@ public FSDataOutputStream create(Path f, FsPermission permission, */ @InterfaceAudience.Private public WriteOperationHelper getWriteOperationHelper() { - return writeHelper; + return createWriteOperationHelper(getActiveAuditSpan()); + } + + /** + * Create a Write Operation Helper with the given span. + * All operations made through this helper will activate the + * span before execution. + * @param auditSpan audit span + * @return a new helper. + */ + @InterfaceAudience.Private + public WriteOperationHelper createWriteOperationHelper(AuditSpan auditSpan) { + return new WriteOperationHelper(this, + getConf(), + statisticsContext, + getAuditSpanSource(), + auditSpan); } /** @@ -1392,29 +1649,31 @@ public FSDataOutputStream createNonRecursive(Path p, short replication, long blockSize, Progressable progress) throws IOException { - entryPoint(INVOCATION_CREATE_NON_RECURSIVE); final Path path = makeQualified(p); - Path parent = path.getParent(); - // expect this to raise an exception if there is no parent dir - if (parent != null && !parent.isRoot()) { - S3AFileStatus status; - try { - // optimize for the directory existing: Call list first - status = innerGetFileStatus(parent, false, - StatusProbeEnum.DIRECTORIES); - } catch (FileNotFoundException e) { - // no dir, fall back to looking for a file - // (failure condition if true) - status = innerGetFileStatus(parent, false, - StatusProbeEnum.HEAD_ONLY); - } - if (!status.isDirectory()) { - throw new FileAlreadyExistsException("Not a directory: " + parent); + // this span is passed into the stream. + try (AuditSpan span = entryPoint(INVOCATION_CREATE_NON_RECURSIVE, path)) { + Path parent = path.getParent(); + // expect this to raise an exception if there is no parent dir + if (parent != null && !parent.isRoot()) { + S3AFileStatus status; + try { + // optimize for the directory existing: Call list first + status = innerGetFileStatus(parent, false, + StatusProbeEnum.DIRECTORIES); + } catch (FileNotFoundException e) { + // no dir, fall back to looking for a file + // (failure condition if true) + status = innerGetFileStatus(parent, false, + StatusProbeEnum.HEAD_ONLY); + } + if (!status.isDirectory()) { + throw new FileAlreadyExistsException("Not a directory: " + parent); + } } + return innerCreateFile(path, permission, + flags.contains(CreateFlag.OVERWRITE), bufferSize, + replication, blockSize, progress); } - return create(path, permission, - flags.contains(CreateFlag.OVERWRITE), bufferSize, - replication, blockSize, progress); } /** @@ -1454,9 +1713,10 @@ public FSDataOutputStream append(Path f, int bufferSize, */ @Retries.RetryTranslated public boolean rename(Path src, Path dst) throws IOException { - try (DurationInfo ignored = new DurationInfo(LOG, false, - "rename(%s, %s", src, dst)) { - long bytesCopied = innerRename(src, dst); + try { + long bytesCopied = trackDurationAndSpan( + INVOCATION_RENAME, src.toString(), dst.toString(), () -> + innerRename(src, dst)); LOG.debug("Copied {} bytes", bytesCopied); return true; } catch (AmazonClientException e) { @@ -1594,7 +1854,6 @@ private long innerRename(Path source, Path dest) Path dst = qualify(dest); LOG.debug("Rename path {} to {}", src, dst); - entryPoint(INVOCATION_RENAME); String srcKey = pathToKey(src); String dstKey = pathToKey(dst); @@ -1644,7 +1903,8 @@ public S3ObjectAttributes createObjectAttributes( public S3AReadOpContext createReadContext(final FileStatus fileStatus) { return S3AFileSystem.this.createReadContext(fileStatus, inputPolicy, - changeDetectionPolicy, readAhead); + changeDetectionPolicy, readAhead, + getAuditManager().getActiveThreadSpan()); } @Override @@ -1727,10 +1987,14 @@ public RemoteIterator listObjects( createListObjectsRequest(key, null), ACCEPT_ALL, Listing.ACCEPT_ALL_BUT_S3N, - null)); + null, + getAuditManager().getActiveThreadSpan())); } } + /** + * Callbacks from {@link Listing}. + */ protected class ListingOperationCallbacksImpl implements ListingOperationCallbacks { @@ -1738,9 +2002,10 @@ protected class ListingOperationCallbacksImpl implements @Retries.RetryRaw public CompletableFuture listObjectsAsync( S3ListRequest request, - DurationTrackerFactory trackerFactory) + DurationTrackerFactory trackerFactory, + AuditSpan span) throws IOException { - return submit(unboundedThreadPool, () -> + return submit(unboundedThreadPool, span, () -> listObjects(request, pairedTrackerFactory(trackerFactory, getDurationTrackerFactory()))); @@ -1751,9 +2016,10 @@ public CompletableFuture listObjectsAsync( public CompletableFuture continueListObjectsAsync( S3ListRequest request, S3ListResult prevResult, - DurationTrackerFactory trackerFactory) + DurationTrackerFactory trackerFactory, + AuditSpan span) throws IOException { - return submit(unboundedThreadPool, + return submit(unboundedThreadPool, span, () -> continueListObjects(request, prevResult, pairedTrackerFactory(trackerFactory, getDurationTrackerFactory()))); @@ -1796,15 +2062,20 @@ public boolean allowAuthoritative(final Path p) { /** * Low-level call to get at the object metadata. + * This method is used in some external applications and so + * must be viewed as a public entry point. * @param path path to the object. This will be qualified. * @return metadata * @throws IOException IO and object access problems. */ @VisibleForTesting + @InterfaceAudience.LimitedPrivate("utilities") @Retries.RetryTranslated + @InterfaceStability.Evolving public ObjectMetadata getObjectMetadata(Path path) throws IOException { - return getObjectMetadata(makeQualified(path), null, invoker, - "getObjectMetadata"); + return trackDurationAndSpan(INVOCATION_GET_FILE_STATUS, path, () -> + getObjectMetadata(makeQualified(path), null, invoker, + "getObjectMetadata")); } /** @@ -1820,7 +2091,6 @@ public ObjectMetadata getObjectMetadata(Path path) throws IOException { private ObjectMetadata getObjectMetadata(Path path, ChangeTracker changeTracker, Invoker changeInvoker, String operation) throws IOException { - checkNotClosed(); String key = pathToKey(path); return once(operation, path.toString(), () -> @@ -1868,12 +2138,74 @@ void setMetadataStore(MetadataStore ms) { /** * Entry point to an operation. * Increments the statistic; verifies the FS is active. - * @param operation The operation to increment - * @throws IOException if the + * @param operation The operation being invoked + * @param path1 first path of operation + * @return a span for the audit + * @throws IOException failure of audit service */ - protected void entryPoint(Statistic operation) throws IOException { + protected AuditSpan entryPoint(Statistic operation, + Path path1) throws IOException { + return entryPoint(operation, path1.toString(), null); + } + + /** + * Entry point to an operation. + * Increments the statistic; verifies the FS is active. + * @param operation The operation being invoked + * @param path1 first path of operation + * @param path2 second path of operation + * @return a span for the audit + * @throws IOException failure of audit service + */ + protected AuditSpan entryPoint(Statistic operation, + @Nullable String path1, + @Nullable String path2) throws IOException { checkNotClosed(); incrementStatistic(operation); + return createSpan(operation.getSymbol(), + path1, path2); + } + + /** + * Given an IOException raising callable/lambda expression, + * execute it and update the relevant statistic within a span + * of the same statistic. + * @param statistic statistic key + * @param path first path for span (nullable) + * @param path2 second path for span + * @param input input callable. + * @param return type. + * @return the result of the operation. + * @throws IOException if raised in the callable + */ + private B trackDurationAndSpan( + Statistic statistic, String path, String path2, + CallableRaisingIOE input) throws IOException { + checkNotClosed(); + try (AuditSpan span = createSpan(statistic.getSymbol(), + path, path2)) { + return trackDuration(getDurationTrackerFactory(), + statistic.getSymbol(), input); + } + } + + /** + * Overloaded version of + * {@link #trackDurationAndSpan(Statistic, String, String, CallableRaisingIOE)}. + * Takes a single nullable path as the path param, + * @param statistic statistic key + * @param path path for span (nullable) + * @param input input callable. + * @param return type. + * @return the result of the operation. + * @throws IOException if raised in the callable + */ + private B trackDurationAndSpan( + Statistic statistic, Path path, CallableRaisingIOE input) + throws IOException { + return trackDurationAndSpan(statistic, + path != null ? path.toString(): null, + null, input); } /** @@ -2016,12 +2348,17 @@ protected DurationTrackerFactory getDurationTrackerFactory() { /** * Request object metadata; increments counters in the process. * Retry policy: retry untranslated. + * This method is used in some external applications and so + * must be viewed as a public entry point. + * It does NOT initiate a new AuditSpan; the expectation + * is that there is already an active span. * @param key key * @return the metadata * @throws IOException if the retry invocation raises one (it shouldn't). */ @Retries.RetryRaw @VisibleForTesting + @InterfaceAudience.LimitedPrivate("external utilities") ObjectMetadata getObjectMetadata(String key) throws IOException { return getObjectMetadata(key, null, invoker, "getObjectMetadata"); } @@ -2043,12 +2380,10 @@ protected ObjectMetadata getObjectMetadata(String key, ChangeTracker changeTracker, Invoker changeInvoker, String operation) throws IOException { - GetObjectMetadataRequest request = - new GetObjectMetadataRequest(bucket, key); - //SSE-C requires to be filled in if enabled for object metadata - generateSSECustomerKey().ifPresent(request::setSSECustomerKey); ObjectMetadata meta = changeInvoker.retryUntranslated("GET " + key, true, () -> { + GetObjectMetadataRequest request + = getRequestFactory().newGetObjectMetadataRequest(key); incrementStatistic(OBJECT_METADATA_REQUESTS); DurationTracker duration = getDurationTrackerFactory() .trackDuration(ACTION_HTTP_HEAD_REQUEST.getSymbol()); @@ -2151,7 +2486,10 @@ protected S3ListResult continueListObjects(S3ListRequest request, () -> { if (useListV1) { return S3ListResult.v1( - s3.listNextBatchOfObjects(prevResult.getV1())); + s3.listNextBatchOfObjects( + getRequestFactory() + .newListNextBatchOfObjectsRequest( + prevResult.getV1()))); } else { request.getV2().setContinuationToken(prevResult.getV2() .getNextContinuationToken()); @@ -2205,7 +2543,8 @@ protected void deleteObject(String key) incrementStatistic(OBJECT_DELETE_OBJECTS); trackDurationOfInvocation(getDurationTrackerFactory(), OBJECT_DELETE_REQUEST.getSymbol(), - () -> s3.deleteObject(bucket, key)); + () -> s3.deleteObject(getRequestFactory() + .newDeleteObjectRequest(key))); return null; }); } @@ -2318,46 +2657,7 @@ private DeleteObjectsResult deleteObjects(DeleteObjectsRequest deleteRequest) */ public PutObjectRequest newPutObjectRequest(String key, ObjectMetadata metadata, File srcfile) { - Preconditions.checkNotNull(srcfile); - PutObjectRequest putObjectRequest = new PutObjectRequest(bucket, key, - srcfile); - setOptionalPutRequestParameters(putObjectRequest); - putObjectRequest.setCannedAcl(cannedACL); - putObjectRequest.setMetadata(metadata); - return putObjectRequest; - } - - /** - * Create a {@link PutObjectRequest} request. - * The metadata is assumed to have been configured with the size of the - * operation. - * @param key key of object - * @param metadata metadata header - * @param inputStream source data. - * @return the request - */ - PutObjectRequest newPutObjectRequest(String key, - ObjectMetadata metadata, - InputStream inputStream) { - Preconditions.checkNotNull(inputStream); - Preconditions.checkArgument(isNotEmpty(key), "Null/empty key"); - PutObjectRequest putObjectRequest = new PutObjectRequest(bucket, key, - inputStream, metadata); - setOptionalPutRequestParameters(putObjectRequest); - putObjectRequest.setCannedAcl(cannedACL); - return putObjectRequest; - } - - /** - * Create a new object metadata instance. - * Any standard metadata headers are added here, for example: - * encryption. - * @return a new metadata instance - */ - public ObjectMetadata newObjectMetadata() { - final ObjectMetadata om = new ObjectMetadata(); - setOptionalObjectMetadata(om); - return om; + return requestFactory.newPutObjectRequest(key, metadata, srcfile); } /** @@ -2369,11 +2669,7 @@ public ObjectMetadata newObjectMetadata() { * @return a new metadata instance */ public ObjectMetadata newObjectMetadata(long length) { - final ObjectMetadata om = newObjectMetadata(); - if (length >= 0) { - om.setContentLength(length); - } - return om; + return requestFactory.newObjectMetadata(length); } /** @@ -2423,13 +2719,15 @@ PutObjectResult putObjectDirect(PutObjectRequest putObjectRequest) LOG.debug("PUT {} bytes to {}", len, putObjectRequest.getKey()); incrementPutStartStatistics(len); try { - PutObjectResult result = s3.putObject(putObjectRequest); + PutObjectResult result = trackDurationOfSupplier(getDurationTrackerFactory(), + OBJECT_PUT_REQUESTS.getSymbol(), () -> + s3.putObject(putObjectRequest)); incrementPutCompletedStatistics(true, len); // update metadata finishedWrite(putObjectRequest.getKey(), len, result.getETag(), result.getVersionId(), null); return result; - } catch (AmazonClientException e) { + } catch (SdkBaseException e) { incrementPutCompletedStatistics(false, len); throw e; } @@ -2468,7 +2766,6 @@ UploadPartResult uploadPart(UploadPartRequest request) long len = request.getPartSize(); incrementPutStartStatistics(len); try { - setOptionalUploadPartRequestParameters(request); UploadPartResult uploadPartResult = s3.uploadPart(request); incrementPutCompletedStatistics(true, len); return uploadPartResult; @@ -2487,7 +2784,6 @@ UploadPartResult uploadPart(UploadPartRequest request) public void incrementPutStartStatistics(long bytes) { LOG.debug("PUT start {} bytes", bytes); incrementWriteOperations(); - incrementStatistic(OBJECT_PUT_REQUESTS); incrementGauge(OBJECT_PUT_REQUESTS_ACTIVE, 1); if (bytes > 0) { incrementGauge(OBJECT_PUT_BYTES_PENDING, bytes); @@ -2503,7 +2799,6 @@ public void incrementPutStartStatistics(long bytes) { */ public void incrementPutCompletedStatistics(boolean success, long bytes) { LOG.debug("PUT completed success={}; {} bytes", success, bytes); - incrementWriteOperations(); if (bytes > 0) { incrementStatistic(OBJECT_PUT_BYTES, bytes); decrementGauge(OBJECT_PUT_BYTES_PENDING, bytes); @@ -2572,9 +2867,7 @@ private DeleteObjectsResult removeKeysS3( try { if (enableMultiObjectsDelete) { result = deleteObjects( - new DeleteObjectsRequest(bucket) - .withKeys(keysToDelete) - .withQuiet(quiet)); + getRequestFactory().newDeleteObjectsRequest(keysToDelete, quiet)); } else { for (DeleteObjectsRequest.KeyVersion keyVersion : keysToDelete) { deleteObject(keyVersion.getKey()); @@ -2712,31 +3005,35 @@ DeleteObjectsResult removeKeys( */ @Retries.RetryTranslated public boolean delete(Path f, boolean recursive) throws IOException { - try { - entryPoint(INVOCATION_DELETE); - DeleteOperation deleteOperation = new DeleteOperation( - createStoreContext(), - innerGetFileStatus(f, true, StatusProbeEnum.ALL), - recursive, - operationCallbacks, - pageSize); - boolean outcome = deleteOperation.execute(); + checkNotClosed(); + final Path path = qualify(f); + // span covers delete, getFileStatus, fake directory operations. + try (AuditSpan span = createSpan(INVOCATION_DELETE.getSymbol(), + path.toString(), null)) { + boolean outcome = trackDuration(getDurationTrackerFactory(), + INVOCATION_DELETE.getSymbol(), + new DeleteOperation( + createStoreContext(), + innerGetFileStatus(path, true, StatusProbeEnum.ALL), + recursive, + operationCallbacks, + pageSize)); if (outcome) { try { - maybeCreateFakeParentDirectory(f); + maybeCreateFakeParentDirectory(path); } catch (AccessDeniedException e) { LOG.warn("Cannot create directory marker at {}: {}", f.getParent(), e.toString()); - LOG.debug("Failed to create fake dir above {}", f, e); + LOG.debug("Failed to create fake dir above {}", path, e); } } return outcome; } catch (FileNotFoundException e) { - LOG.debug("Couldn't delete {} - does not exist: {}", f, e.toString()); + LOG.debug("Couldn't delete {} - does not exist: {}", path, e.toString()); instrumentation.errorIgnored(); return false; } catch (AmazonClientException e) { - throw translateException("delete", f, e); + throw translateException("delete", path, e); } } @@ -2785,9 +3082,11 @@ protected void maybeCreateFakeParentDirectory(Path path) @Override public RemoteIterator listStatusIterator(Path p) throws FileNotFoundException, IOException { - RemoteIterator listStatusItr = once("listStatus", - p.toString(), () -> innerListStatus(p)); - return typeCastingRemoteIterator(listStatusItr); + Path path = qualify(p); + return typeCastingRemoteIterator(trackDurationAndSpan( + INVOCATION_LIST_STATUS, path, () -> + once("listStatus", path.toString(), () -> + innerListStatus(p)))); } /** @@ -2801,15 +3100,16 @@ public RemoteIterator listStatusIterator(Path p) */ public FileStatus[] listStatus(Path f) throws FileNotFoundException, IOException { - return once("listStatus", - f.toString(), - () -> iteratorToStatuses(innerListStatus(f), new HashSet<>())); + Path path = qualify(f); + return trackDurationAndSpan(INVOCATION_LIST_STATUS, path, () -> + once("listStatus", path.toString(), + () -> iteratorToStatuses(innerListStatus(path), + new HashSet<>()))); } /** * List the statuses of the files/directories in the given path if the path is - * a directory. - * + * a directory. The returned iterator is within the current active span. * @param f given path * @return the statuses of the files/directories in the given patch * @throws FileNotFoundException when the path does not exist; @@ -2821,11 +3121,10 @@ private RemoteIterator innerListStatus(Path f) IOException, AmazonClientException { Path path = qualify(f); LOG.debug("List status for path: {}", path); - entryPoint(INVOCATION_LIST_STATUS); Triple, DirListingMetadata, Boolean> statusesAssumingNonEmptyDir = listing - .getFileStatusesAssumingNonEmptyDir(path); + .getFileStatusesAssumingNonEmptyDir(path, getActiveAuditSpan()); if (!statusesAssumingNonEmptyDir.getLeft().hasNext() && statusesAssumingNonEmptyDir.getRight()) { @@ -2887,34 +3186,28 @@ public boolean allowAuthoritative(final Path path) { @VisibleForTesting public S3ListRequest createListObjectsRequest(String key, String delimiter) { - return createListObjectsRequest(key, delimiter, null); + return createListObjectsRequest(key, delimiter, maxKeys); } + /** + * Create the List objects request appropriate for the + * active list request option. + * @param key key for request + * @param delimiter any delimiter + * @param limit limit of keys + * @return the request + */ private S3ListRequest createListObjectsRequest(String key, - String delimiter, Integer overrideMaxKeys) { + String delimiter, int limit) { if (!useListV1) { ListObjectsV2Request request = - new ListObjectsV2Request().withBucketName(bucket) - .withMaxKeys(maxKeys) - .withPrefix(key); - if (delimiter != null) { - request.setDelimiter(delimiter); - } - if (overrideMaxKeys != null) { - request.setMaxKeys(overrideMaxKeys); - } + getRequestFactory().newListObjectsV2Request( + key, delimiter, limit); return S3ListRequest.v2(request); } else { - ListObjectsRequest request = new ListObjectsRequest(); - request.setBucketName(bucket); - request.setMaxKeys(maxKeys); - request.setPrefix(key); - if (delimiter != null) { - request.setDelimiter(delimiter); - } - if (overrideMaxKeys != null) { - request.setMaxKeys(overrideMaxKeys); - } + ListObjectsRequest request = + getRequestFactory().newListObjectsV1Request( + key, delimiter, limit); return S3ListRequest.v1(request); } } @@ -2958,79 +3251,50 @@ public UserGroupInformation getOwner() { * Make the given path and all non-existent parents into * directories. Has the semantics of Unix {@code 'mkdir -p'}. * Existence of the directory hierarchy is not an error. - * @param path path to create - * @param permission to apply to f + * @param p path to create + * @param permission to apply to path * @return true if a directory was created or already existed * @throws FileAlreadyExistsException there is a file at the path specified + * or is discovered on one of its ancestors. * @throws IOException other IO problems */ - // TODO: If we have created an empty file at /foo/bar and we then call - // mkdirs for /foo/bar/baz/roo what happens to the empty file /foo/bar/? - public boolean mkdirs(Path path, FsPermission permission) throws IOException, + public boolean mkdirs(Path p, FsPermission permission) throws IOException, FileAlreadyExistsException { - try { - entryPoint(INVOCATION_MKDIRS); - return innerMkdirs(path, permission); - } catch (AmazonClientException e) { - throw translateException("mkdirs", path, e); - } + Path path = qualify(p); + return trackDurationAndSpan( + INVOCATION_MKDIRS, path, + new MkdirOperation( + createStoreContext(), + path, + createMkdirOperationCallbacks())); } /** - * - * Make the given path and all non-existent parents into - * directories. - * See {@link #mkdirs(Path, FsPermission)} - * @param p path to create - * @param permission to apply to f - * @return true if a directory was created or already existed - * @throws FileAlreadyExistsException there is a file at the path specified - * @throws IOException other IO problems - * @throws AmazonClientException on failures inside the AWS SDK + * Override point: create the callbacks for Mkdir. + * This does not create a new span; caller must be in one. + * @return an implementation of the MkdirCallbacks, */ - private boolean innerMkdirs(Path p, FsPermission permission) - throws IOException, FileAlreadyExistsException, AmazonClientException { - Path f = qualify(p); - LOG.debug("Making directory: {}", f); - if (p.isRoot()) { - // fast exit for root. - return true; - } - FileStatus fileStatus; + @VisibleForTesting + public MkdirOperation.MkdirCallbacks createMkdirOperationCallbacks() { + return new MkdirOperationCallbacks(); + } - try { - fileStatus = innerGetFileStatus(f, false, - StatusProbeEnum.ALL); + /** + * Callbacks from the {@link MkdirOperation}. + */ + private class MkdirOperationCallbacks implements + MkdirOperation.MkdirCallbacks { - if (fileStatus.isDirectory()) { - return true; - } else { - throw new FileAlreadyExistsException("Path is a file: " + f); - } - } catch (FileNotFoundException e) { - // Walk path to root, ensuring closest ancestor is a directory, not file - Path fPart = f.getParent(); - while (fPart != null && !fPart.isRoot()) { - try { - fileStatus = getFileStatus(fPart); - if (fileStatus.isDirectory()) { - break; - } - if (fileStatus.isFile()) { - throw new FileAlreadyExistsException(String.format( - "Can't make directory for path '%s' since it is a file.", - fPart)); - } - } catch (FileNotFoundException fnfe) { - instrumentation.errorIgnored(); - } - fPart = fPart.getParent(); - } - String key = pathToKey(f); - // this will create the marker file, delete the parent entries - // and update S3Guard - createFakeDirectory(key); - return true; + @Override + public S3AFileStatus probePathStatus(final Path path, + final Set probes) throws IOException { + return S3AFileSystem.this.innerGetFileStatus(path, false, probes); + } + + @Override + public void createFakeDirectory(final String key) + throws IOException { + S3AFileSystem.this.createEmptyObject(key); } } @@ -3043,8 +3307,10 @@ private boolean innerMkdirs(Path p, FsPermission permission) */ @Retries.RetryTranslated public FileStatus getFileStatus(final Path f) throws IOException { - entryPoint(INVOCATION_GET_FILE_STATUS); - return innerGetFileStatus(f, false, StatusProbeEnum.ALL); + Path path = qualify(f); + return trackDurationAndSpan( + INVOCATION_GET_FILE_STATUS, path, () -> + innerGetFileStatus(path, false, StatusProbeEnum.ALL)); } /** @@ -3391,10 +3657,13 @@ private boolean s3Exists(final Path path, final Set probes) @Override public void copyFromLocalFile(boolean delSrc, boolean overwrite, Path src, Path dst) throws IOException { - entryPoint(INVOCATION_COPY_FROM_LOCAL_FILE); + checkNotClosed(); LOG.debug("Copying local file from {} to {}", src, dst); -// innerCopyFromLocalFile(delSrc, overwrite, src, dst); - super.copyFromLocalFile(delSrc, overwrite, src, dst); + trackDurationAndSpan(INVOCATION_COPY_FROM_LOCAL_FILE, dst, () -> { + // innerCopyFromLocalFile(delSrc, overwrite, src, dst); + super.copyFromLocalFile(delSrc, overwrite, src, dst); + return null; + }); } /** @@ -3423,7 +3692,6 @@ public void copyFromLocalFile(boolean delSrc, boolean overwrite, Path src, private void innerCopyFromLocalFile(boolean delSrc, boolean overwrite, Path src, Path dst) throws IOException, FileAlreadyExistsException, AmazonClientException { - entryPoint(INVOCATION_COPY_FROM_LOCAL_FILE); LOG.debug("Copying local file from {} to {}", src, dst); // Since we have a local file, we don't need to stream into a temporary file @@ -3437,7 +3705,7 @@ private void innerCopyFromLocalFile(boolean delSrc, boolean overwrite, } try { - FileStatus status = getFileStatus(dst); + FileStatus status = innerGetFileStatus(dst, false, StatusProbeEnum.ALL); if (!status.isFile()) { throw new FileAlreadyExistsException(dst + " exists and is not a file"); } @@ -3564,7 +3832,8 @@ protected synchronized void stopAllServices() { metadataStore, instrumentation, delegationTokens.orElse(null), - signerManager); + signerManager, + auditManager); closeAutocloseables(LOG, credentials); delegationTokens = Optional.empty(); signerManager = null; @@ -3624,11 +3893,14 @@ public String getCanonicalServiceName() { @Override public Token getDelegationToken(String renewer) throws IOException { - entryPoint(Statistic.INVOCATION_GET_DELEGATION_TOKEN); + checkNotClosed(); LOG.debug("Delegation token requested"); if (delegationTokens.isPresent()) { - return delegationTokens.get().getBoundOrNewDT(encryptionSecrets, - (renewer != null ? new Text(renewer) : new Text())); + return trackDurationAndSpan( + INVOCATION_GET_DELEGATION_TOKEN, null, () -> + delegationTokens.get().getBoundOrNewDT( + encryptionSecrets, + (renewer != null ? new Text(renewer) : new Text()))); } else { // Delegation token support is not set up LOG.debug("Token support is not enabled"); @@ -3646,6 +3918,7 @@ public Token getDelegationToken(String renewer) @Override public DelegationTokenIssuer[] getAdditionalTokenIssuers() throws IOException { + checkNotClosed(); if (delegationTokens.isPresent()) { return delegationTokens.get().getAdditionalTokenIssuers(); } else { @@ -3664,6 +3937,7 @@ public DelegationTokenIssuer[] getAdditionalTokenIssuers() * @return a policy for use in roles */ @Override + @InterfaceAudience.Private public List listAWSPolicyRules( final Set access) { if (access.isEmpty()) { @@ -3752,24 +4026,17 @@ private CopyResult copyFile(String srcKey, String dstKey, long size, message, e); } - ObjectMetadata dstom = cloneObjectMetadata(srcom); - setOptionalObjectMetadata(dstom); return readInvoker.retry( action, srcKey, true, () -> { CopyObjectRequest copyObjectRequest = - new CopyObjectRequest(bucket, srcKey, bucket, dstKey); + getRequestFactory().newCopyObjectRequest(srcKey, dstKey, srcom); changeTracker.maybeApplyConstraint(copyObjectRequest); - - setOptionalCopyObjectRequestParameters(srcom, copyObjectRequest); - copyObjectRequest.setCannedAccessControlList(cannedACL); - copyObjectRequest.setNewObjectMetadata(dstom); - Optional.ofNullable(srcom.getStorageClass()) - .ifPresent(copyObjectRequest::setStorageClass); incrementStatistic(OBJECT_COPY_REQUESTS); - Copy copy = transfers.copy(copyObjectRequest); + Copy copy = transfers.copy(copyObjectRequest, + getAuditManager().createStateChangeListener()); copy.addProgressListener(progressListener); CopyOutcome copyOutcome = CopyOutcome.waitForCopy(copy); InterruptedException interruptedException = @@ -3794,62 +4061,6 @@ private CopyResult copyFile(String srcKey, String dstKey, long size, }); } - /** - * Propagate encryption parameters from source file if set else use the - * current filesystem encryption settings. - * @param srcom source object meta. - * @param copyObjectRequest copy object request body. - */ - private void setOptionalCopyObjectRequestParameters( - ObjectMetadata srcom, - CopyObjectRequest copyObjectRequest) { - String sourceKMSId = srcom.getSSEAwsKmsKeyId(); - if (isNotEmpty(sourceKMSId)) { - // source KMS ID is propagated - LOG.debug("Propagating SSE-KMS settings from source {}", - sourceKMSId); - copyObjectRequest.setSSEAwsKeyManagementParams( - new SSEAwsKeyManagementParams(sourceKMSId)); - } - switch(getServerSideEncryptionAlgorithm()) { - /** - * Overriding with client encryption settings. - */ - case SSE_C: - generateSSECustomerKey().ifPresent(customerKey -> { - copyObjectRequest.setSourceSSECustomerKey(customerKey); - copyObjectRequest.setDestinationSSECustomerKey(customerKey); - }); - break; - case SSE_KMS: - generateSSEAwsKeyParams().ifPresent( - copyObjectRequest::setSSEAwsKeyManagementParams); - break; - default: - } - } - - /** - * Set the optional parameters when initiating the request (encryption, - * headers, storage, etc). - * @param request request to patch. - */ - protected void setOptionalMultipartUploadRequestParameters( - InitiateMultipartUploadRequest request) { - generateSSEAwsKeyParams().ifPresent(request::setSSEAwsKeyManagementParams); - generateSSECustomerKey().ifPresent(request::setSSECustomerKey); - } - - /** - * Sets server side encryption parameters to the part upload - * request when encryption is enabled. - * @param request upload part request - */ - protected void setOptionalUploadPartRequestParameters( - UploadPartRequest request) { - generateSSECustomerKey().ifPresent(request::setSSECustomerKey); - } - /** * Initiate a multipart upload from the preconfigured request. * Retry policy: none + untranslated. @@ -3862,42 +4073,9 @@ protected void setOptionalUploadPartRequestParameters( InitiateMultipartUploadResult initiateMultipartUpload( InitiateMultipartUploadRequest request) throws IOException { LOG.debug("Initiate multipart upload to {}", request.getKey()); - incrementStatistic(OBJECT_MULTIPART_UPLOAD_INITIATED); - return getAmazonS3Client().initiateMultipartUpload(request); - } - - private void setOptionalPutRequestParameters(PutObjectRequest request) { - generateSSEAwsKeyParams().ifPresent(request::setSSEAwsKeyManagementParams); - generateSSECustomerKey().ifPresent(request::setSSECustomerKey); - } - - private void setOptionalObjectMetadata(ObjectMetadata metadata) { - final S3AEncryptionMethods algorithm - = getServerSideEncryptionAlgorithm(); - if (S3AEncryptionMethods.SSE_S3.equals(algorithm)) { - metadata.setSSEAlgorithm(algorithm.getMethod()); - } - } - - /** - * Create the AWS SDK structure used to configure SSE, - * if the encryption secrets contain the information/settings for this. - * @return an optional set of KMS Key settings - */ - private Optional generateSSEAwsKeyParams() { - return EncryptionSecretOperations.createSSEAwsKeyManagementParams( - encryptionSecrets); - } - - /** - * Create the SSE-C structure for the AWS SDK, if the encryption secrets - * contain the information/settings for this. - * This will contain a secret extracted from the bucket/configuration. - * @return an optional customer key. - */ - private Optional generateSSECustomerKey() { - return EncryptionSecretOperations.createSSECustomerKey( - encryptionSecrets); + return trackDurationOfSupplier(getDurationTrackerFactory(), + OBJECT_MULTIPART_UPLOAD_INITIATED.getSymbol(), + () -> getAmazonS3Client().initiateMultipartUpload(request)); } /** @@ -3942,7 +4120,7 @@ void finishedWrite(String key, long length, String eTag, String versionId, CompletableFuture deletion; if (!keepDirectoryMarkers(p)) { deletion = submit( - unboundedThreadPool, + unboundedThreadPool, getActiveAuditSpan(), () -> { deleteUnnecessaryFakeDirectories( p.getParent(), @@ -4061,11 +4239,7 @@ private void deleteUnnecessaryFakeDirectories(Path path, @Retries.RetryTranslated private void createFakeDirectory(final String objectName) throws IOException { - if (!objectName.endsWith("/")) { - createEmptyObject(objectName + "/"); - } else { - createEmptyObject(objectName); - } + createEmptyObject(objectName); } /** @@ -4077,36 +4251,14 @@ private void createFakeDirectory(final String objectName) @Retries.RetryTranslated private void createEmptyObject(final String objectName) throws IOException { - final InputStream im = new InputStream() { - @Override - public int read() throws IOException { - return -1; - } - }; - - PutObjectRequest putObjectRequest = newPutObjectRequest(objectName, - newObjectMetadata(0L), - im); invoker.retry("PUT 0-byte object ", objectName, - true, - () -> putObjectDirect(putObjectRequest)); + true, () -> + putObjectDirect(getRequestFactory() + .newDirectoryMarkerRequest(objectName))); incrementPutProgressStatistics(objectName, 0); instrumentation.directoryCreated(); } - /** - * Creates a copy of the passed {@link ObjectMetadata}. - * Does so without using the {@link ObjectMetadata#clone()} method, - * to avoid copying unnecessary headers. - * @param source the {@link ObjectMetadata} to copy - * @return a copy of {@link ObjectMetadata} with only relevant attributes - */ - private ObjectMetadata cloneObjectMetadata(ObjectMetadata source) { - ObjectMetadata ret = newObjectMetadata(source.getContentLength()); - getHeaderProcessing().cloneObjectMetadata(source, ret); - return ret; - } - /** * Return the number of bytes that large input files should be optimally * be split into to minimize I/O time. @@ -4238,13 +4390,14 @@ public FileStatus[] globStatus( final Path pathPattern, final PathFilter filter) throws IOException { - entryPoint(INVOCATION_GLOB_STATUS); - return Globber.createGlobber(this) - .withPathPattern(pathPattern) - .withPathFiltern(filter) - .withResolveSymlinks(false) - .build() - .glob(); + return trackDurationAndSpan( + INVOCATION_GLOB_STATUS, pathPattern, () -> + Globber.createGlobber(this) + .withPathPattern(pathPattern) + .withPathFiltern(filter) + .withResolveSymlinks(false) + .build() + .glob()); } /** @@ -4253,8 +4406,15 @@ public FileStatus[] globStatus( */ @Override public boolean exists(Path f) throws IOException { - entryPoint(INVOCATION_EXISTS); - return super.exists(f); + final Path path = qualify(f); + try { + trackDurationAndSpan( + INVOCATION_EXISTS, path, () -> + innerGetFileStatus(path, false, StatusProbeEnum.ALL)); + return true; + } catch (FileNotFoundException e) { + return false; + } } /** @@ -4266,10 +4426,12 @@ public boolean exists(Path f) throws IOException { @Override @SuppressWarnings("deprecation") public boolean isDirectory(Path f) throws IOException { - entryPoint(INVOCATION_IS_DIRECTORY); + final Path path = qualify(f); try { - return innerGetFileStatus(f, false, StatusProbeEnum.DIRECTORIES) - .isDirectory(); + return trackDurationAndSpan( + INVOCATION_IS_DIRECTORY, path, () -> + innerGetFileStatus(path, false, StatusProbeEnum.DIRECTORIES) + .isDirectory()); } catch (FileNotFoundException e) { // not found or it is a file. return false; @@ -4285,10 +4447,11 @@ public boolean isDirectory(Path f) throws IOException { @Override @SuppressWarnings("deprecation") public boolean isFile(Path f) throws IOException { - entryPoint(INVOCATION_IS_FILE); + final Path path = qualify(f); try { - return innerGetFileStatus(f, false, StatusProbeEnum.HEAD_ONLY) - .isFile(); + return trackDurationAndSpan(INVOCATION_IS_FILE, path, () -> + innerGetFileStatus(path, false, StatusProbeEnum.HEAD_ONLY) + .isFile()); } catch (FileNotFoundException e) { // not found or it is a dir. return false; @@ -4319,15 +4482,17 @@ public boolean isFile(Path f) throws IOException { public EtagChecksum getFileChecksum(Path f, final long length) throws IOException { Preconditions.checkArgument(length >= 0); - entryPoint(INVOCATION_GET_FILE_CHECKSUM); - + final Path path = qualify(f); if (getConf().getBoolean(ETAG_CHECKSUM_ENABLED, ETAG_CHECKSUM_ENABLED_DEFAULT)) { - Path path = qualify(f); - LOG.debug("getFileChecksum({})", path); - ObjectMetadata headers = getObjectMetadata(path); - String eTag = headers.getETag(); - return eTag != null ? new EtagChecksum(eTag) : null; + return trackDurationAndSpan(INVOCATION_GET_FILE_CHECKSUM, path, () -> { + LOG.debug("getFileChecksum({})", path); + ObjectMetadata headers = getObjectMetadata(path, null, + invoker, + "getFileChecksum are"); + String eTag = headers.getETag(); + return eTag != null ? new EtagChecksum(eTag) : null; + }); } else { // disabled return null; @@ -4345,24 +4510,44 @@ private HeaderProcessing getHeaderProcessing() { @Override public byte[] getXAttr(final Path path, final String name) throws IOException { - return getHeaderProcessing().getXAttr(path, name); + checkNotClosed(); + try (AuditSpan span = createSpan( + INVOCATION_XATTR_GET_NAMED.getSymbol(), + path.toString(), null)) { + return getHeaderProcessing().getXAttr(path, name); + } } @Override public Map getXAttrs(final Path path) throws IOException { - return getHeaderProcessing().getXAttrs(path); + checkNotClosed(); + try (AuditSpan span = createSpan( + INVOCATION_XATTR_GET_MAP.getSymbol(), + path.toString(), null)) { + return getHeaderProcessing().getXAttrs(path); + } } @Override public Map getXAttrs(final Path path, final List names) throws IOException { - return getHeaderProcessing().getXAttrs(path, names); + checkNotClosed(); + try (AuditSpan span = createSpan( + INVOCATION_XATTR_GET_NAMED_MAP.getSymbol(), + path.toString(), null)) { + return getHeaderProcessing().getXAttrs(path, names); + } } @Override public List listXAttrs(final Path path) throws IOException { - return getHeaderProcessing().listXAttrs(path); + checkNotClosed(); + try (AuditSpan span = createSpan( + INVOCATION_OP_XATTR_LIST.getSymbol(), + path.toString(), null)) { + return getHeaderProcessing().listXAttrs(path); + } } /** @@ -4394,8 +4579,11 @@ public List listXAttrs(final Path path) throws IOException { @Retries.RetryTranslated public RemoteIterator listFiles(Path f, boolean recursive) throws FileNotFoundException, IOException { - return toLocatedFileStatusIterator(innerListFiles(f, recursive, - new Listing.AcceptFilesOnly(qualify(f)), null, true, false)); + final Path path = qualify(f); + return toLocatedFileStatusIterator( + trackDurationAndSpan(INVOCATION_LIST_FILES, path, () -> + innerListFiles(path, recursive, + new Listing.AcceptFilesOnly(path), null, true, false))); } /** @@ -4404,26 +4592,34 @@ public RemoteIterator listFiles(Path f, * @return an iterator. * @throws IOException failure */ + @InterfaceAudience.Private @Retries.RetryTranslated public RemoteIterator listFilesAndEmptyDirectories( Path f, boolean recursive) throws IOException { - return innerListFiles(f, recursive, Listing.ACCEPT_ALL_BUT_S3N, - null, true, false); + final Path path = qualify(f); + return trackDurationAndSpan(INVOCATION_LIST_FILES, path, () -> + innerListFiles(path, recursive, + Listing.ACCEPT_ALL_BUT_S3N, + null, true, false)); } /** * Recursive List of files and empty directories, force metadatastore * to act like it is non-authoritative. * @param f path to list from - * @param recursive + * @param recursive recursive listing? * @return an iterator. * @throws IOException failure */ + @InterfaceAudience.Private @Retries.RetryTranslated public RemoteIterator listFilesAndEmptyDirectoriesForceNonAuth( Path f, boolean recursive) throws IOException { - return innerListFiles(f, recursive, Listing.ACCEPT_ALL_BUT_S3N, - null, true, true); + final Path path = qualify(f); + return trackDurationAndSpan(INVOCATION_LIST_FILES, path, () -> + innerListFiles(path, recursive, + Listing.ACCEPT_ALL_BUT_S3N, + null, true, true)); } /** @@ -4472,7 +4668,6 @@ private RemoteIterator innerListFiles( final S3AFileStatus status, final boolean collectTombstones, final boolean forceNonAuthoritativeMS) throws IOException { - entryPoint(INVOCATION_LIST_FILES); Path path = qualify(f); LOG.debug("listFiles({}, {})", path, recursive); try { @@ -4490,7 +4685,8 @@ private RemoteIterator innerListFiles( recursive, acceptor, collectTombstones, - forceNonAuthoritativeMS); + forceNonAuthoritativeMS, + getActiveAuditSpan()); // If there are no list entries present, we // fallback to file existence check as the path // can be a file or empty directory. @@ -4498,7 +4694,7 @@ private RemoteIterator innerListFiles( // If file status was already passed, reuse it. final S3AFileStatus fileStatus = status != null ? status - : (S3AFileStatus) getFileStatus(path); + : innerGetFileStatus(path, false, StatusProbeEnum.ALL); if (fileStatus.isFile()) { return listing.createSingleStatusIterator( toLocatedFileStatus(fileStatus)); @@ -4540,8 +4736,8 @@ public RemoteIterator listLocatedStatus(Path f) public RemoteIterator listLocatedStatus(final Path f, final PathFilter filter) throws FileNotFoundException, IOException { - entryPoint(INVOCATION_LIST_LOCATED_STATUS); Path path = qualify(f); + AuditSpan span = entryPoint(INVOCATION_LIST_LOCATED_STATUS, path); LOG.debug("listLocatedStatus({}, {}", path, filter); RemoteIterator iterator = once("listLocatedStatus", path.toString(), @@ -4550,12 +4746,13 @@ public RemoteIterator listLocatedStatus(final Path f, // trigger a list call directly. final RemoteIterator locatedFileStatusIteratorForDir = - listing.getLocatedFileStatusIteratorForDir(path, filter); + listing.getLocatedFileStatusIteratorForDir(path, filter, + span); // If no listing is present then path might be a file. if (!locatedFileStatusIteratorForDir.hasNext()) { final S3AFileStatus fileStatus = - (S3AFileStatus) getFileStatus(path); + innerGetFileStatus(path, false, StatusProbeEnum.ALL); if (fileStatus.isFile()) { // simple case: File LOG.debug("Path is a file"); @@ -4571,43 +4768,6 @@ public RemoteIterator listLocatedStatus(final Path f, return toLocatedFileStatusIterator(iterator); } - /** - * Generate list located status for a directory. - * Also performing tombstone reconciliation for guarded directories. - * @param dir directory to check. - * @param filter a path filter. - * @return an iterator that traverses statuses of the given dir. - * @throws IOException in case of failure. - */ - private RemoteIterator getLocatedFileStatusIteratorForDir( - Path dir, PathFilter filter) throws IOException { - final String key = maybeAddTrailingSlash(pathToKey(dir)); - final Listing.FileStatusAcceptor acceptor = - new Listing.AcceptAllButSelfAndS3nDirs(dir); - boolean allowAuthoritative = allowAuthoritative(dir); - DirListingMetadata meta = - S3Guard.listChildrenWithTtl(metadataStore, dir, - ttlTimeProvider, allowAuthoritative); - Set tombstones = meta != null - ? meta.listTombstones() - : null; - final RemoteIterator cachedFileStatusIterator = - listing.createProvidedFileStatusIterator( - dirMetaToStatuses(meta), filter, acceptor); - return (allowAuthoritative && meta != null - && meta.isAuthoritative()) - ? listing.createLocatedFileStatusIterator( - cachedFileStatusIterator) - : listing.createTombstoneReconcilingIterator( - listing.createLocatedFileStatusIterator( - listing.createFileStatusListingIterator(dir, - createListObjectsRequest(key, "/"), - filter, - acceptor, - cachedFileStatusIterator)), - tombstones); - } - /** * Build a {@link S3ALocatedFileStatus} from a {@link FileStatus} instance. * @param status file status @@ -4632,10 +4792,16 @@ S3ALocatedFileStatus toLocatedFileStatus(S3AFileStatus status) * @return Iterator over multipart uploads. * @throws IOException on failure */ + @InterfaceAudience.Private + @Retries.RetryTranslated public MultipartUtils.UploadIterator listUploads(@Nullable String prefix) throws IOException { - return MultipartUtils.listMultipartUploads(s3, invoker, bucket, maxKeys, - prefix); + // span is picked up retained in the listing. + return trackDurationAndSpan(MULTIPART_UPLOAD_LIST, prefix, null, () -> + MultipartUtils.listMultipartUploads( + createStoreContext(), + s3, prefix, maxKeys + )); } /** @@ -4651,17 +4817,16 @@ public MultipartUtils.UploadIterator listUploads(@Nullable String prefix) @Retries.RetryTranslated public List listMultipartUploads(String prefix) throws IOException { - ListMultipartUploadsRequest request = new ListMultipartUploadsRequest( - bucket); - if (!prefix.isEmpty()) { - if (!prefix.endsWith("/")) { - prefix = prefix + "/"; - } - request.setPrefix(prefix); + // add a trailing / if needed. + if (prefix != null && !prefix.isEmpty() && !prefix.endsWith("/")) { + prefix = prefix + "/"; } - - return invoker.retry("listMultipartUploads", prefix, true, - () -> s3.listMultipartUploads(request).getMultipartUploads()); + String p = prefix; + return invoker.retry("listMultipartUploads", p, true, () -> { + ListMultipartUploadsRequest request = getRequestFactory() + .newListMultipartUploadsRequest(p); + return s3.listMultipartUploads(request).getMultipartUploads(); + }); } /** @@ -4674,7 +4839,7 @@ public List listMultipartUploads(String prefix) void abortMultipartUpload(String destKey, String uploadId) { LOG.info("Aborting multipart upload {} to {}", uploadId, destKey); getAmazonS3Client().abortMultipartUpload( - new AbortMultipartUploadRequest(getBucket(), + getRequestFactory().newAbortMultipartUploadRequest( destKey, uploadId)); } @@ -4697,7 +4862,7 @@ void abortMultipartUpload(MultipartUpload upload) { df.format(upload.getInitiated())); } getAmazonS3Client().abortMultipartUpload( - new AbortMultipartUploadRequest(getBucket(), + getRequestFactory().newAbortMultipartUploadRequest( destKey, uploadId)); } @@ -4817,7 +4982,7 @@ private FSDataInputStream select(final Path source, final Configuration options, final Optional providedStatus) throws IOException { - entryPoint(OBJECT_SELECT_REQUESTS); + final AuditSpan auditSpan = entryPoint(OBJECT_SELECT_REQUESTS, source); requireSelectSupport(source); final Path path = makeQualified(source); final S3AFileStatus fileStatus = extractOrFetchSimpleFileStatus(path, @@ -4827,7 +4992,7 @@ private FSDataInputStream select(final Path source, long ra = options.getLong(READAHEAD_RANGE, readAhead); S3ObjectAttributes objectAttributes = createObjectAttributes(fileStatus); S3AReadOpContext readContext = createReadContext(fileStatus, inputPolicy, - changeDetectionPolicy, ra); + changeDetectionPolicy, ra, auditSpan); if (changeDetectionPolicy.getSource() != ChangeDetectionPolicy.Source.None && fileStatus.getETag() != null) { @@ -4857,7 +5022,6 @@ private FSDataInputStream select(final Path source, readContext, expression, options, - generateSSECustomerKey(), objectAttributes); } @@ -4986,12 +5150,15 @@ public CompletableFuture openFileWithOptions( public S3AMultipartUploaderBuilder createMultipartUploader( final Path basePath) throws IOException { - StoreContext ctx = createStoreContext(); - return new S3AMultipartUploaderBuilder(this, - getWriteOperationHelper(), - ctx, - basePath, - statisticsContext.createMultipartUploaderStatistics()); + final Path path = makeQualified(basePath); + try (AuditSpan span = entryPoint(MULTIPART_UPLOAD_INSTANTIATED, path)) { + StoreContext ctx = createStoreContext(); + return new S3AMultipartUploaderBuilder(this, + createWriteOperationHelper(span), + ctx, + path, + statisticsContext.createMultipartUploaderStatistics()); + } } /** @@ -5078,5 +5245,15 @@ public ObjectMetadata getObjectMetadata(final String key) return once("getObjectMetadata", key, () -> S3AFileSystem.this.getObjectMetadata(key)); } + + @Override + public AuditSpan getActiveAuditSpan() { + return getAuditManager().getActiveThreadSpan(); + } + + @Override + public RequestFactory getRequestFactory() { + return S3AFileSystem.this.getRequestFactory(); + } } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java index c725fdf37ff8a..d56d4ac433eb7 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java @@ -20,14 +20,11 @@ import javax.annotation.Nullable; -import com.amazonaws.services.s3.AmazonS3; import com.amazonaws.services.s3.model.GetObjectRequest; import com.amazonaws.services.s3.model.S3Object; import com.amazonaws.services.s3.model.S3ObjectInputStream; -import com.amazonaws.services.s3.model.SSECustomerKey; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; -import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.CanSetReadahead; @@ -45,6 +42,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.Closeable; import java.io.EOFException; import java.io.IOException; import java.net.SocketTimeoutException; @@ -101,7 +99,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead, private S3Object object; private S3ObjectInputStream wrappedStream; private final S3AReadOpContext context; - private final AmazonS3 client; + private final InputStreamCallbacks client; private final String bucket; private final String key; private final String pathStr; @@ -110,8 +108,6 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead, private static final Logger LOG = LoggerFactory.getLogger(S3AInputStream.class); private final S3AInputStreamStatistics streamStatistics; - private S3AEncryptionMethods serverSideEncryptionAlgorithm; - private String serverSideEncryptionKey; private S3AInputPolicy inputPolicy; private long readahead = Constants.DEFAULT_READAHEAD_RANGE; @@ -150,7 +146,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead, */ public S3AInputStream(S3AReadOpContext ctx, S3ObjectAttributes s3Attributes, - AmazonS3 client) { + InputStreamCallbacks client) { Preconditions.checkArgument(isNotEmpty(s3Attributes.getBucket()), "No Bucket"); Preconditions.checkArgument(isNotEmpty(s3Attributes.getKey()), "No Key"); @@ -166,9 +162,6 @@ public S3AInputStream(S3AReadOpContext ctx, this.streamStatistics = ctx.getS3AStatisticsContext() .newInputStreamStatistics(); this.ioStatistics = streamStatistics.getIOStatistics(); - this.serverSideEncryptionAlgorithm = - s3Attributes.getServerSideEncryptionAlgorithm(); - this.serverSideEncryptionKey = s3Attributes.getServerSideEncryptionKey(); this.changeTracker = new ChangeTracker(uri, ctx.getChangeDetectionPolicy(), streamStatistics.getChangeTrackerStatistics(), @@ -211,16 +204,13 @@ private synchronized void reopen(String reason, long targetPos, long length, inputPolicy); long opencount = streamStatistics.streamOpened(); - GetObjectRequest request = new GetObjectRequest(bucket, key) + GetObjectRequest request = client.newGetRequest(key) .withRange(targetPos, contentRangeFinish - 1); - if (S3AEncryptionMethods.SSE_C.equals(serverSideEncryptionAlgorithm) && - StringUtils.isNotBlank(serverSideEncryptionKey)){ - request.setSSECustomerKey(new SSECustomerKey(serverSideEncryptionKey)); - } String operation = opencount == 0 ? OPERATION_OPEN : OPERATION_REOPEN; String text = String.format("%s %s at %d", operation, uri, targetPos); changeTracker.maybeApplyConstraint(request); + DurationTracker tracker = streamStatistics.initiateGetRequest(); try { object = Invoker.once(text, uri, @@ -567,6 +557,8 @@ public synchronized void close() throws IOException { // close or abort the stream closeStream("close() operation", this.contentRangeFinish, false); LOG.debug("Statistics of stream {}\n{}", key, streamStatistics); + // end the client+audit span. + client.close(); // this is actually a no-op super.close(); } finally { @@ -908,4 +900,27 @@ boolean isObjectStreamOpen() { public IOStatistics getIOStatistics() { return ioStatistics; } + + /** + * Callbacks for input stream IO. + */ + public interface InputStreamCallbacks extends Closeable { + + /** + * Create a GET request. + * @param key object key + * @return the request + */ + GetObjectRequest newGetRequest(String key); + + /** + * Execute the request. + * @param request the request + * @return the response + */ + @Retries.OnceRaw + S3Object getObject(GetObjectRequest request); + + } + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java index 3729341dbfe27..653d08019ed9e 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java @@ -21,6 +21,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.audit.AuditSpan; import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext; @@ -55,6 +56,8 @@ public class S3AReadOpContext extends S3AOpContext { */ private final long readahead; + private final AuditSpan auditSpan; + /** * Instantiate. * @param path path of read @@ -65,8 +68,9 @@ public class S3AReadOpContext extends S3AOpContext { * @param instrumentation statistics context * @param dstFileStatus target file status * @param inputPolicy the input policy - * @param readahead readahead for GET operations/skip, etc. * @param changeDetectionPolicy change detection policy. + * @param readahead readahead for GET operations/skip, etc. + * @param auditSpan active audit */ public S3AReadOpContext( final Path path, @@ -78,11 +82,13 @@ public S3AReadOpContext( FileStatus dstFileStatus, S3AInputPolicy inputPolicy, ChangeDetectionPolicy changeDetectionPolicy, - final long readahead) { + final long readahead, + final AuditSpan auditSpan) { super(isS3GuardEnabled, invoker, s3guardInvoker, stats, instrumentation, dstFileStatus); this.path = checkNotNull(path); + this.auditSpan = auditSpan; Preconditions.checkArgument(readahead >= 0, "invalid readahead %d", readahead); this.inputPolicy = checkNotNull(inputPolicy); @@ -133,6 +139,14 @@ public long getReadahead() { return readahead; } + /** + * Get the audit which was active when the file was opened. + * @return active span + */ + public AuditSpan getAuditSpan() { + return auditSpan; + } + @Override public String toString() { final StringBuilder sb = new StringBuilder( diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java index 314f13f0352e9..7d22776ff17ff 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java @@ -34,7 +34,6 @@ import com.amazonaws.services.s3.model.AmazonS3Exception; import com.amazonaws.services.s3.model.MultiObjectDeleteException; import com.amazonaws.services.s3.model.S3ObjectSummary; - import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; @@ -42,13 +41,13 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.util.functional.RemoteIterators; +import org.apache.hadoop.fs.s3a.audit.AuditFailureException; import org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider; import org.apache.hadoop.fs.s3a.impl.NetworkBinding; import org.apache.hadoop.fs.s3native.S3xLoginHelper; @@ -91,6 +90,7 @@ import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isUnknownBucket; import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.translateDeleteException; import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; +import static org.apache.hadoop.util.functional.RemoteIterators.filteringRemoteIterator; /** * Utility methods for S3A code. @@ -199,9 +199,11 @@ public static IOException translateException(@Nullable String operation, // call considered an sign of connectivity failure return (EOFException)new EOFException(message).initCause(exception); } - if (exception instanceof CredentialInitializationException) { + if (exception instanceof CredentialInitializationException + || exception instanceof AuditFailureException) { // the exception raised by AWSCredentialProvider list if the - // credentials were not accepted. + // credentials were not accepted, + // or auditing blocked the operation. return (AccessDeniedException)new AccessDeniedException(path, null, exception.toString()).initCause(exception); } @@ -1423,23 +1425,19 @@ private static void initUserAgent(Configuration conf, * an array. Given tombstones are filtered out. If the iterator * does return any item, an empty array is returned. * @param iterator a non-null iterator - * @param tombstones + * @param tombstones possibly empty set of tombstones * @return a possibly-empty array of file status entries - * @throws IOException + * @throws IOException failure */ public static S3AFileStatus[] iteratorToStatuses( RemoteIterator iterator, Set tombstones) throws IOException { - List statuses = new ArrayList<>(); - - while (iterator.hasNext()) { - S3AFileStatus status = iterator.next(); - if (!tombstones.contains(status.getPath())) { - statuses.add(status); - } - } - - return statuses.toArray(new S3AFileStatus[0]); + // this will close the span afterwards + RemoteIterator source = filteringRemoteIterator(iterator, + st -> !tombstones.contains(st.getPath())); + S3AFileStatus[] statuses = RemoteIterators + .toArray(source, new S3AFileStatus[0]); + return statuses; } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java index 1a53f0d1f8797..7542d21334476 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java @@ -96,11 +96,11 @@ public enum Statistic { INVOCATION_COPY_FROM_LOCAL_FILE( StoreStatisticNames.OP_COPY_FROM_LOCAL_FILE, "Calls of copyFromLocalFile()", - TYPE_COUNTER), + TYPE_DURATION), INVOCATION_CREATE( StoreStatisticNames.OP_CREATE, "Calls of create()", - TYPE_COUNTER), + TYPE_DURATION), INVOCATION_CREATE_NON_RECURSIVE( StoreStatisticNames.OP_CREATE_NON_RECURSIVE, "Calls of createNonRecursive()", @@ -108,39 +108,39 @@ public enum Statistic { INVOCATION_DELETE( StoreStatisticNames.OP_DELETE, "Calls of delete()", - TYPE_COUNTER), + TYPE_DURATION), INVOCATION_EXISTS( StoreStatisticNames.OP_EXISTS, "Calls of exists()", - TYPE_COUNTER), + TYPE_DURATION), INVOCATION_GET_DELEGATION_TOKEN( StoreStatisticNames.OP_GET_DELEGATION_TOKEN, "Calls of getDelegationToken()", - TYPE_COUNTER), + TYPE_DURATION), INVOCATION_GET_FILE_CHECKSUM( StoreStatisticNames.OP_GET_FILE_CHECKSUM, "Calls of getFileChecksum()", - TYPE_COUNTER), + TYPE_DURATION), INVOCATION_GET_FILE_STATUS( StoreStatisticNames.OP_GET_FILE_STATUS, "Calls of getFileStatus()", - TYPE_COUNTER), + TYPE_DURATION), INVOCATION_GLOB_STATUS( StoreStatisticNames.OP_GLOB_STATUS, "Calls of globStatus()", - TYPE_COUNTER), + TYPE_DURATION), INVOCATION_IS_DIRECTORY( StoreStatisticNames.OP_IS_DIRECTORY, "Calls of isDirectory()", - TYPE_COUNTER), + TYPE_DURATION), INVOCATION_IS_FILE( StoreStatisticNames.OP_IS_FILE, "Calls of isFile()", - TYPE_COUNTER), + TYPE_DURATION), INVOCATION_LIST_FILES( StoreStatisticNames.OP_LIST_FILES, "Calls of listFiles()", - TYPE_COUNTER), + TYPE_DURATION), INVOCATION_LIST_LOCATED_STATUS( StoreStatisticNames.OP_LIST_LOCATED_STATUS, "Calls of listLocatedStatus()", @@ -148,11 +148,11 @@ public enum Statistic { INVOCATION_LIST_STATUS( StoreStatisticNames.OP_LIST_STATUS, "Calls of listStatus()", - TYPE_COUNTER), + TYPE_DURATION), INVOCATION_MKDIRS( StoreStatisticNames.OP_MKDIRS, "Calls of mkdirs()", - TYPE_COUNTER), + TYPE_DURATION), INVOCATION_OPEN( StoreStatisticNames.OP_OPEN, "Calls of open()", @@ -160,7 +160,7 @@ public enum Statistic { INVOCATION_RENAME( StoreStatisticNames.OP_RENAME, "Calls of rename()", - TYPE_COUNTER), + TYPE_DURATION), /* The XAttr API metrics are all durations */ INVOCATION_XATTR_GET_MAP( @@ -207,15 +207,15 @@ public enum Statistic { OBJECT_MULTIPART_UPLOAD_INITIATED( StoreStatisticNames.OBJECT_MULTIPART_UPLOAD_INITIATED, "Object multipart upload initiated", - TYPE_COUNTER), + TYPE_DURATION), OBJECT_MULTIPART_UPLOAD_ABORTED( StoreStatisticNames.OBJECT_MULTIPART_UPLOAD_ABORTED, "Object multipart upload aborted", - TYPE_COUNTER), + TYPE_DURATION), OBJECT_PUT_REQUESTS( StoreStatisticNames.OBJECT_PUT_REQUEST, "Object put/multipart upload count", - TYPE_COUNTER), + TYPE_DURATION), OBJECT_PUT_REQUESTS_COMPLETED( StoreStatisticNames.OBJECT_PUT_REQUEST_COMPLETED, "Object put/multipart upload completed count", @@ -479,6 +479,9 @@ public enum Statistic { /* General Store operations */ + STORE_EXISTS_PROBE(StoreStatisticNames.STORE_EXISTS_PROBE, + "Store Existence Probe", + TYPE_DURATION), STORE_IO_REQUEST(StoreStatisticNames.STORE_IO_REQUEST, "requests made of the remote store", TYPE_COUNTER), @@ -530,10 +533,20 @@ public enum Statistic { StoreStatisticNames.MULTIPART_UPLOAD_COMPLETED, "Multipart Upload Completed", TYPE_COUNTER), + MULTIPART_UPLOAD_LIST( + StoreStatisticNames.MULTIPART_UPLOAD_LIST, + "Multipart Upload List", + TYPE_DURATION), MULTIPART_UPLOAD_STARTED( StoreStatisticNames.MULTIPART_UPLOAD_STARTED, "Multipart Upload Started", - TYPE_COUNTER); + TYPE_COUNTER), + + /* Audit statistics */ + AUDIT_SPAN_START( + "audit_span_start", + "Audit Span Started", + TYPE_DURATION); /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java index 9bdf61c22a1ff..00dd49284c0ba 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java @@ -23,7 +23,6 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; -import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; @@ -51,6 +50,10 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.s3a.audit.AuditSpan; +import org.apache.hadoop.fs.s3a.audit.AuditSpanSource; +import org.apache.hadoop.fs.s3a.api.RequestFactory; +import org.apache.hadoop.fs.s3a.impl.StoreContext; import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext; import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; import org.apache.hadoop.fs.s3a.s3guard.S3Guard; @@ -58,12 +61,9 @@ import org.apache.hadoop.util.DurationInfo; import org.apache.hadoop.util.functional.CallableRaisingIOE; -import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkArgument; import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull; +import static org.apache.hadoop.fs.s3a.audit.AuditIntegration.withinSpan; import static org.apache.hadoop.fs.s3a.Invoker.*; -import static org.apache.hadoop.fs.s3a.S3AUtils.longOption; -import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DEFAULT_UPLOAD_PART_COUNT_LIMIT; -import static org.apache.hadoop.fs.s3a.impl.InternalConstants.UPLOAD_PART_COUNT_LIMIT; /** * Helper for low-level operations against an S3 Bucket for writing data, @@ -87,6 +87,17 @@ * * * This API is for internal use only. + * Span scoping: This helper is instantiated with span; it will be used + * before operations which query/update S3 + * + * History + *

+ * - A nested class in S3AFileSystem
+ * - Single shared instance created and reused.
+ * - [HADOOP-13786] A separate class, single instance in S3AFS
+ * - [HDFS-13934] Split into interface and implementation
+ * - [HADOOP-15711] Adds audit tracking; one instance per use.
+ * 
*/ @InterfaceAudience.Private @InterfaceStability.Unstable @@ -116,22 +127,47 @@ public class WriteOperationHelper implements WriteOperations { */ private final S3AStatisticsContext statisticsContext; + private final StoreContext storeContext; + + /** + * Source of Audit spans. + */ + private final AuditSpanSource auditSpanSource; + + /** + * Audit Span. + */ + private AuditSpan auditSpan; + + /** + * Factory for AWS requests. + */ + private final RequestFactory requestFactory; + /** * Constructor. * @param owner owner FS creating the helper * @param conf Configuration object * @param statisticsContext statistics context + * @param auditSpanSource source of spans + * @param auditSpan span to activate * */ protected WriteOperationHelper(S3AFileSystem owner, Configuration conf, - S3AStatisticsContext statisticsContext) { + S3AStatisticsContext statisticsContext, + final AuditSpanSource auditSpanSource, + final AuditSpan auditSpan) { this.owner = owner; this.invoker = new Invoker(new S3ARetryPolicy(conf), this::operationRetried); this.conf = conf; this.statisticsContext = statisticsContext; - bucket = owner.getBucket(); + this.storeContext = owner.createStoreContext(); + this.bucket = owner.getBucket(); + this.auditSpanSource = auditSpanSource; + this.auditSpan = checkNotNull(auditSpan); + this.requestFactory = owner.getRequestFactory(); } /** @@ -150,6 +186,7 @@ void operationRetried(String text, Exception ex, int retries, /** * Execute a function with retry processing. + * Also activates the current span. * @param type of return value * @param action action to execute (used in error messages) * @param path path of work (used in error messages) @@ -164,10 +201,32 @@ public T retry(String action, boolean idempotent, CallableRaisingIOE operation) throws IOException { - + activateAuditSpan(); return invoker.retry(action, path, idempotent, operation); } + /** + * Get the audit span this object was created with. + * @return the audit span + */ + public AuditSpan getAuditSpan() { + return auditSpan; + } + + /** + * Activate the audit span. + */ + private void activateAuditSpan() { + auditSpan.activate(); + } + + /** + * Deactivate the audit span. + */ + private void deactivateAuditSpan() { + auditSpan.deactivate(); + } + /** * Create a {@link PutObjectRequest} request against the specific key. * @param destKey destination key @@ -176,15 +235,18 @@ public T retry(String action, * @param headers optional map of custom headers. * @return the request */ + @Retries.OnceRaw public PutObjectRequest createPutObjectRequest(String destKey, InputStream inputStream, long length, final Map headers) { + activateAuditSpan(); ObjectMetadata objectMetadata = newObjectMetadata(length); if (headers != null) { objectMetadata.setUserMetadata(headers); } - return owner.newPutObjectRequest(destKey, + return getRequestFactory().newPutObjectRequest( + destKey, objectMetadata, inputStream); } @@ -195,13 +257,16 @@ public PutObjectRequest createPutObjectRequest(String destKey, * @param sourceFile source file * @return the request */ + @Retries.OnceRaw public PutObjectRequest createPutObjectRequest(String dest, File sourceFile) { Preconditions.checkState(sourceFile.length() < Integer.MAX_VALUE, "File length is too big for a single PUT upload"); - return owner.newPutObjectRequest(dest, - newObjectMetadata((int) sourceFile.length()), - sourceFile); + activateAuditSpan(); + return getRequestFactory(). + newPutObjectRequest(dest, + newObjectMetadata((int) sourceFile.length()), + sourceFile); } /** @@ -227,7 +292,7 @@ public void writeFailed(Exception ex) { * @return a new metadata instance */ public ObjectMetadata newObjectMetadata(long length) { - return owner.newObjectMetadata(length); + return getRequestFactory().newObjectMetadata(length); } /** @@ -239,16 +304,16 @@ public ObjectMetadata newObjectMetadata(long length) { */ @Retries.RetryTranslated public String initiateMultiPartUpload(String destKey) throws IOException { + activateAuditSpan(); LOG.debug("Initiating Multipart upload to {}", destKey); - final InitiateMultipartUploadRequest initiateMPURequest = - new InitiateMultipartUploadRequest(bucket, - destKey, - newObjectMetadata(-1)); - initiateMPURequest.setCannedACL(owner.getCannedACL()); - owner.setOptionalMultipartUploadRequestParameters(initiateMPURequest); - return retry("initiate MultiPartUpload", destKey, true, - () -> owner.initiateMultipartUpload(initiateMPURequest).getUploadId()); + () -> { + final InitiateMultipartUploadRequest initiateMPURequest = + getRequestFactory().newMultipartUploadRequest( + destKey); + return owner.initiateMultipartUpload(initiateMPURequest) + .getUploadId(); + }); } /** @@ -278,23 +343,25 @@ private CompleteMultipartUploadResult finalizeMultipartUpload( throw new PathIOException(destKey, "No upload parts in multipart upload"); } - CompleteMultipartUploadResult uploadResult = - invoker.retry("Completing multipart upload", destKey, - true, - retrying, - () -> { - // a copy of the list is required, so that the AWS SDK doesn't - // attempt to sort an unmodifiable list. - return owner.getAmazonS3Client().completeMultipartUpload( - new CompleteMultipartUploadRequest(bucket, - destKey, - uploadId, - new ArrayList<>(partETags))); - } - ); - owner.finishedWrite(destKey, length, uploadResult.getETag(), - uploadResult.getVersionId(), operationState); - return uploadResult; + activateAuditSpan(); + try { + CompleteMultipartUploadResult uploadResult; + uploadResult = invoker.retry("Completing multipart upload", destKey, + true, + retrying, + () -> { + final CompleteMultipartUploadRequest request = + getRequestFactory().newCompleteMultipartUploadRequest( + destKey, uploadId, partETags); + return owner.getAmazonS3Client().completeMultipartUpload( + request); + }); + owner.finishedWrite(destKey, length, uploadResult.getETag(), + uploadResult.getVersionId(), operationState); + return uploadResult; + } finally { + getAuditSpan().deactivate(); + } } /** @@ -351,16 +418,17 @@ public void abortMultipartUpload(String destKey, String uploadId, destKey, true, retrying, - () -> owner.abortMultipartUpload( - destKey, - uploadId)); + withinSpan(getAuditSpan(), () -> + owner.abortMultipartUpload( + destKey, uploadId))); } else { // single pass attempt. once("Aborting multipart upload ID " + uploadId, destKey, - () -> owner.abortMultipartUpload( - destKey, - uploadId)); + withinSpan(getAuditSpan(), () -> + owner.abortMultipartUpload( + destKey, + uploadId))); } } @@ -373,7 +441,8 @@ public void abortMultipartUpload(String destKey, String uploadId, public void abortMultipartUpload(MultipartUpload upload) throws IOException { invoker.retry("Aborting multipart commit", upload.getKey(), true, - () -> owner.abortMultipartUpload(upload)); + withinSpan(getAuditSpan(), + () -> owner.abortMultipartUpload(upload))); } @@ -389,7 +458,7 @@ public int abortMultipartUploadsUnderPath(String prefix) throws IOException { LOG.debug("Aborting multipart uploads under {}", prefix); int count = 0; - List multipartUploads = owner.listMultipartUploads(prefix); + List multipartUploads = listMultipartUploads(prefix); LOG.debug("Number of outstanding uploads: {}", multipartUploads.size()); for (MultipartUpload upload: multipartUploads) { try { @@ -402,6 +471,18 @@ public int abortMultipartUploadsUnderPath(String prefix) return count; } + @Override + @Retries.RetryTranslated + public List listMultipartUploads(final String prefix) + throws IOException { + activateAuditSpan(); + try { + return owner.listMultipartUploads(prefix); + } finally { + deactivateAuditSpan(); + } + } + /** * Abort a multipart commit operation. * @param destKey destination key of ongoing operation @@ -409,6 +490,7 @@ public int abortMultipartUploadsUnderPath(String prefix) * @throws IOException on problems. * @throws FileNotFoundException if the abort ID is unknown */ + @Override @Retries.RetryTranslated public void abortMultipartCommit(String destKey, String uploadId) throws IOException { @@ -423,6 +505,7 @@ public void abortMultipartCommit(String destKey, String uploadId) * in {@code offset} and a length of block in {@code size} equal to * or less than the remaining bytes. * The part number must be less than 10000. + * Retry policy is once-translated; to much effort * @param destKey destination key of ongoing operation * @param uploadId ID of ongoing upload * @param partNumber current part number of the upload @@ -431,9 +514,11 @@ public void abortMultipartCommit(String destKey, String uploadId) * @param sourceFile optional source file. * @param offset offset in file to start reading. * @return the request. - * @throws IllegalArgumentException if the parameters are invalid -including + * @throws IllegalArgumentException if the parameters are invalid. * @throws PathIOException if the part number is out of range. */ + @Override + @Retries.OnceTranslated public UploadPartRequest newUploadPartRequest( String destKey, String uploadId, @@ -441,52 +526,17 @@ public UploadPartRequest newUploadPartRequest( int size, InputStream uploadStream, File sourceFile, - Long offset) throws PathIOException { - checkNotNull(uploadId); - // exactly one source must be set; xor verifies this - checkArgument((uploadStream != null) ^ (sourceFile != null), - "Data source"); - checkArgument(size >= 0, "Invalid partition size %s", size); - checkArgument(partNumber > 0, - "partNumber must be between 1 and %s inclusive, but is %s", - DEFAULT_UPLOAD_PART_COUNT_LIMIT, partNumber); - - LOG.debug("Creating part upload request for {} #{} size {}", - uploadId, partNumber, size); - long partCountLimit = longOption(conf, - UPLOAD_PART_COUNT_LIMIT, - DEFAULT_UPLOAD_PART_COUNT_LIMIT, - 1); - if (partCountLimit != DEFAULT_UPLOAD_PART_COUNT_LIMIT) { - LOG.warn("Configuration property {} shouldn't be overridden by client", - UPLOAD_PART_COUNT_LIMIT); - } - final String pathErrorMsg = "Number of parts in multipart upload exceeded." - + " Current part count = %s, Part count limit = %s "; - if (partNumber > partCountLimit) { - throw new PathIOException(destKey, - String.format(pathErrorMsg, partNumber, partCountLimit)); - } - UploadPartRequest request = new UploadPartRequest() - .withBucketName(bucket) - .withKey(destKey) - .withUploadId(uploadId) - .withPartNumber(partNumber) - .withPartSize(size); - if (uploadStream != null) { - // there's an upload stream. Bind to it. - request.setInputStream(uploadStream); - } else { - checkArgument(sourceFile.exists(), - "Source file does not exist: %s", sourceFile); - checkArgument(offset >= 0, "Invalid offset %s", offset); - long length = sourceFile.length(); - checkArgument(offset == 0 || offset < length, - "Offset %s beyond length of file %s", offset, length); - request.setFile(sourceFile); - request.setFileOffset(offset); - } - return request; + Long offset) throws IOException { + return once("upload part request", destKey, + withinSpan(getAuditSpan(), () -> + getRequestFactory().newUploadPartRequest( + destKey, + uploadId, + partNumber, + size, + uploadStream, + sourceFile, + offset))); } /** @@ -514,7 +564,8 @@ public PutObjectResult putObject(PutObjectRequest putObjectRequest) throws IOException { return retry("Writing Object", putObjectRequest.getKey(), true, - () -> owner.putObjectDirect(putObjectRequest)); + withinSpan(getAuditSpan(), () -> + owner.putObjectDirect(putObjectRequest))); } /** @@ -529,7 +580,8 @@ public UploadResult uploadObject(PutObjectRequest putObjectRequest) // no retry; rely on xfer manager logic return retry("Writing Object", putObjectRequest.getKey(), true, - () -> owner.executePut(putObjectRequest, null)); + withinSpan(getAuditSpan(), () -> + owner.executePut(putObjectRequest, null))); } /** @@ -543,13 +595,12 @@ public UploadResult uploadObject(PutObjectRequest putObjectRequest) public void revertCommit(String destKey, @Nullable BulkOperationState operationState) throws IOException { once("revert commit", destKey, - () -> { + withinSpan(getAuditSpan(), () -> { Path destPath = owner.keyToQualifiedPath(destKey); owner.deleteObjectAtPath(destPath, destKey, true, operationState); owner.maybeCreateFakeParentDirectory(destPath); - } - ); + })); } /** @@ -620,10 +671,11 @@ public BulkOperationState initiateOperation(final Path path, public UploadPartResult uploadPart(UploadPartRequest request) throws IOException { return retry("upload part #" + request.getPartNumber() - + " upload ID "+ request.getUploadId(), + + " upload ID " + request.getUploadId(), request.getKey(), true, - () -> owner.uploadPart(request)); + withinSpan(getAuditSpan(), + () -> owner.uploadPart(request))); } /** @@ -642,10 +694,8 @@ public Configuration getConf() { * @return the request */ public SelectObjectContentRequest newSelectRequest(Path path) { - SelectObjectContentRequest request = new SelectObjectContentRequest(); - request.setBucketName(bucket); - request.setKey(owner.pathToKey(path)); - return request; + return getRequestFactory().newSelectRequest( + storeContext.pathToKey(path)); } /** @@ -664,6 +714,8 @@ public SelectObjectContentResult select( final SelectObjectContentRequest request, final String action) throws IOException { + // no setting of span here as the select binding is (statically) created + // without any span. String bucketName = request.getBucketName(); Preconditions.checkArgument(bucket.equals(bucketName), "wrong bucket: %s", bucketName); @@ -676,7 +728,7 @@ public SelectObjectContentResult select( action, source.toString(), true, - () -> { + withinSpan(getAuditSpan(), () -> { try (DurationInfo ignored = new DurationInfo(LOG, "S3 Select operation")) { try { @@ -691,6 +743,30 @@ public SelectObjectContentResult select( throw e; } } - }); + })); + } + + @Override + public AuditSpan createSpan(final String name, + @Nullable final String path1, + @Nullable final String path2) throws IOException { + return auditSpanSource.createSpan(name, path1, path2); } + + /** + * Deactivate the audit span. + */ + @Override + public void close() throws IOException { + deactivateAuditSpan(); + } + + /** + * Get the request factory which uses this store's audit span. + * @return the request factory. + */ + public RequestFactory getRequestFactory() { + return requestFactory; + } + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java index 09b9cc924c6a4..e64e694ef13b1 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java @@ -19,6 +19,7 @@ package org.apache.hadoop.fs.s3a; import javax.annotation.Nullable; +import java.io.Closeable; import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; @@ -42,6 +43,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.s3a.audit.AuditSpanSource; import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; import org.apache.hadoop.util.functional.CallableRaisingIOE; @@ -54,7 +56,7 @@ * use `WriteOperationHelper` directly. * @since Hadoop 3.3.0 */ -public interface WriteOperations { +public interface WriteOperations extends AuditSpanSource, Closeable { /** * Execute a function with retry processing. @@ -184,6 +186,16 @@ void abortMultipartUpload(MultipartUpload upload) int abortMultipartUploadsUnderPath(String prefix) throws IOException; + /** + * Abort multipart uploads under a path: limited to the first + * few hundred. + * @param prefix prefix for uploads to abort + * @return a count of aborts + * @throws IOException trouble; FileNotFoundExceptions are swallowed. + */ + List listMultipartUploads(String prefix) + throws IOException; + /** * Abort a multipart commit operation. * @param destKey destination key of ongoing operation @@ -210,7 +222,7 @@ void abortMultipartCommit(String destKey, String uploadId) * @param sourceFile optional source file. * @param offset offset in file to start reading. * @return the request. - * @throws IllegalArgumentException if the parameters are invalid -including + * @throws IllegalArgumentException if the parameters are invalid * @throws PathIOException if the part number is out of range. */ UploadPartRequest newUploadPartRequest( @@ -220,7 +232,7 @@ UploadPartRequest newUploadPartRequest( int size, InputStream uploadStream, File sourceFile, - Long offset) throws PathIOException; + Long offset) throws IOException; /** * PUT an object directly (i.e. not via the transfer manager). diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java new file mode 100644 index 0000000000000..b2c3b5b042b8e --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java @@ -0,0 +1,282 @@ +/* + * 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.api; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.InputStream; +import java.util.List; +import java.util.Optional; + +import com.amazonaws.services.s3.model.AbortMultipartUploadRequest; +import com.amazonaws.services.s3.model.CannedAccessControlList; +import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest; +import com.amazonaws.services.s3.model.CopyObjectRequest; +import com.amazonaws.services.s3.model.DeleteObjectRequest; +import com.amazonaws.services.s3.model.DeleteObjectsRequest; +import com.amazonaws.services.s3.model.GetObjectMetadataRequest; +import com.amazonaws.services.s3.model.GetObjectRequest; +import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest; +import com.amazonaws.services.s3.model.ListMultipartUploadsRequest; +import com.amazonaws.services.s3.model.ListNextBatchOfObjectsRequest; +import com.amazonaws.services.s3.model.ListObjectsRequest; +import com.amazonaws.services.s3.model.ListObjectsV2Request; +import com.amazonaws.services.s3.model.ObjectListing; +import com.amazonaws.services.s3.model.ObjectMetadata; +import com.amazonaws.services.s3.model.PartETag; +import com.amazonaws.services.s3.model.PutObjectRequest; +import com.amazonaws.services.s3.model.SSEAwsKeyManagementParams; +import com.amazonaws.services.s3.model.SSECustomerKey; +import com.amazonaws.services.s3.model.SelectObjectContentRequest; +import com.amazonaws.services.s3.model.UploadPartRequest; + +import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.s3a.S3AEncryptionMethods; +import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets; + +/** + * Factory for S3 objects. + * + * This is where the owner FS's {@code prepareRequest()} + * callback is invoked to mark up a request for this span. + * + * All AWS request objects must be created through this, in + * the active audit span. + * The standard implementation provides a callback for the S3AFS or + * tests to be invoked to prepare each request. + * Such callbacks SHOULD NOT raise exceptions other + * than argument validation exceptions. + * as there are no guarantees how they are processed. + * That is: no guarantees of retry or translation. + */ +public interface RequestFactory { + + /** + * Get the canned ACL of this FS. + * @return an ACL, if any + */ + CannedAccessControlList getCannedACL(); + + /** + * Create the AWS SDK structure used to configure SSE, + * if the encryption secrets contain the information/settings for this. + * @return an optional set of KMS Key settings + */ + Optional generateSSEAwsKeyParams(); + + /** + * Create the SSE-C structure for the AWS SDK, if the encryption secrets + * contain the information/settings for this. + * This will contain a secret extracted from the bucket/configuration. + * @return an optional customer key. + */ + Optional generateSSECustomerKey(); + + /** + * Get the encryption algorithm of this endpoint. + * @return the encryption algorithm. + */ + S3AEncryptionMethods getServerSideEncryptionAlgorithm(); + + /** + * Sets server side encryption parameters to the part upload + * request when encryption is enabled. + * @param request upload part request + */ + void setOptionalUploadPartRequestParameters( + UploadPartRequest request); + + /** + * Sets server side encryption parameters to the part upload + * request when encryption is enabled. + * @param request upload part request + */ + void setOptionalGetObjectMetadataParameters( + GetObjectMetadataRequest request); + + /** + * Set the optional parameters when initiating the request (encryption, + * headers, storage, etc). + * @param request request to patch. + */ + void setOptionalMultipartUploadRequestParameters( + InitiateMultipartUploadRequest request); + + void setOptionalObjectMetadata(ObjectMetadata metadata); + + /** + * Create a new object metadata instance. + * Any standard metadata headers are added here, for example: + * encryption. + * + * @param length length of data to set in header; Ignored if negative + * @return a new metadata instance + */ + ObjectMetadata newObjectMetadata(long length); + + /** + * Create a copy request. + * This includes the work of copying the relevant parts + * of the metadata from the source + * @param srcKey source + * @param dstKey destination + * @param srcom source object metadata. + * @return the request + */ + CopyObjectRequest newCopyObjectRequest(String srcKey, + String dstKey, + ObjectMetadata srcom); + + /** + * Create a putObject request. + * Adds the ACL and metadata + * @param key key of object + * @param metadata metadata header + * @param srcfile source file + * @return the request + */ + PutObjectRequest newPutObjectRequest(String key, + ObjectMetadata metadata, File srcfile); + + /** + * Create a {@link PutObjectRequest} request. + * The metadata is assumed to have been configured with the size of the + * operation. + * @param key key of object + * @param metadata metadata header + * @param inputStream source data. + * @return the request + */ + PutObjectRequest newPutObjectRequest(String key, + ObjectMetadata metadata, + InputStream inputStream); + + /** + * Create a {@link PutObjectRequest} request for creating + * an empty directory. + * + * @param directory destination directory. + * @return request for a zero byte upload. + */ + PutObjectRequest newDirectoryMarkerRequest(String directory); + + /** + * List all multipart uploads under a prefix. + * @param prefix prefix to list under + * @return the request. + */ + ListMultipartUploadsRequest newListMultipartUploadsRequest( + @Nullable String prefix); + + /** + * Abort a multipart upload. + * @param destKey destination object key + * @param uploadId ID of initiated upload + * @return the request. + */ + + AbortMultipartUploadRequest newAbortMultipartUploadRequest(String destKey, + String uploadId); + + /** + * Start a multipart upload. + * @param destKey destination object key + * @return the request. + */ + InitiateMultipartUploadRequest newMultipartUploadRequest(String destKey); + + /** + * Complete a multipart upload. + * @param destKey destination object key + * @param uploadId ID of initiated upload + * @param partETags ordered list of etags + * @return the request. + */ + CompleteMultipartUploadRequest newCompleteMultipartUploadRequest( + String destKey, + String uploadId, + List partETags); + + /** + * Create a HEAD request. + * @param key key, may have trailing / + * @return the request. + */ + GetObjectMetadataRequest newGetObjectMetadataRequest(String key); + + /** + * Create a GET request. + * @param key object key + * @return the request. + */ + GetObjectRequest newGetObjectRequest(String key); + + /** + * Create and initialize a part request of a multipart upload. + * Exactly one of: {@code uploadStream} or {@code sourceFile} + * must be specified. + * A subset of the file may be posted, by providing the starting point + * in {@code offset} and a length of block in {@code size} equal to + * or less than the remaining bytes. + * @param destKey destination key of ongoing operation + * @param uploadId ID of ongoing upload + * @param partNumber current part number of the upload + * @param size amount of data + * @param uploadStream source of data to upload + * @param sourceFile optional source file. + * @param offset offset in file to start reading. + * @return the request. + * @throws PathIOException if the part number is out of range. + */ + UploadPartRequest newUploadPartRequest( + String destKey, + String uploadId, + int partNumber, + int size, + InputStream uploadStream, + File sourceFile, + long offset) throws PathIOException; + + /** + * Create a S3 Select request for the destination object. + * This does not build the query. + * @param key object key + * @return the request + */ + SelectObjectContentRequest newSelectRequest(String key); + + ListObjectsRequest newListObjectsV1Request(String key, + String delimiter, + int maxKeys); + + ListNextBatchOfObjectsRequest newListNextBatchOfObjectsRequest( + ObjectListing prev); + + ListObjectsV2Request newListObjectsV2Request(String key, + String delimiter, + int maxKeys); + + DeleteObjectRequest newDeleteObjectRequest(String key); + + DeleteObjectsRequest newDeleteObjectsRequest( + List keysToDelete, + boolean quiet); + + void setEncryptionSecrets(EncryptionSecrets secrets); +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/package-info.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/package-info.java new file mode 100644 index 0000000000000..625c8a3fca2cc --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/package-info.java @@ -0,0 +1,34 @@ +/* + * 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. + */ + +/** + * Where the interfaces for classes implemented in {@code o.a.h.fs.s3a.impl} + * should go. This is to allow for extension points to use these interfaces + * without having any java module access to the .impl package. + * + * This is public for S3A extension points, however there are no + * guarantees of stability -changes may break things, possibly + * unintentionally. + */ + +@InterfaceAudience.LimitedPrivate("extensions") +@InterfaceStability.Unstable +package org.apache.hadoop.fs.s3a.api; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AWSRequestAnalyzer.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AWSRequestAnalyzer.java new file mode 100644 index 0000000000000..67bc184805641 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AWSRequestAnalyzer.java @@ -0,0 +1,256 @@ +/* + * 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.audit; + +import java.util.List; + +import com.amazonaws.AmazonWebServiceRequest; +import com.amazonaws.services.s3.model.AbortMultipartUploadRequest; +import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest; +import com.amazonaws.services.s3.model.CopyPartRequest; +import com.amazonaws.services.s3.model.DeleteObjectRequest; +import com.amazonaws.services.s3.model.DeleteObjectsRequest; +import com.amazonaws.services.s3.model.GetBucketLocationRequest; +import com.amazonaws.services.s3.model.GetObjectMetadataRequest; +import com.amazonaws.services.s3.model.GetObjectRequest; +import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest; +import com.amazonaws.services.s3.model.ListMultipartUploadsRequest; +import com.amazonaws.services.s3.model.ListObjectsRequest; +import com.amazonaws.services.s3.model.ListObjectsV2Request; +import com.amazonaws.services.s3.model.PutObjectRequest; +import com.amazonaws.services.s3.model.UploadPartRequest; + +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.*; + +/** + * Extract information from a request. + * Intended for reporting and error logs. + */ +public class AWSRequestAnalyzer { + + /** + * Given an AWS request, try to analyze it to operation, + * read/write and path. + * @param request request. + * @return information about the request. + */ + public RequestInfo analyze(AmazonWebServiceRequest request) { + + if (request instanceof GetObjectMetadataRequest) { + return reading(ACTION_HTTP_HEAD_REQUEST, + ((GetObjectMetadataRequest) request).getKey(), 0); + } else if (request instanceof GetObjectRequest) { + GetObjectRequest r = (GetObjectRequest) request; + long[] range = r.getRange(); + long size = range == null + ? -1 + : range[1] - range[0]; + return reading(ACTION_HTTP_GET_REQUEST, + r.getKey(), + size); + } else if (request instanceof ListObjectsV2Request) { + ListObjectsV2Request r = (ListObjectsV2Request) request; + return reading(OBJECT_LIST_REQUEST, + r.getPrefix(), + r.getMaxKeys()); + } else if (request instanceof ListObjectsRequest) { + ListObjectsRequest r = (ListObjectsRequest) request; + return reading(OBJECT_LIST_REQUEST, + r.getPrefix(), + r.getMaxKeys()); + } else if (request instanceof InitiateMultipartUploadRequest) { + return writing(MULTIPART_UPLOAD_STARTED, + ((InitiateMultipartUploadRequest) request).getKey(), + 0); + } else if (request instanceof UploadPartRequest) { + UploadPartRequest r = (UploadPartRequest) request; + return writing(MULTIPART_UPLOAD_PART_PUT, + r.getKey(), + r.getPartSize()); + } else if (request instanceof CompleteMultipartUploadRequest) { + CompleteMultipartUploadRequest r + = (CompleteMultipartUploadRequest) request; + return writing(MULTIPART_UPLOAD_COMPLETED, + r.getKey(), + r.getPartETags().size()); + } else if (request instanceof AbortMultipartUploadRequest) { + return writing(MULTIPART_UPLOAD_ABORTED, + ((AbortMultipartUploadRequest) request).getKey(), + 0); + } else if (request instanceof ListMultipartUploadsRequest) { + ListMultipartUploadsRequest r + = (ListMultipartUploadsRequest) request; + return reading(MULTIPART_UPLOAD_LIST, + r.getPrefix(), + r.getMaxUploads()); + } else if (request instanceof DeleteObjectRequest) { + return writing(OBJECT_DELETE_REQUEST, + ((DeleteObjectRequest) request).getKey(), + 1); + } else if (request instanceof DeleteObjectsRequest) { + // use first key as the path + DeleteObjectsRequest r = (DeleteObjectsRequest) request; + List keys + = r.getKeys(); + return writing(OBJECT_BULK_DELETE_REQUEST, + keys.isEmpty() ? null : keys.get(0).getKey(), + keys.size()); + } else if (request instanceof GetBucketLocationRequest) { + GetBucketLocationRequest r = (GetBucketLocationRequest) request; + return reading(STORE_EXISTS_PROBE, + r.getBucketName(), + 0); + } else if (request instanceof PutObjectRequest) { + PutObjectRequest r = (PutObjectRequest) request; + return writing(OBJECT_PUT_REQUEST, + r.getKey(), + 0); + } + // no explicit support, return classname + return writing(request.getClass().getName(), null, 0); + } + + /** + * A request. + * @param verb verb + * @param mutating does this update the store + * @param key object/prefix, etc. + * @param size nullable size + * @return request info + */ + private RequestInfo request(final String verb, + final boolean mutating, + final String key, + final Number size) { + return new RequestInfo(verb, mutating, key, size); + } + + /** + * A read request. + * @param verb verb + * @param key object/prefix, etc. + * @param size nullable size + * @return request info + */ + private RequestInfo reading(final String verb, + final String key, final Number size) { + return request(verb, false, key, size); + } + + /** + * A write request of some form. + * @param verb verb + * @param key object/prefix, etc. + * @param size nullable size + * @return request info + */ + private RequestInfo writing(final String verb, + final String key, final Number size) { + return request(verb, true, key, size); + } + + /** + * Predicate which returns true if the request is of a kind which + * could be outside a span because of how the AWS SDK generates them. + * @param request request + * @param type of request + * @return true if the transfer manager creates them. + */ + public static final boolean + isRequestNotAlwaysInSpan(final T request) { + return request instanceof CopyPartRequest + || request instanceof CompleteMultipartUploadRequest; + } + + /** + * Info about a request. + */ + public static final class RequestInfo { + + /** + * Verb. + */ + private String verb; + + /** + * Is thisd a mutating call? + */ + private boolean mutating; + + /** + * Key if there is one; maybe first key in a list. + */ + private String key; + + /** + * Size, where the meaning of size depends on the request. + */ + private long size; + + + /** + * Construct. + * @param verb operation/classname, etc. + * @param mutating does this update S3 State. + * @param key key/path/bucket operated on. + * @param size size of request (bytes, elements, limit...). Nullable. + */ + private RequestInfo(final String verb, + final boolean mutating, + final String key, + final Number size) { + this.verb = verb; + this.mutating = mutating; + this.key = key; + this.size = toSafeLong(size); + } + + public String getVerb() { + return verb; + } + + public boolean isMutating() { + return mutating; + } + + public String getKey() { + return key; + } + + public long getSize() { + return size; + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder( + "{"); + sb.append(verb); + sb.append(" '").append(key).append('\''); + sb.append(" size=").append(size); + sb.append(", mutating=").append(mutating); + sb.append('}'); + return sb.toString(); + } + } + + private static long toSafeLong(final Number size) { + return size != null ? size.longValue() : 0; + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AbstractAuditSpanImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AbstractAuditSpanImpl.java new file mode 100644 index 0000000000000..52c5caea0720c --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AbstractAuditSpanImpl.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.audit; + +public abstract class AbstractAuditSpanImpl implements AuditSpan{ + + protected AbstractAuditSpanImpl() { + } + + @Override + public final void close() { + deactivate(); + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AbstractOperationAuditor.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AbstractOperationAuditor.java new file mode 100644 index 0000000000000..eae7bf7803526 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AbstractOperationAuditor.java @@ -0,0 +1,102 @@ +/* + * 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.audit; + +import java.io.IOException; +import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; +import org.apache.hadoop.service.AbstractService; + +/** + * This is a long-lived service which is created in S3A FS initialize + * (make it fast!) which provides context for tracking operations made to S3. + * An IOStatisticsStore is passed in -in production this is expected to + * be the S3AFileSystem instrumentation, which will have the + * {@code AUDIT_SPAN_START} statistic configured for counting durations. + */ +public abstract class AbstractOperationAuditor extends AbstractService + implements OperationAuditor { + + /** + * Destination for recording statistics, especially duration/count of + * operations. + */ + private final IOStatisticsStore iostatistics; + + /** + * Construct. + * @param name name + * @param iostatistics store of statistics. + */ + protected AbstractOperationAuditor(final String name, + final IOStatisticsStore iostatistics) { + super(name); + this.iostatistics = iostatistics; + } + + /** + * Get the IOStatistics Store. + * @return the IOStatistics store updated with statistics. + */ + public IOStatisticsStore getIOStatistics() { + return iostatistics; + } + + /** + * Create, and initialize and an audit service. + * The service start operation is not called: that is left to + * the caller. + * @param conf configuration to read the key from and to use to init + * the service. + * @param key key containing the classname + * @param store IOStatistics store. + * @return instantiated class. + * @throws IOException failure to initialise. + */ + @SuppressWarnings("ClassReferencesSubclass") + public static OperationAuditor createInstance( + Configuration conf, + String key, + IOStatisticsStore store) throws IOException { + try { + final Class auditClassname + = conf.getClass( + key, + LoggingAuditor.class, + OperationAuditor.class); + final Constructor constructor + = auditClassname.getConstructor(String.class, + IOStatisticsStore.class); + final OperationAuditor instance = constructor.newInstance( + auditClassname.getCanonicalName(), store); + instance.init(conf); + return instance; + } catch (NoSuchMethodException | InstantiationException + | RuntimeException + | IllegalAccessException | InvocationTargetException e) { + throw new IOException("Failed to instantiate class " + + "defined in " + key, + e); + } + } + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/ActiveAuditManager.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/ActiveAuditManager.java new file mode 100644 index 0000000000000..b68c5b32b7337 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/ActiveAuditManager.java @@ -0,0 +1,454 @@ +/* + * 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.audit; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import com.amazonaws.AmazonWebServiceRequest; +import com.amazonaws.Request; +import com.amazonaws.Response; +import com.amazonaws.SdkBaseException; +import com.amazonaws.handlers.RequestHandler2; +import com.amazonaws.services.s3.transfer.Transfer; +import com.amazonaws.services.s3.transfer.internal.TransferStateChangeListener; +import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; +import org.apache.hadoop.service.CompositeService; + +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.fs.s3a.audit.AuditInternalConstants.AUDIT_SPAN_HANDLER_CONTEXT; + +/** + * Thread management for the active audit. + * This should be created by whatever wants + * to have active audit span tracking. + * Creates and starts the actual + * {@link OperationAuditor} for auditing. + * It stores the thread specific span and returns a wrapping + * span. + * When the wrapper is closed/deactivated it + * will deactivate the wrapped span and then + * switch the active span to the unbounded span. + */ +@InterfaceAudience.Private +public final class ActiveAuditManager + extends CompositeService + implements AuditSpanCallbacks, AuditManager { + + /** + * This is where the context gets logged to. + */ + private static final Logger LOG = + LoggerFactory.getLogger(LoggingAuditor.class); + + /** + * Audit service. + */ + private OperationAuditor auditService; + + /** + * Some basic analysis for the logs. + */ + private final AWSRequestAnalyzer analyzer = new AWSRequestAnalyzer(); + + /** + * This is the span returned to after a wrapper is closed or + * the span is reset to the unbonded span.. + */ + private WrappingAuditSpan unboundedSpan; + + /** + * Thread local span. This defaults to being + * the unbonded span. + */ + private final ThreadLocal activeSpan = + ThreadLocal.withInitial(() -> getUnboundedSpan()); + + /** + * Destination for recording statistics, especially duration/count of + * operations. + */ + private final IOStatisticsStore iostatistics; + + /** + * Instantiate. + * @param iostatistics statistics target + */ + public ActiveAuditManager(final IOStatisticsStore iostatistics) { + super("ActiveAuditManager"); + this.iostatistics = iostatistics; + } + + @Override + protected void serviceInit(final Configuration conf) throws Exception { + super.serviceInit(conf); + // create and register the service so it follows the same lifecycle + auditService = AbstractOperationAuditor.createInstance( + getConfig(), + AuditConstants.AUDIT_SERVICE_CLASSNAME, + iostatistics); + addService(auditService); + } + + @Override + protected void serviceStart() throws Exception { + super.serviceStart(); + setUnboundedSpan(new WrappingAuditSpan( + auditService.getUnbondedSpan(), false)); + } + + /** + * Get the unbounded span. Until this manager + * is fully initialized it will return the no-op + * span. + * @return the unbounded span. + */ + private WrappingAuditSpan getUnboundedSpan() { + return unboundedSpan; + } + + public void setUnboundedSpan(final WrappingAuditSpan unboundedSpan) { + this.unboundedSpan = unboundedSpan; + } + + @Override + public AuditSpan getActiveThreadSpan() { + return activeSpan.get(); + } + + /** + * Set a specific span as the active span. + * @param span span to use. + * @return the wrapping span. + */ + private AuditSpan setActiveThreadSpan(AuditSpan span) { + return switchToActiveSpan(new WrappingAuditSpan(span, true)); + } + + /** + * Switch to a given span. If it is null, use the + * unbounded span. + * @param span to switch to. + * @return the span switched to + */ + private WrappingAuditSpan switchToActiveSpan(WrappingAuditSpan span) { + if (span != null && span.isValidSpan()) { + activeSpan.set(span); + } else { + activeSpan.set(unboundedSpan); + } + return activeSpan.get(); + } + + /** + * Start an operation; as well as invoking the audit + * service to do this, sets the operation as the + * active operation for this thread. + * @param name operation name. + * @param path1 first path of operation + * @param path2 second path of operation + * @return a wrapped audit span + * @throws IOException failure + */ + @Override + public AuditSpan createSpan(final String name, + @Nullable final String path1, + @Nullable final String path2) throws IOException { + // must be started + Preconditions.checkState(isInState(STATE.STARTED), + "Audit Manager %s is in wrong state: %s", + this, getServiceState()); + return setActiveThreadSpan(auditService.createSpan( + name, path1, path2)); + } + + @Override + public List createRequestHandlers() { + + // wire up the AWS SDK To call back into this class when + // preparing to make S3 calls. + List requestHandlers = new ArrayList<>(); + requestHandlers.add(new SdkRequestHandler()); + return requestHandlers; + } + + @Override + public TransferStateChangeListener createStateChangeListener() { + final AuditSpan span = getActiveThreadSpan(); + return new TransferStateChangeListener() { + @Override + public void transferStateChanged(final Transfer transfer, + final Transfer.TransferState state) { + setActiveThreadSpan(span); + } + }; + } + + /** + * Attach a reference to the active thread span, then + * invoke the same callback on that active thread. + */ + @Override + public T requestCreated( + final T request) { + AuditSpan span = getActiveThreadSpan(); + if (LOG.isTraceEnabled()) { + LOG.trace("Created Request {} in span {}", + analyzer.analyze(request), span); + } + request.addHandlerContext(AUDIT_SPAN_HANDLER_CONTEXT, span); + return span.requestCreated(request); + } + + @Override + public T beforeExecution(final T request) { + // identify the span + AuditSpanCallbacks span; + span = request.getHandlerContext(AUDIT_SPAN_HANDLER_CONTEXT); + if (span == null) { + LOG.warn("no span attached to request {}", analyzer.analyze(request)); + span = getActiveThreadSpan(); + } + return span.beforeExecution(request); + } + + /** + * Forward to active span. + * @param request request + * @param response response. + */ + @Override + public void afterResponse(final Request request, + final Response response) + throws AuditFailureException, SdkBaseException { + + getActiveThreadSpan().afterResponse(request, response); + } + + /** + * Forward to active span. + * @param request request + * @param response response. + * @param exception exception raised. + */ + @Override + public void afterError(final Request request, + final Response response, + final Exception exception) + throws AuditFailureException, SdkBaseException { + + getActiveThreadSpan().afterError(request, response, exception); + } + + /** + * Create and start an audit manager. + * @param conf configuration + * @param iostatistics IOStatistics source. + * @return audit manager. + */ + public static AuditManager createAuditManager( + Configuration conf, + IOStatisticsStore iostatistics) { + ActiveAuditManager auditManager = new ActiveAuditManager( + requireNonNull(iostatistics)); + auditManager.init(conf); + auditManager.start(); + return auditManager; + } + + /** + * Return a stub audit manager + * @return an audit manager. + */ + public static AuditManager stubAuditManager() { + return new NoopAuditManager(); + } + + /** + * Callbacks from the AWS SDK; all forward to the ActiveAuditManager. + */ + private class SdkRequestHandler extends RequestHandler2 { + + @Override + public AmazonWebServiceRequest beforeExecution( + final AmazonWebServiceRequest request) { + return ActiveAuditManager.this.beforeExecution(request); + } + + @Override + public void afterResponse(final Request request, + final Response response) { + ActiveAuditManager.this.afterResponse(request, response); + } + + @Override + public void afterError(final Request request, + final Response response, + final Exception e) { + ActiveAuditManager.this.afterError(request, response, e); + } + } + + + /** + * Wraps the plugged in spans with management of the active thread + * span, including switching to the unbounded span when a valid + * span is deactivated. + * Package private for testing. + */ + final class WrappingAuditSpan extends AbstractAuditSpanImpl { + + /** + * Inner span. + */ + private final AuditSpan span; + + /** + * Is this span considered valid? + */ + private final boolean isValid; + + /** + * Create, wrapped. + * @param span inner span. + * @param isValid is the span valid + */ + private WrappingAuditSpan( + final AuditSpan span, final boolean isValid) { + this.span = requireNonNull(span); + this.isValid = isValid; + } + + /** + * Is the span active? + * @return true if this span is the active one for the current thread. + */ + private boolean isActive() { + return this == getActiveThreadSpan(); + } + + /** + * Makes this the thread's active span and activate. + * If the span was already active: no-op. + */ + @Override + public AuditSpan activate() { + if (!isActive()) { + switchToActiveSpan(this); + span.activate(); + } + return this; + } + + /** + * Switch to the unbounded span and then deactivate this span. + * No-op for invalid spans, + * so as to prevent the unbounded span from being closed + * and everything getting very confused. + */ + @Override + public void deactivate() { + // no-op for invalid spans, + // so as to prevent the unbounded span from being closed + // and everything getting very confused. + if (!isValid || !isActive()) { + return; + } + // deactivate the span + span.deactivate(); + // and go to the unbounded one. + switchToActiveSpan(getUnboundedSpan()); + } + + /** + * Forward to the wrapped span. + * {@inheritDoc} + */ + @Override + public T requestCreated( + final T request) { + return span.requestCreated(request); + } + + /** + * This span is valid if the span isn't closed and the inner + * span is valid. + * @return true if the span is considered valid. + */ + @Override + public boolean isValidSpan() { + return isValid && span.isValidSpan(); + } + + /** + * Forward to the inner span. + * @param request request + * @param type of request + * @return an updated request. + */ + @Override + public T beforeExecution( + final T request) { + return span.beforeExecution(request); + } + + /** + * Forward to the inner span. + * @param request request + * @param response response. + */ + @Override + public void afterResponse(final Request request, + final Response response) { + span.afterResponse(request, response); + } + + /** + * Forward to the inner span. + * @param request request + * @param response response. + * @param exception exception raised. + */ + @Override + public void afterError(final Request request, + final Response response, + final Exception exception) { + span.afterError(request, response, exception); + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder( + "WrappingAuditSpan{"); + sb.append("span=").append(span); + sb.append(", valid=").append(isValidSpan()); + sb.append('}'); + return sb.toString(); + } + } + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditConstants.java new file mode 100644 index 0000000000000..1b15f2866451c --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditConstants.java @@ -0,0 +1,108 @@ +/* + * 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.audit; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Constants related to auditing. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public final class AuditConstants { + + private AuditConstants() { + } + + /** + * Name of class used for audit logs: {@value}. + */ + public static final String AUDIT_SERVICE_CLASSNAME = + "fs.s3a.audit.service.classname"; + + /** + * Classname of the logging auditor: {@value}. + */ + public static final String LOGGING_AUDIT_SERVICE = + "org.apache.hadoop.fs.s3a.audit.LoggingAuditor"; + + /** + * Classname of the No-op auditor: {@value}. + */ + public static final String NOOP_AUDIT_SERVICE = + "org.apache.hadoop.fs.s3a.audit.NoopAuditor"; + + /** + * List of extra AWS SDK request handlers: {@value}. + * These are added to the SDK request chain after + * any audit service. + */ + public static final String AUDIT_REQUEST_HANDLERS = + "fs.s3a.audit.request.handlers"; + + /** + * Should operations outside spans be rejected? + * This is for testing coverage of the span code; if used + * in production there's a risk of unexpected failures. + * {@value}. + */ + public static final String REJECT_OUT_OF_SPAN_OPERATIONS + = "fs.s3a.audit.reject.out.of.span.operations"; + + public static final String PATH_FORMAT + = "/audit/%1$s/s/%2$s/"; + + public static final String REFERRER_FORMAT + = "http://hadoop.apache.org/" + PATH_FORMAT; + + /** + * JobID query header: {@value}. + */ + public static final String JOB_ID = "job"; + + /** + * Principal query header: {@value}. + */ + public static final String PRINCIPAL = "pr"; + + public static final String OP = "op"; + + public static final String PATH = "path"; + + public static final String PATH2 = "path2"; + + public static final String PROCESS = "ps"; + + /** + * Thread header: {@value}. + */ + public static final String THREAD = "tr"; + + /** + * Thread header: {@value}. + */ + public static final String THREAD2 = "t2"; + + /** + * Span name used during initialization. + */ + public static final String INITIALIZE_SPAN = "initialize"; + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditFailureException.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditFailureException.java new file mode 100644 index 0000000000000..750b4991ad473 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditFailureException.java @@ -0,0 +1,43 @@ +/* + * 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.audit; + +import com.amazonaws.AmazonClientException; + +/** + * This is in the AWS exception tree so that exceptions raised in the + * AWS SDK are correctly reported up. + * {@code S3AUtils.translateException()} recognizes these exceptions + * and converts to AccessDeniedException. + */ +public class AuditFailureException extends AmazonClientException { + + public AuditFailureException(final String message, final Throwable t) { + super(message, t); + } + + public AuditFailureException(final String message) { + super(message); + } + + @Override + public boolean isRetryable() { + return false; + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditIntegration.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditIntegration.java new file mode 100644 index 0000000000000..da66b659e9c41 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditIntegration.java @@ -0,0 +1,73 @@ +/* + * 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.audit; + +import org.apache.hadoop.util.functional.CallableRaisingIOE; +import org.apache.hadoop.util.functional.InvocationRaisingIOE; + +/** + * Support for integrating auditing within the S3A code. + */ +public final class AuditIntegration { + + private AuditIntegration() { + } + + /** + * Given a callable, return a new callable which + * activates and deactivates the span around the inner invocation. + * @param auditSpan audit span + * @param operation operation + * @param type of result + * @return a new invocation. + */ + public static CallableRaisingIOE withinSpan( + AuditSpan auditSpan, + CallableRaisingIOE operation) { + return () -> { + auditSpan.activate(); + try { + return operation.apply(); + } finally { + auditSpan.deactivate(); + } + }; + } + + /** + * Given a invocation, return a new invocation which + * activates and deactivates the span around the inner invocation. + * @param auditSpan audit span + * @param operation operation + * @return a new invocation. + */ + public static InvocationRaisingIOE withinSpan( + AuditSpan auditSpan, + InvocationRaisingIOE operation) { + return () -> { + auditSpan.activate(); + try { + operation.apply();; + } finally { + auditSpan.deactivate(); + } + }; + } + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditInternalConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditInternalConstants.java new file mode 100644 index 0000000000000..e2fce316bd5ad --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditInternalConstants.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.audit; + +import com.amazonaws.handlers.HandlerContextKey; + +import org.apache.hadoop.classification.InterfaceAudience; + +/** + * Internal constants; not intended for public use. + */ +@InterfaceAudience.Private +public final class AuditInternalConstants { + + private AuditInternalConstants() { + } + + /** + * Handler key for audit span callbacks. + * This is used to bind the handler + */ + public static final HandlerContextKey + AUDIT_SPAN_HANDLER_CONTEXT = + new HandlerContextKey( + "org.apache.hadoop.fs.s3a.audit.AuditSpanCallbacks"); +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditManager.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditManager.java new file mode 100644 index 0000000000000..947544a47aa69 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditManager.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.audit; + +import java.util.List; + +import com.amazonaws.handlers.RequestHandler2; +import com.amazonaws.services.s3.transfer.internal.TransferStateChangeListener; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.service.Service; + +/** + * Interface for Audit Managers. + * The Audit Manager is the binding between S3AFS and the instantiated + * plugin point -it adds + *
    + *
  1. per-thread tracking of audit spans
  2. + *
  3. The wiring up to the AWS SDK
  4. + *
  5. State change tracking for copy operations (does not address issue)
  6. + *
+ */ +@InterfaceAudience.Private +public interface AuditManager extends Service, AuditSpanSource, + AuditSpanCallbacks { + + /** + * Get the wrapped active span. + * @return the currently active span. + */ + AuditSpan getActiveThreadSpan(); + + /** + * Create the request handler(s) for this audit service. + * The list returned is mutable; new handlers may be added. + * @return list of handlers for the SDK. + */ + List createRequestHandlers(); + + /** + * Return a transfer state change callback which + * fixes the active span context to be that in which + * the state change listener was created. + * This ensures that copy operations get bounded + * to the correct span. + * @return a state change listener. + */ + TransferStateChangeListener createStateChangeListener(); +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditSpan.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditSpan.java new file mode 100644 index 0000000000000..6c2acd29d7bfb --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditSpan.java @@ -0,0 +1,67 @@ +/* + * 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.audit; + +import java.io.Closeable; +import java.io.IOException; + +import com.amazonaws.AmazonWebServiceRequest; +import com.amazonaws.Request; +import com.amazonaws.Response; +import com.amazonaws.SdkBaseException; + +import org.apache.hadoop.fs.s3a.Retries; + +/** + * This is a span created by an {@link OperationAuditor}. + */ +public interface AuditSpan extends Closeable, AuditSpanCallbacks { + + /** + * Make this span active in the current thread. + * @return the activated span. + * This is makes it easy to use in try with resources + */ + default AuditSpan activate() { + return this; + } + + /** + * Deactivate the span in the current thread. + */ + void deactivate(); + + /** + * Close calls {@link #deactivate()}; subclasses may override + * but the audit manager's wrapping span will always relay to + * {@link #deactivate()} rather + * than call this method on the wrapped span. + */ + default void close() { + deactivate(); + } + + /** + * Is the span valid? False == this is a span to indicate unbonded. + * @return true if this span represents a real operation. + */ + default boolean isValidSpan() { + return true; + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditSpanCallbacks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditSpanCallbacks.java new file mode 100644 index 0000000000000..43adec86551b2 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditSpanCallbacks.java @@ -0,0 +1,91 @@ +/* + * 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.audit; + +import com.amazonaws.AmazonWebServiceRequest; +import com.amazonaws.Request; +import com.amazonaws.Response; +import com.amazonaws.SdkBaseException; + +import org.apache.hadoop.fs.s3a.Retries; + +/** + * Callbacks for audit spans. This is implemented + * in the span manager as well as individual audit spans. + * If any of the code in a callback raises an InterruptedException, + * it must be caught and {@code Thread.interrupt()} called to + * redeclare the thread as interrupted. The AWS SDK will + * detect this and raise an exception. + */ +public interface AuditSpanCallbacks { + + /** + * Callback when a request is created in the S3A code. + * It is not invoked on any AWS requests created in the SDK. + * Avoid raising exceptions or talking to any remote service; + * this callback is for annotation rather than validation. + * @param request request request. + * @param type of request + * @return the request, possibly modified. + */ + default T requestCreated(T request) { + return request; + } + + /** + * Preflight preparation of AWS request. + * @param request request + * @param type of request + * @return an updated request. + * @throws AuditFailureException for generic audit failures + * @throws SdkBaseException for other reasons. + */ + @Retries.OnceRaw + default T beforeExecution(T request) + throws AuditFailureException, SdkBaseException { + return request; + } + + /** + * Callback after S3 responded to a request. + * @param request request + * @param response response. + * @throws AuditFailureException for generic audit failures + * @throws SdkBaseException for other reasons. + */ + default void afterResponse(Request request, + Response response) + throws AuditFailureException, SdkBaseException { + } + + /** + * Callback after a request resulted in an error. + * @param request request + * @param response response. + * @param exception exception raised. + * @throws AuditFailureException for generic audit failures + * @throws SdkBaseException for other reasons. + */ + default void afterError(Request request, + Response response, + Exception exception) + throws AuditFailureException, SdkBaseException{ + } + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditSpanSource.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditSpanSource.java new file mode 100644 index 0000000000000..5248c291779f1 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditSpanSource.java @@ -0,0 +1,45 @@ +/* + * 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.audit; + +import javax.annotation.Nullable; +import java.io.IOException; + +/** + * A source of audit spans. + */ +public interface AuditSpanSource { + + /** + * Create a span for an operation. + * + * All operation names SHOULD come from + * {@code StoreStatisticNames} or + * {@code StreamStatisticNames}. + * @param name operation name. + * @param path1 first path of operation + * @param path2 second path of operation + * @return a span for the audit + * @throws IOException failure + */ + AuditSpan createSpan(String name, + @Nullable String path1, + @Nullable String path2) + throws IOException; +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AwsS3Verbs.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AwsS3Verbs.java new file mode 100644 index 0000000000000..d5757db7b8324 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AwsS3Verbs.java @@ -0,0 +1,50 @@ +/* + * 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.audit; + +/** + * Various verbs in the S3 logs. + * https://stackoverflow.com/questions/42707878/amazon-s3-logs-operation-definition + */ +public final class AwsS3Verbs { + + private AwsS3Verbs() { + } + + public static final String DELETE = "REST.DELETE.OBJECT"; + public static final String COPY = "REST.COPY.OBJECT"; + public static final String DELETE_BULK = "REST.POST.MULTI_OBJECT_DELETE"; + public static final String DELETE_BULK_ENTRY = "BATCH.DELETE.OBJECT"; + public static final String GET = "REST.GET.OBJECT"; + public static final String HEAD = "REST.HEAD.OBJECT"; + public static final String GET_ACL = "REST.GET.ACL"; + public static final String GET_LOGGING_STATUS = "REST.GET.LOGGING_STATUS"; + public static final String LIST = "REST.GET.BUCKET"; + public static final String MULTIPART_UPLOAD_START = "REST.POST.UPLOADS"; + public static final String MULTIPART_UPLOAD_PART = "REST.PUT.PART"; + public static final String MULTIPART_UPLOAD_COMPLETE = "REST.POST.UPLOAD"; + public static final String MULTIPART_UPLOADS_LIST = "REST.GET.UPLOADS"; + public static final String MULTIPART_UPLOAD_ABORT = "REST.DELETE.UPLOAD"; + public static final String PUT = "REST.PUT.OBJECT"; + public static final String REST_GET_POLICY_STATUS = "REST.GET.POLICY_STATUS"; + public static final String REST_GET_PUBLIC_ACCESS_BLOCK = + "REST.GET.PUBLIC_ACCESS_BLOCK"; + public static final String REST_GET_TAGGING = "REST.GET.TAGGING"; + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/CommonAuditContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/CommonAuditContext.java new file mode 100644 index 0000000000000..4642326627d0a --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/CommonAuditContext.java @@ -0,0 +1,147 @@ +/* + * 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.audit; + +import java.time.LocalDateTime; +import java.time.format.DateTimeFormatter; +import java.time.format.DateTimeFormatterBuilder; +import java.time.temporal.ChronoField; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Supplier; + +import static org.apache.hadoop.fs.s3a.audit.AuditConstants.PROCESS; +import static org.apache.hadoop.fs.s3a.audit.AuditConstants.THREAD; + +/** + * The common audit context is a map of common context information + * which can be used with any audit span. + * Audit spans will be created with a reference to the current + * context of their thread; + * That reference is retained even as they are moved across threads, so + * context information (including initial thread ID from + * {@link #THREAD_ID_COUNTER}) can be retrieved/used. + */ +public final class CommonAuditContext { + + private CommonAuditContext() { + } + + /** + * Process ID; currently built from UUID and timestamp + */ + public static final String PROCESS_ID; + + static { + final LocalDateTime now = LocalDateTime.now(); + final DateTimeFormatter formatter = new DateTimeFormatterBuilder() + .appendValue(ChronoField.HOUR_OF_DAY, 2) + .appendLiteral('.') + .appendValue(ChronoField.MINUTE_OF_HOUR, 2) + .appendLiteral('.') + .appendValue(ChronoField.SECOND_OF_MINUTE, 2) + .toFormatter(); + PROCESS_ID = UUID.randomUUID().toString() + "-" + now.format(formatter); + + } + + /** + * Counter of thread IDs. + */ + private static final AtomicLong THREAD_ID_COUNTER = new AtomicLong(); + + /** + * Map of data. Concurrent so when shared across threads + * there are no problems. + * Supplier operations must themselves be threadsafe. + */ + private final Map> map = new ConcurrentHashMap<>(1); + + /** + * Put a context entry. + * @param key key + * @param value new value + * @return old value or null + */ + public Supplier put(String key, String value) { + return map.put(key, () -> value); + } + + /** + * Remove a context entry. + * @param key key + */ + public void remove(String key) { + map.remove(key); + } + + /** + * Get a context entry. + * @param key key + * @return value or null + */ + public String get(String key) { + return map.get(key).get(); + } + + /** + * Does the context contain a specific key + * @param key key + * @return true if it is in the context. + */ + public boolean containsKey(String key) { + return map.containsKey(key); + } + + /** + * Thread local context. + * Use a weak reference just to keep memory costs down. + * The S3A committers all have a strong reference, so if they are + * retained, context is retained. + */ + private static final ThreadLocal activeContext = + ThreadLocal.withInitial(() -> createInstance()); + + /** + * Demand invoked to create an instance for this thread. + * Sets a new thread ID for it. + * @return an instance. + */ + private static CommonAuditContext createInstance() { + CommonAuditContext context = new CommonAuditContext(); + context.put(PROCESS, PROCESS_ID); + context.put(THREAD, Long.toHexString(nextThreadId())); + return context; + } + + /** + * Get the current common context. Thread local. + * @return the context of this thread. + */ + public static CommonAuditContext currentContext() { + return activeContext.get(); + } + + private static long nextThreadId() { + return THREAD_ID_COUNTER.incrementAndGet(); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/HttpReferrerAuditEntry.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/HttpReferrerAuditEntry.java new file mode 100644 index 0000000000000..cef3ba76380ef --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/HttpReferrerAuditEntry.java @@ -0,0 +1,336 @@ +/* + * 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.audit; + +import java.net.URI; +import java.net.URISyntaxException; +import java.util.HashMap; +import java.util.Locale; +import java.util.Map; +import java.util.StringJoiner; +import java.util.stream.Collectors; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.fs.s3a.impl.LogExactlyOnce; + +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.fs.s3a.audit.AuditConstants.*; + +/** + * Contains all the logic for generating an HTTP "Referer" + * entry; includes escaping query params. + */ +public final class HttpReferrerAuditEntry { + + private static final Logger LOG = + LoggerFactory.getLogger(HttpReferrerAuditEntry.class); + + public static final String AUTHORITY = "hadoop.apache.org"; + + private final LogExactlyOnce warnOfUrlCreation = new LogExactlyOnce( + LOG); + + private final String context; + + private final String operationName; + + private final String operationId; + + private final String path1; + + private final String path2; + + private final String header; + + private final Map parameters; + + /** + * Instantiate. + * + * Context and operationId are expected to be well formed + * numeric/hex strings, at least adequate to be + * used as individual path elements in a URL. + * @param context context as string + * @param operationId operation ID as a string + * @param operationName operation name + * @param path1 optional first path + * @param path2 optional second path + * @param attributes map of attributes to add as query parameters. + * @param attributes2 second map of attributes to add as query parameters. + */ + private HttpReferrerAuditEntry( + final Builder builder) { + this.context = requireNonNull(builder.context); + this.operationName = requireNonNull(builder.operationName); + this.operationId = requireNonNull(builder.operationId); + this.path1 = builder.path1; + this.path2 = builder.path2; + // clone params + parameters = new HashMap<>(); + add(parameters, builder.attributes); + add(parameters, builder.attributes2); + addParameter(parameters, OP, operationName); + addParameter(parameters, PATH, path1); + addParameter(parameters, PATH2, path2); + // build the referrer up. so as to find/report problems early + header = buildHttpReferrerString(); + } + + /** + * Build the referrer string. + * If there is an error creating the string it will be logged once + * per entry, and "" returned. + * @return a referrer string or "" + */ + private String buildHttpReferrerString() { + final String queries; + // queries as ? params. + queries = parameters.entrySet().stream() + .map(e -> e.getKey() + "=" + e.getValue()) + .collect(Collectors.joining("&")); + String h; + try { + final URI uri = new URI("https", AUTHORITY, + String.format(Locale.ENGLISH, PATH_FORMAT, + context, operationId), + queries, + null); + h = uri.toASCIIString(); + } catch (URISyntaxException e) { + warnOfUrlCreation.warn("Failed to build URI for {}/{}", e); + h = ""; + } + return h; + } + + /** + * Add the map of attributes to the map of request parameters + * @param requestParams HTTP request parameters + * @param attributes attributes + */ + private void add(final Map requestParams, + final Map attributes) { + if (attributes != null) { + attributes.entrySet() + .forEach(e -> addParameter(requestParams, e.getKey(), e.getValue())); + } + } + + /** + * Add a query parameter if not null/empty + * There's no need to escape here as it is done in the URI + * constructor. + * @param requestParams query map + * @param key query key + * @param value query value + */ + private void addParameter(Map requestParams, + String key, + String value) { + if (StringUtils.isNotEmpty(value)) { + requestParams.put(key, value); + } + } + + public String getContext() { + return context; + } + + public String getOperationName() { + return operationName; + } + + public String getOperationId() { + return operationId; + } + + public String getPath1() { + return path1; + } + + public String getPath2() { + return path2; + } + + /** + * Get the header or "" if there were problems creating it. + * @return the referrer header + */ + public String getReferrerHeader() { + return buildHttpReferrerString(); + } + + @Override + public String toString() { + return new StringJoiner(", ", + HttpReferrerAuditEntry.class.getSimpleName() + "[", "]") + .add(header) + .toString(); + } + + /** + * Perform any escaping to valid path elements in advance of + * new URI() doing this itself. Only path separators need to + * be escaped/converted at this point. + * @param source source string + * @return an escaped path element. + */ + public static String escapeToPathElement(CharSequence source) { + int len = source.length(); + StringBuilder r = new StringBuilder(len); + for (int i = 0; i < len; i++) { + char c = source.charAt(i); + String s = Character.toString(c); + switch (c) { + case '/': + case '@': + s = "+"; + break; + default: + break; + } + r.append(s); + } + return r.toString(); + + } + + /** + * Get a builder. + * @return a new builder. + */ + public static Builder builder() { + return new Builder(); + } + + /** + * Builder. + * + * Context and operationId are expected to be well formed + * numeric/hex strings, at least adequate to be + * used as individual path elements in a URL. + * @param context + * @param operationId operation ID as a string + * @param operationName operation name + * @param path1 optional first path + * @param path2 optional second path + * @param attributes map of attributes to add as query parameters. + * @param attributes2 second map of attributes to add as query parameters. + */ + public static final class Builder { + + private String context; + + private String operationName; + + private String operationId; + + private String path1; + + private String path2; + + private Map attributes; + private Map attributes2; + + private int i; + private Builder() { + } + + /** + * Build. + * @return + */ + public HttpReferrerAuditEntry build() { + return new HttpReferrerAuditEntry(this); + } + + /** + * Set context as string + * @param value context + * @return the builder + */ + public Builder withContext(final String value) { + context = value; + return this; + } + + /** + * Set Operation name. + * @param value new value + * @return the builder + */ + public Builder withOperationName(final String value) { + operationName = value; + return this; + } + + /** + * Set ID. + * @param value new value + * @return the builder + */ + public Builder withOperationId(final String value) { + operationId = value; + return this; + } + + /** + * Set Path1 of operation. + * @param value new value + * @return the builder + */ + public Builder withPath1(final String value) { + path1 = value; + return this; + } + + /** + * Set Path2 of operation. + * @param value new value + * @return the builder + */ + public Builder withPath2(final String _path2) { + path2 = _path2; + return this; + } + + /** + * Set map 1 of attributes (common context) + * @param value new value + * @return the builder + */ + public Builder withAttributes(final Map value) { + attributes = value; + return this; + } + + /** + * Set map 2 of attributes (span attributes) + * @param value new value + * @return the builder + */ + public Builder withAttributes2(final Map value) { + attributes2 = value; + return this; + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/LoggingAuditor.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/LoggingAuditor.java new file mode 100644 index 0000000000000..b38a2a181dc93 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/LoggingAuditor.java @@ -0,0 +1,366 @@ +/* + * 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.audit; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; + +import com.amazonaws.AmazonWebServiceRequest; +import com.amazonaws.Request; +import com.amazonaws.Response; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.s3a.Statistic; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; +import org.apache.hadoop.security.UserGroupInformation; + +import static org.apache.hadoop.fs.s3a.audit.AWSRequestAnalyzer.isRequestNotAlwaysInSpan; +import static org.apache.hadoop.fs.s3a.audit.AuditConstants.PRINCIPAL; +import static org.apache.hadoop.fs.s3a.audit.CommonAuditContext.PROCESS_ID; +import static org.apache.hadoop.fs.s3a.audit.CommonAuditContext.currentContext; +import static org.apache.hadoop.fs.s3a.commit.CommitUtils.extractJobID; +import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.HEADER_REFERRER; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_FAILURES; + +/** + * The LoggingAuditor logs operations at DEBUG (in SDK Request) and + * in span lifecycle and S3 request class construction at TRACE. + * The context information is added as the HTTP referrer. + */ +@InterfaceAudience.Private +public final class LoggingAuditor + extends AbstractOperationAuditor { + + /** + * What to look for in logs for ops outside any audit. + * {@value}. + */ + public static final String UNAUDITED_OPERATION = "unaudited operation"; + + /** + * This is where the context gets logged to. + */ + private static final Logger LOG = + LoggerFactory.getLogger(LoggingAuditor.class); + + /** + * Counter for next operation in this service. + * Initial value is what will be used for the span ID when there + * is no active request span. + */ + private final AtomicLong nextOperationId = new AtomicLong(1); + + /** + * Some basic analysis for the logs. + */ + private final AWSRequestAnalyzer analyzer = new AWSRequestAnalyzer(); + + /** + * Default span to use when there is no other. + */ + private final AuditSpan warningSpan; + + /** + * Prefix built up to use in log messages. + */ + private final String processID; + + /** + * Should out of scope ops be rejected? + */ + private boolean rejectOutOfSpan; + + /** + * Map of attributes which will be added to all operations. + */ + private final Map attributes = new HashMap<>(); + + /** + * UGI principal at time of creation. + * This is mapped into the common context if it is not already set there + * when a span is created. + */ + private final String principal; + + /** + * Create an operation ID. The nature of it should be opaque. + * @return an ID for the constructor. + */ + private long newOperationId() { + return nextOperationId.getAndIncrement(); + } + + public LoggingAuditor(final String name, + final IOStatisticsStore iostatistics) { + + super(name, iostatistics); + processID = PROCESS_ID; + final CommonAuditContext currentContext = currentContext(); + warningSpan = new WarningSpan("Operation without an audit span", + currentContext, newOperationId(), null, null); + // add the principal + String p; + try { + UserGroupInformation ugi = UserGroupInformation.getCurrentUser(); + p = ugi.getUserName(); + attributes.put(PRINCIPAL, p); + } catch (IOException ignored) { + p = ""; + } + principal = p; + } + + /** + * Service init, look for jobID and attach as an attribute in log entries. + * @param conf configuration + * @throws Exception failure + */ + @Override + protected void serviceInit(final Configuration conf) throws Exception { + super.serviceInit(conf); + rejectOutOfSpan = conf.getBoolean( + AuditConstants.REJECT_OUT_OF_SPAN_OPERATIONS, false); + // attach the job ID if there is one in the configuration used + // to create this file. + String jobID = extractJobID(conf); + if (jobID != null) { + attributes.put(AuditConstants.JOB_ID, jobID); + } + } + + @Override + public AuditSpan createSpan(final String name, + @Nullable final String path1, + @Nullable final String path2) { + getIOStatistics().incrementCounter( + Statistic.AUDIT_SPAN_START.getSymbol()); + final LoggingAuditSpan span = new LoggingAuditSpan(name, + prepareActiveContext(), newOperationId(), path1, path2); + span.start(); + return span; + } + + /** + * Get/Prepare the active context for a span. + * Includes patching in the principal if unset. + * @return the common audit context. + */ + private CommonAuditContext prepareActiveContext() { + final CommonAuditContext currentContext = currentContext(); + // put principal in current context if unset. + if (!currentContext.containsKey(PRINCIPAL)) { + currentContext.put(PRINCIPAL, principal); + } + + return currentContext; + } + + @Override + public AuditSpan getUnbondedSpan() { + return warningSpan; + } + + public String getProcessID() { + return processID; + } + + /** + * Span which logs. + */ + private class LoggingAuditSpan extends AbstractAuditSpanImpl { + + private final HttpReferrerAuditEntry entry; + + private final String operationName; + + private final String description; + + private final CommonAuditContext context; + + private final String id; + + private LoggingAuditSpan( + final String name, + final CommonAuditContext context, + final long operationId, + final String path1, final String path2) { + this.operationName = name; + this.context = context; + this.id = String.format("%s-%08d", getProcessID(), operationId); + entry = HttpReferrerAuditEntry.builder() + .withContext(processID) + .withOperationId(String.format("%08x", operationId)) + .withOperationName(name) + .withPath1(path1) + .withPath2(path2) + .withAttributes(attributes) + .build(); + this.description = entry.getReferrerHeader(); + } + + public void start() { + LOG.trace("{} Start {}", getId(), getDescription()); + } + + protected String getOperationName() { + return operationName; + } + + protected String getDescription() { + return description; + } + + protected String getId() { + return id; + } + + @Override + public AuditSpan activate() { + LOG.trace("{} Activate {}", id, description); + return this; + } + + @Override + public void deactivate() { + LOG.trace("{} Deactivate {}", id, description); + } + + @Override + public T beforeExecution( + final T request) { + if (LOG.isDebugEnabled()) { + LOG.debug("{} Executing {} with {}; {}", id, + getOperationName(), + analyzer.analyze(request), + getDescription()); + } + // add the referrer header + request.putCustomRequestHeader(HEADER_REFERRER, entry.getReferrerHeader()); + return request; + } + + @Override + public void afterResponse(final Request request, + final Response response) { + final Object awsResponse = response.getAwsResponse(); + } + + @Override + public void afterError(final Request request, + final Response response, + final Exception exception) { + + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder( + "LoggingAuditSpan{"); + sb.append(", id='").append(id).append('\''); + sb.append("description='").append(description).append('\''); + sb.append('}'); + return sb.toString(); + } + } + + /** + * Span which logs at WARN; used to highlight spans + * without a containing span. + */ + private final class WarningSpan extends LoggingAuditSpan { + + private WarningSpan( + final String name, + final CommonAuditContext context, + final long operationId, + final String path1, final String path2) { + super(name, context, operationId, path1, path2); + } + + @Override + public void start() { + LOG.warn("{} Start {}", getId(), getDescription()); + } + + @Override + public AuditSpan activate() { + LOG.warn("{} Activate {}", getId(), getDescription()); + return this; + } + + @Override + public boolean isValidSpan() { + return false; + } + + @Override + public T requestCreated( + final T request) { + String error = "Creating a request outside an audit span " + + analyzer.analyze(request); + LOG.info(error); + if (LOG.isDebugEnabled()) { + LOG.debug(error, new AuditFailureException("unaudited")); + } + return request; + } + + /** + * Handle requests made without a real context by logging and + * increment the failure count. + * Some requests (e.g. copy part) are not expected in spans due + * to how they are executed; these do not trigger failures. + * @param request request + * @param type of request + * @return an updated request. + * @throws AuditFailureException if failure is enabled. + */ + @Override + public T beforeExecution( + final T request) { + + getIOStatistics().incrementCounter( + Statistic.AUDIT_SPAN_START.getSymbol() + SUFFIX_FAILURES); + String error = "Executing a request outside an audit span " + + analyzer.analyze(request); + LOG.warn("{} {}", + getId(), error); + final String unaudited = getId() + " " + + UNAUDITED_OPERATION + " " + error; + if (isRequestNotAlwaysInSpan(request)) { + // can get by auditing during a copy, so don't overreact + LOG.debug(unaudited); + } else { + final RuntimeException ex = new AuditFailureException(unaudited); + LOG.info(unaudited, ex); + if (rejectOutOfSpan) { + throw ex; + } + } + // now hand off to the superclass for its normal preparation + return super.beforeExecution(request); + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/NoopAuditManager.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/NoopAuditManager.java new file mode 100644 index 0000000000000..fb44bf8e271c7 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/NoopAuditManager.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.audit; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import com.amazonaws.handlers.RequestHandler2; +import com.amazonaws.services.s3.transfer.Transfer; +import com.amazonaws.services.s3.transfer.internal.TransferStateChangeListener; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.service.AbstractService; + +/** + * Simple No-op audit manager for use before a real + * audit chain is set up. + * It does have the service lifecycle, so do + * create a unique instance whenever used. + */ +@InterfaceAudience.Private +public class NoopAuditManager extends AbstractService + implements AuditManager { + + public NoopAuditManager() { + super("NoopAuditManager"); + } + + @Override + public AuditSpan getActiveThreadSpan() { + return NoopSpan.INSTANCE; + } + + @Override + public AuditSpan createSpan(final String name, + @Nullable final String path1, + @Nullable final String path2) throws IOException { + return new NoopSpan(name, path1, path2); + } + + @Override + public List createRequestHandlers() { + return new ArrayList<>(); + } + + @Override + public TransferStateChangeListener createStateChangeListener() { + return new TransferStateChangeListener() { + public void transferStateChanged(final Transfer transfer, + final Transfer.TransferState state) { + } + }; + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/NoopAuditor.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/NoopAuditor.java new file mode 100644 index 0000000000000..53f73089c3501 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/NoopAuditor.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.audit; + +import javax.annotation.Nullable; +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; + +/** + * An audit service which returns the {@link NoopSpan}. + */ +public class NoopAuditor extends AbstractOperationAuditor { + + public NoopAuditor(final String name, + final IOStatisticsStore iostatistics) { + super(name, iostatistics); + } + + @Override + public AuditSpan createSpan(final String name, + @Nullable final String path1, + @Nullable final String path2) throws IOException { + return new NoopSpan(name, path1, path2); + } + + @Override + public AuditSpan getUnbondedSpan() { + return NoopSpan.INSTANCE; + } + + /** + * Create, init and start an instance. + * @param conf configuration. + * @return a started instance. + */ + public static OperationAuditor newInstance(Configuration conf) { + NoopAuditor noop = new NoopAuditor("noop", + IOStatisticsBinding.emptyStatisticsStore()); + noop.init(conf); + noop.start(); + return noop; + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/NoopSpan.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/NoopSpan.java new file mode 100644 index 0000000000000..da365d35a7cd0 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/NoopSpan.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.audit; + +/** + * Simple no-op span. + */ +public final class NoopSpan implements AuditSpan { + + private final String name; + + private final String path1; + + private final String path2; + + /** + * Static public instance. + */ + public static final NoopSpan INSTANCE = new NoopSpan(); + + public NoopSpan(final String name, final String path1, final String path2) { + this.name = name; + this.path1 = path1; + this.path2 = path2; + } + + NoopSpan() { + this("no-op", null, null); + } + + @Override + public void deactivate() { + // no-op + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder("NoopSpan{"); + sb.append("name='").append(name).append('\''); + sb.append(", path1='").append(path1).append('\''); + sb.append(", path2='").append(path2).append('\''); + sb.append('}'); + return sb.toString(); + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/NoopSpanSource.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/NoopSpanSource.java new file mode 100644 index 0000000000000..eafc079c45892 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/NoopSpanSource.java @@ -0,0 +1,44 @@ +/* + * 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.audit; + +import javax.annotation.Nullable; +import java.io.IOException; + +/** + * Simple no-op source, which always returns + * a no-op span. + */ +public final class NoopSpanSource implements AuditSpanSource { + + /** + * Sole public instance. + */ + public static final NoopSpanSource INSTANCE = new NoopSpanSource(); + + private NoopSpanSource() { + } + + @Override + public AuditSpan createSpan(final String name, + @Nullable final String path1, + @Nullable final String path2) throws IOException { + return new NoopSpan(name, path1, path2); + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/OperationAuditor.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/OperationAuditor.java new file mode 100644 index 0000000000000..1738374f31873 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/OperationAuditor.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.audit; + +import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.service.Service; + +/** + * Interfaces for audit services to implement. + */ +public interface OperationAuditor extends Service, + IOStatisticsSource, AuditSpanSource { + + /** + * Get the unbonded span to use after deactivating an active + * span. + * @return a span. + */ + AuditSpan getUnbondedSpan(); +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/package-info.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/package-info.java new file mode 100644 index 0000000000000..a7da624eead37 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/package-info.java @@ -0,0 +1,41 @@ +/* + * 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. + */ + +/** + * Support for auditing and ultimately tracing operations. + * This is a public API for extension points, e.g. opentracing. + * However, it is very unstable as we evolve how best to audit/trace + * operation. + * + * An audit service is instantiated when an S3A Filesystem is initialized during + * creation. + * The choice of service is determined in the configuration option + * {@link org.apache.hadoop.fs.s3a.audit.AuditConstants#AUDIT_SERVICE_CLASSNAME}. + * The service MUST implement the interface + * {@link org.apache.hadoop.fs.s3a.audit.OperationAuditor} + * to provide an {@link org.apache.hadoop.fs.s3a.audit.AuditSpan} whenever + * an operation is started through a public FileSystem API call + * (+some other operations). + */ + +@InterfaceAudience.LimitedPrivate("auditing extensions") +@InterfaceStability.Unstable +package org.apache.hadoop.fs.s3a.audit; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java index f73689979ec25..e3e4aceee0ff3 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java @@ -18,6 +18,7 @@ package org.apache.hadoop.fs.s3a.commit; +import javax.annotation.Nullable; import java.io.FileNotFoundException; import java.io.IOException; import java.text.DateFormat; @@ -30,6 +31,9 @@ import java.util.concurrent.TimeUnit; import com.amazonaws.services.s3.model.MultipartUpload; + +import org.apache.hadoop.fs.s3a.audit.AuditConstants; +import org.apache.hadoop.fs.s3a.audit.CommonAuditContext; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; @@ -42,8 +46,9 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.s3a.Invoker; import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.fs.s3a.audit.AuditSpan; +import org.apache.hadoop.fs.s3a.audit.AuditSpanSource; import org.apache.hadoop.fs.s3a.commit.files.PendingSet; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; import org.apache.hadoop.fs.s3a.commit.files.SuccessData; @@ -60,11 +65,13 @@ import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.util.DurationInfo; import org.apache.hadoop.util.concurrent.HadoopExecutors; +import org.apache.hadoop.util.functional.InvocationRaisingIOE; import static org.apache.hadoop.fs.s3a.Constants.THREAD_POOL_SHUTDOWN_DELAY_SECONDS; import static org.apache.hadoop.fs.s3a.Invoker.ignoreIOExceptions; import static org.apache.hadoop.fs.s3a.S3AUtils.*; import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_COMMIT_JOB; +import static org.apache.hadoop.fs.s3a.audit.CommonAuditContext.currentContext; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; import static org.apache.hadoop.fs.s3a.commit.CommitUtils.*; import static org.apache.hadoop.fs.s3a.commit.CommitUtilsWithMR.*; @@ -129,6 +136,8 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter */ private final JobUUIDSource uuidSource; + private final CommonAuditContext commonAuditContext; + /** * Has this instance been used for job setup? * If so then it is safe for a locally generated @@ -175,6 +184,11 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter private final CommitterStatistics committerStatistics; + /** + * Source of Audit spans. + */ + private final AuditSpanSource auditSpanSource; + /** * Create a committer. * This constructor binds the destination directory and configuration, but @@ -203,6 +217,13 @@ protected AbstractS3ACommitter( LOG.debug("{} instantiated for job \"{}\" ID {} with destination {}", role, jobName(context), jobIdString(context), outputPath); S3AFileSystem fs = getDestS3AFS(); + // set this thread's context with the job ID. + // audit spans created in this thread will pick + // up this value. + this.commonAuditContext = currentContext(); + updateCommonContext(); + // the filesystem is the span source, always. + auditSpanSource = fs.getAuditSpanSource(); this.createJobMarker = context.getConfiguration().getBoolean( CREATE_SUCCESSFUL_JOB_OUTPUT_DIR_MARKER, DEFAULT_CREATE_SUCCESSFUL_JOB_DIR_MARKER); @@ -535,6 +556,8 @@ public void setupJob(JobContext context) throws IOException { @Override public void setupTask(TaskAttemptContext context) throws IOException { TaskAttemptID attemptID = context.getTaskAttemptID(); + updateCommonContext(); + try (DurationInfo d = new DurationInfo(LOG, "Setup Task %s", attemptID)) { // reject attempts to set up the task where the output won't be @@ -947,11 +970,11 @@ public void cleanupJob(JobContext context) throws IOException { protected void maybeIgnore( boolean suppress, String action, - Invoker.VoidOperation operation) throws IOException { + InvocationRaisingIOE operation) throws IOException { if (suppress) { ignoreIOExceptions(LOG, action, "", operation); } else { - operation.execute(); + operation.apply(); } } @@ -1361,6 +1384,43 @@ public String toString() { } } + /** + * Add jobID to current context. + */ + protected final void updateCommonContext() { + currentContext().put(AuditConstants.JOB_ID, uuid); + } + + /** + * Remove JobID from the current thread's context. + */ + protected final void resetCommonContext() { + currentContext().put(AuditConstants.JOB_ID, uuid); + } + + protected AuditSpanSource getAuditSpanSource() { + return auditSpanSource; + } + + /** + * Start an operation; retrieve an audit span. + * + * All operation names SHOULD come from + * {@code StoreStatisticNames} or + * {@code StreamStatisticNames}. + * @param name operation name. + * @param path1 first path of operation + * @param path2 second path of operation + * @return a span for the audit + * @throws IOException failure + */ + protected AuditSpan startOperation(String name, + @Nullable String path1, + @Nullable String path2) + throws IOException { + return getAuditSpanSource().createSpan(name, path1, path2); + } + /** * State of the active commit operation. * diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java index c09278fbdcf32..8c1af65e3de05 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java @@ -48,10 +48,11 @@ import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3AUtils; -import org.apache.hadoop.fs.s3a.WriteOperationHelper; +import org.apache.hadoop.fs.s3a.WriteOperations; import org.apache.hadoop.fs.s3a.commit.files.PendingSet; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; import org.apache.hadoop.fs.s3a.commit.files.SuccessData; +import org.apache.hadoop.fs.s3a.impl.AbstractStoreOperation; import org.apache.hadoop.fs.s3a.impl.HeaderProcessing; import org.apache.hadoop.fs.s3a.impl.InternalConstants; import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; @@ -65,11 +66,13 @@ import static java.util.Objects.requireNonNull; 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.Statistic.COMMITTER_MATERIALIZE_FILE; import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_STAGE_FILE_UPLOAD; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration; +import static org.apache.hadoop.util.functional.RemoteIterators.cleanupRemoteIterator; /** * The implementation of the various actions a committer needs. @@ -81,7 +84,8 @@ * duplicate that work. * */ -public class CommitOperations implements IOStatisticsSource { +public class CommitOperations extends AbstractStoreOperation + implements IOStatisticsSource { private static final Logger LOG = LoggerFactory.getLogger( CommitOperations.class); @@ -96,7 +100,7 @@ public class CommitOperations implements IOStatisticsSource { /** * Write operations for the destination fs. */ - private final WriteOperationHelper writeOperations; + private final WriteOperations writeOperations; /** * Filter to find all {code .pendingset} files. @@ -113,21 +117,29 @@ public class CommitOperations implements IOStatisticsSource { /** * Instantiate. * @param fs FS to bind to + * @throws IOException failure to bind. */ - public CommitOperations(S3AFileSystem fs) { + public CommitOperations(S3AFileSystem fs) throws IOException { this(requireNonNull(fs), fs.newCommitterStatistics()); } /** - * Instantiate. + * Instantiate. This creates a new audit span for + * the commit operations. * @param fs FS to bind to * @param committerStatistics committer statistics + * @throws IOException failure to bind. */ public CommitOperations(S3AFileSystem fs, - CommitterStatistics committerStatistics) { - this.fs = requireNonNull(fs); + CommitterStatistics committerStatistics) throws IOException { + super(requireNonNull(fs).createStoreContext()); + this.fs = fs; statistics = requireNonNull(committerStatistics); - writeOperations = fs.getWriteOperationHelper(); + // create a span + writeOperations = fs.createWriteOperationHelper( + fs.getAuditSpanSource().createSpan( + COMMITTER_COMMIT_JOB.getSymbol(), + "/", null)); } /** @@ -365,6 +377,7 @@ public MaybeIOE abortAllSinglePendingCommits(Path pendingDir, } } } + cleanupRemoteIterator(pendingFiles); return outcome; } @@ -388,7 +401,7 @@ protected RemoteIterator ls(Path path, boolean recursive) */ public List listPendingUploadsUnderPath(Path dest) throws IOException { - return fs.listMultipartUploads(fs.pathToKey(dest)); + return writeOperations.listMultipartUploads(fs.pathToKey(dest)); } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitUtils.java index 9c684c77e1b25..6ea65b9132f8f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitUtils.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitUtils.java @@ -125,5 +125,29 @@ public static void validateCollectionClass(Iterable it, Class classname) } } + /** + * Extract the job ID from a configuration. + * @param conf configuration + * @return a job ID or null. + */ + public static String extractJobID(Configuration conf) { + + String jobUUID = conf.getTrimmed(FS_S3A_COMMITTER_UUID, ""); + + if (!jobUUID.isEmpty()) { + return jobUUID; + } + // there is no job UUID. + // look for one from spark + jobUUID = conf.getTrimmed(SPARK_WRITE_UUID, ""); + if (!jobUUID.isEmpty()) { + return jobUUID; + } + jobUUID = conf.getTrimmed(MR_JOB_ID, ""); + if (!jobUUID.isEmpty()) { + return jobUUID; + } + return null; + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/InternalCommitterConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/InternalCommitterConstants.java index 461c9a5e646c3..fcafdd1ed1280 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/InternalCommitterConstants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/InternalCommitterConstants.java @@ -124,4 +124,10 @@ private InternalCommitterConstants() { "Job/task context does not contain a unique ID in " + SPARK_WRITE_UUID; + /** + * The MR job ID; copies from MRJobConfig so that it can be + * referred to without needing hadoop-mapreduce on the classpath. + */ + public static final String MR_JOB_ID = "mapreduce.job.id"; + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java index 41f95c8620692..8cc9922e72b1e 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java @@ -27,7 +27,7 @@ import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.Statistic; import org.apache.hadoop.fs.s3a.commit.magic.MagicCommitTracker; -import org.apache.hadoop.fs.s3a.impl.StoreContext; +import org.apache.hadoop.fs.s3a.impl.AbstractStoreOperation; import static org.apache.hadoop.fs.s3a.commit.MagicCommitPaths.*; @@ -45,14 +45,12 @@ *

Important

: must not directly or indirectly import a class which * uses any datatype in hadoop-mapreduce. */ -public class MagicCommitIntegration { +public class MagicCommitIntegration extends AbstractStoreOperation { private static final Logger LOG = LoggerFactory.getLogger(MagicCommitIntegration.class); private final S3AFileSystem owner; private final boolean magicCommitEnabled; - private final StoreContext storeContext; - /** * Instantiate. * @param owner owner class @@ -60,9 +58,9 @@ public class MagicCommitIntegration { */ public MagicCommitIntegration(S3AFileSystem owner, boolean magicCommitEnabled) { + super(owner.createStoreContext()); this.owner = owner; this.magicCommitEnabled = magicCommitEnabled; - this.storeContext = owner.createStoreContext(); } /** @@ -85,6 +83,10 @@ public String keyOfFinalDestination(List elements, String key) { * Given a path and a key to that same path, create a tracker for it. * This specific tracker will be chosen based on whether or not * the path is a magic one. + * There's no attempt to switch to the span used at construction time, + * because this is a long-lived class. the span used to invoke + * this method will be the one used to create the write operation helper + * for the commit tracker. * @param path path of nominal write * @param key key of path of nominal write * @return the tracker for this operation. @@ -98,10 +100,10 @@ public PutTracker createTracker(Path path, String key) { if (isMagicCommitPath(elements)) { final String destKey = keyOfFinalDestination(elements, key); String pendingsetPath = key + CommitConstants.PENDING_SUFFIX; - storeContext.incrementStatistic( + getStoreContext().incrementStatistic( Statistic.COMMITTER_MAGIC_FILES_CREATED); tracker = new MagicCommitTracker(path, - storeContext.getBucket(), + getStoreContext().getBucket(), key, destKey, pendingsetPath, diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java index c3a70bffac057..71af2bef24bd8 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java @@ -65,7 +65,7 @@ public class MagicCommitTracker extends PutTracker { * @param originalDestKey the original key, in the magic directory. * @param destKey key for the destination * @param pendingsetKey key of the pendingset file - * @param writer writer instance to use for operations + * @param writer writer instance to use for operations; includes audit span */ public MagicCommitTracker(Path path, String bucket, diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitter.java index 59114f7ab7340..ea49988c101e8 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitter.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitter.java @@ -56,6 +56,7 @@ import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; import static org.apache.hadoop.fs.s3a.commit.CommitUtils.*; import static org.apache.hadoop.fs.s3a.commit.CommitUtilsWithMR.*; +import static org.apache.hadoop.util.functional.RemoteIterators.cleanupRemoteIterator; /** * Committer based on the contributed work of the @@ -828,6 +829,7 @@ protected PathExistsException failDestinationExists(final Path path, ? " dir" : ("file size " + status.getLen() + " bytes")); } + cleanupRemoteIterator(lf); } catch (IOException e) { LOG.info("Discarding exception raised when listing {}: " + e, path); LOG.debug("stack trace ", e); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AbstractStoreOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AbstractStoreOperation.java index 98b76b15da48f..f3b85ffedf332 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AbstractStoreOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AbstractStoreOperation.java @@ -18,24 +18,46 @@ package org.apache.hadoop.fs.s3a.impl; +import org.apache.hadoop.fs.s3a.audit.AuditSpan; + import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull; /** * Base class of operations in the store. * An operation is something which executes against the context to * perform a single function. - * It is expected to have a limited lifespan. */ public abstract class AbstractStoreOperation { + /** + * Store context. + */ private final StoreContext storeContext; /** - * constructor. + * Audit Span. + */ + private AuditSpan auditSpan; + + /** + * Constructor. + * Picks up the active audit span from the store context and + * stores it for later. * @param storeContext store context. */ protected AbstractStoreOperation(final StoreContext storeContext) { + this(storeContext, storeContext.getActiveAuditSpan()); + } + + /** + * Constructor. + * @param storeContext store context. + * @param auditSpan active span + */ + protected AbstractStoreOperation(final StoreContext storeContext, + final AuditSpan auditSpan) { this.storeContext = checkNotNull(storeContext); + this.auditSpan = checkNotNull(auditSpan); } /** @@ -46,4 +68,18 @@ public final StoreContext getStoreContext() { return storeContext; } + /** + * Get the audit span this object was created with. + * @return the current span + */ + public AuditSpan getAuditSpan() { + return auditSpan; + } + + /** + * Activate the audit span. + */ + public void activateAuditSpan() { + auditSpan.activate(); + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CallableSupplier.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CallableSupplier.java index b788f507735a0..e5f41f94502b3 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CallableSupplier.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CallableSupplier.java @@ -32,6 +32,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.fs.s3a.audit.AuditSpan; import org.apache.hadoop.util.DurationInfo; import static org.apache.hadoop.fs.impl.FutureIOSupport.raiseInnerCause; @@ -48,17 +49,41 @@ public final class CallableSupplier implements Supplier { private final Callable call; + /** + * Audit Span; may be null. + */ + private final AuditSpan auditSpan; + /** * Create. * @param call call to invoke. */ public CallableSupplier(final Callable call) { + this(null, call); + } + + /** + * Create. + * @param auditSpan audit span (or null) + * @param call call to invoke. + */ + public CallableSupplier( + final AuditSpan auditSpan, + final Callable call) { this.call = call; + this.auditSpan = auditSpan; } + /** + * Active any span and then call the supplied callable. + * @return the result. + */ @Override - public Object get() { + public T get() { try { + if (auditSpan != null) { + auditSpan.activate(); + } return call.call(); } catch (RuntimeException e) { throw e; @@ -66,6 +91,10 @@ public Object get() { throw new UncheckedIOException(e); } catch (Exception e) { throw new UncheckedIOException(new IOException(e)); + } finally { + if (auditSpan != null) { + auditSpan.deactivate(); + } } } @@ -86,11 +115,32 @@ public static CompletableFuture submit( return CompletableFuture.supplyAsync( new CallableSupplier(call), executor); } + /** + * Submit a callable into a completable future. + * RTEs are rethrown. + * Non RTEs are caught and wrapped; IOExceptions to + * {@code RuntimeIOException} instances. + * @param executor executor. + * @param auditSpan audit span (or null) + * @param call call to invoke + * @param type + * @return the future to wait for + */ + @SuppressWarnings("unchecked") + public static CompletableFuture submit( + final Executor executor, + final AuditSpan auditSpan, + final Callable call) { + return CompletableFuture.supplyAsync( + new CallableSupplier(auditSpan, call), + executor); + } /** * Wait for a list of futures to complete. If the list is empty, * return immediately. * @param futures list of futures. + * @param type * @throws IOException if one of the called futures raised an IOE. * @throws RuntimeException if one of the futures raised one. */ @@ -108,6 +158,7 @@ public static void waitForCompletion( /** * Wait for a single of future to complete, extracting IOEs afterwards. * @param future future to wait for. + * @param type * @throws IOException if one of the called futures raised an IOE. * @throws RuntimeException if one of the futures raised one. */ @@ -127,6 +178,7 @@ public static void waitForCompletion( /** * Wait for a single of future to complete, ignoring exceptions raised. * @param future future to wait for. + * @param type */ public static void waitForCompletionIgnoringExceptions( @Nullable final CompletableFuture future) { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ContextAccessors.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ContextAccessors.java index 27ac7dec1dd19..055ef45610840 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ContextAccessors.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ContextAccessors.java @@ -26,6 +26,8 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.Retries; +import org.apache.hadoop.fs.s3a.api.RequestFactory; +import org.apache.hadoop.fs.s3a.audit.AuditSpan; /** * An interface to implement for providing accessors to @@ -94,4 +96,17 @@ public interface ContextAccessors { @Retries.RetryTranslated ObjectMetadata getObjectMetadata(String key) throws IOException; + /** + * Return the active audit span. + * This is thread local -it MUST be picked up and passed into workers. + * Collect and cache the value during construction. + * @return active audit span. + */ + AuditSpan getActiveAuditSpan(); + + /** + * Get the request factory. + * @return the factory for requests. + */ + RequestFactory getRequestFactory(); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DeleteOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DeleteOperation.java index 2292179b3fd66..7e6e1cdfcf22b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DeleteOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DeleteOperation.java @@ -504,7 +504,7 @@ private CompletableFuture submitDelete( return null; } filesDeleted += keyList.size(); - return submit(executor, () -> { + return submit(executor, getAuditSpan(), () -> { asyncDeleteAction(operationState, keyList, pathList, diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ExecutingStoreOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ExecutingStoreOperation.java index ba8e743f293cb..afd64f94d41e7 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ExecutingStoreOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ExecutingStoreOperation.java @@ -23,14 +23,21 @@ import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; +import org.apache.hadoop.fs.s3a.audit.AuditSpan; +import org.apache.hadoop.util.functional.CallableRaisingIOE; + /** * A subclass of {@link AbstractStoreOperation} which * provides a method {@link #execute()} that may be invoked * exactly once. + * It declares itself a {@code CallableRaisingIOE} and + * can be handed straight to methods which take those + * as parameters. * @param return type of executed operation. */ public abstract class ExecutingStoreOperation - extends AbstractStoreOperation { + extends AbstractStoreOperation + implements CallableRaisingIOE { /** * Used to stop any re-entrancy of the rename. @@ -39,11 +46,34 @@ public abstract class ExecutingStoreOperation private final AtomicBoolean executed = new AtomicBoolean(false); /** - * constructor. + * Constructor. + * Picks up the active audit span from the store context and + * stores it for later. * @param storeContext store context. */ protected ExecutingStoreOperation(final StoreContext storeContext) { - super(storeContext); + this(storeContext, storeContext.getActiveAuditSpan()); + } + + /** + * Constructor. + * @param storeContext store context. + * @param auditSpan active span + */ + protected ExecutingStoreOperation( + final StoreContext storeContext, + final AuditSpan auditSpan) { + super(storeContext, auditSpan); + } + + /** + * Apply calls {@link #execute()}. + * @return the result. + * @throws IOException IO problem + */ + @Override + public final T apply() throws IOException { + return execute(); } /** @@ -58,12 +88,14 @@ protected ExecutingStoreOperation(final StoreContext storeContext) { /** * Check that the operation has not been invoked twice. * This is an atomic check. + * After the check: activates the span. * @throws IllegalStateException on a second invocation. */ protected void executeOnlyOnce() { Preconditions.checkState( !executed.getAndSet(true), "Operation attempted twice"); + activateAuditSpan(); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/HeaderProcessing.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/HeaderProcessing.java index 5efec2b36dafe..1c1f66ded47c3 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/HeaderProcessing.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/HeaderProcessing.java @@ -188,6 +188,15 @@ public class HeaderProcessing extends AbstractStoreOperation { public static final String XA_STORAGE_CLASS = XA_HEADER_PREFIX + Headers.STORAGE_CLASS; + /** + * HTTP Referrer for logs: {@value}. + * This can be found in S3 logs, but is not set as + * an attribute in objects. + * important: the header value is deliberately + * a mis-spelling, as that is defined in RFC-1945. + */ + public static final String HEADER_REFERRER = "Referer"; + /** * Standard headers which are retrieved from HEAD Requests * and set as XAttrs if the response included the relevant header. @@ -230,6 +239,13 @@ public class HeaderProcessing extends AbstractStoreOperation { public static final String CONTENT_TYPE_APPLICATION_XML = "application/xml"; + /** + * Directory content type : {@value}. + * Matches use/expectations of AWS S3 console. + */ + public static final String CONTENT_TYPE_X_DIRECTORY = + "application/x-directory"; + /** * Construct. * @param storeContext store context. @@ -443,7 +459,7 @@ public static Optional extractXAttrLongValue(byte[] data) { * @param source the {@link ObjectMetadata} to copy * @param dest the metadata to update; this is the return value. */ - public void cloneObjectMetadata(ObjectMetadata source, + public static void cloneObjectMetadata(ObjectMetadata source, ObjectMetadata dest) { // Possibly null attributes diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java index a5ce1f68ad3fc..d6142f49c94a2 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java @@ -37,6 +37,16 @@ */ public final class InternalConstants { + /** + * This declared delete as idempotent. + * This is an "interesting" topic in past Hadoop FS work. + * Essentially: with a single caller, DELETE is idempotent + * but in a shared filesystem, it is is very much not so. + * Here, on the basis that isn't a filesystem with consistency guarantees, + * retryable results in files being deleted. + */ + public static final boolean DELETE_CONSIDERED_IDEMPOTENT = true; + private InternalConstants() { } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ListingOperationCallbacks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ListingOperationCallbacks.java index b0ebf1ca16975..e61a2a6414e16 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ListingOperationCallbacks.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ListingOperationCallbacks.java @@ -28,6 +28,7 @@ import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus; import org.apache.hadoop.fs.s3a.S3ListRequest; import org.apache.hadoop.fs.s3a.S3ListResult; +import org.apache.hadoop.fs.s3a.audit.AuditSpan; import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider; import org.apache.hadoop.fs.statistics.DurationTrackerFactory; @@ -46,13 +47,15 @@ public interface ListingOperationCallbacks { * Retry policy: retry untranslated. * @param request request to initiate * @param trackerFactory tracker with statistics to update + * @param span audit span for this operation * @return the results * @throws IOException if the retry invocation raises one (it shouldn't). */ @Retries.RetryRaw CompletableFuture listObjectsAsync( - S3ListRequest request, - DurationTrackerFactory trackerFactory) + S3ListRequest request, + DurationTrackerFactory trackerFactory, + AuditSpan span) throws IOException; /** @@ -61,14 +64,16 @@ CompletableFuture listObjectsAsync( * @param request last list objects request to continue * @param prevResult last paged result to continue from * @param trackerFactory tracker with statistics to update + * @param span audit span for the IO * @return the next result object * @throws IOException none, just there for retryUntranslated. */ @Retries.RetryRaw CompletableFuture continueListObjectsAsync( - S3ListRequest request, - S3ListResult prevResult, - DurationTrackerFactory trackerFactory) + S3ListRequest request, + S3ListResult prevResult, + DurationTrackerFactory trackerFactory, + AuditSpan span) throws IOException; /** @@ -82,7 +87,9 @@ S3ALocatedFileStatus toLocatedFileStatus( throws IOException; /** * Create a {@code ListObjectsRequest} request against this bucket, - * with the maximum keys returned in a query set by + * with the maximum keys returned in a query set in the FS config. + * The active span for the FS is handed the request to prepare it + * before this method returns. * {@link #getMaxKeys()}. * @param key key for request * @param delimiter any delimiter diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/LogExactlyOnce.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/LogExactlyOnce.java index 54a8836d02ba4..efc48154e6572 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/LogExactlyOnce.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/LogExactlyOnce.java @@ -39,4 +39,14 @@ public void warn(String format, Object...args) { log.warn(format, args); } } + public void info(String format, Object...args) { + if (!logged.getAndSet(true)) { + log.info(format, args); + } + } + public void error(String format, Object...args) { + if (!logged.getAndSet(true)) { + log.error(format, args); + } + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MkdirOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MkdirOperation.java new file mode 100644 index 0000000000000..c963b09ae3bf9 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MkdirOperation.java @@ -0,0 +1,184 @@ +/* + * 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 java.io.FileNotFoundException; +import java.io.IOException; +import java.nio.file.AccessDeniedException; +import java.util.Set; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.FileAlreadyExistsException; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.Retries; +import org.apache.hadoop.fs.s3a.S3AFileStatus; + +/** + * The mkdir operation. + * A walk up the ancestor list halting as soon as a directory (good) + * or file (bad) is found. + * Optimized with the expectation that there is a marker up the path + * or (ultimately) a sibling of the path being created. + * It performs the directory listing probe ahead of the simple object HEAD + * call for this reason -the object is the failure mode which SHOULD NOT + * be encountered on normal execution. + */ +public class MkdirOperation extends ExecutingStoreOperation { + + private static final Logger LOG = LoggerFactory.getLogger( + RenameOperation.class); + + private final Path dir; + + private final MkdirCallbacks callbacks; + + public MkdirOperation( + final StoreContext storeContext, + final Path dir, + final MkdirCallbacks callbacks) { + super(storeContext); + this.dir = dir; + this.callbacks = callbacks; + } + + /** + * + * Make the given path and all non-existent parents into + * directories. + * @return true if a directory was created or already existed + * @throws FileAlreadyExistsException there is a file at the path specified + * @throws IOException other IO problems + */ + @Override + @Retries.RetryTranslated + public Boolean execute() throws IOException { + LOG.debug("Making directory: {}", dir); + if (dir.isRoot()) { + // fast exit for root. + return true; + } + + FileStatus fileStatus = getPathStatusExpectingDir(dir); + if (fileStatus != null) { + if (fileStatus.isDirectory()) { + return true; + } else { + throw new FileAlreadyExistsException("Path is a file: " + dir); + } + } + // dir, walk up tree + // Walk path to root, ensuring closest ancestor is a directory, not file + Path fPart = dir.getParent(); + try { + while (fPart != null && !fPart.isRoot()) { + fileStatus = getPathStatusExpectingDir(fPart); + if (fileStatus == null) { + // nothing at this path, so validate the parent + fPart = fPart.getParent(); + continue; + } + if (fileStatus.isDirectory()) { + // the parent dir exists. All is good. + break; + } + + // there's a file at the parent entry + throw new FileAlreadyExistsException(String.format( + "Can't make directory for path '%s' since it is a file.", + fPart)); + } + } catch (AccessDeniedException e) { + LOG.info("mkdirs({}}: Access denied when looking" + + " for parent directory {}; skipping checks", + dir, fPart); + LOG.debug("{}", e.toString(), e); + } + + // if we get here there is no directory at the destination. + // so create one. + String key = getStoreContext().pathToKey(dir); + // this will create the marker file, delete the parent entries + // and update S3Guard + callbacks.createFakeDirectory(key); + return true; + } + + /** + * Get the status of a path, downgrading FNFE to null result. + * @param path path to probe. + * @param probes probes to exec + * @return the status or null + * @throws IOException failure other than FileNotFound + */ + private S3AFileStatus probePathStatusOrNull(final Path path, + final Set probes) throws IOException { + try { + return callbacks.probePathStatus(path, probes); + } catch (FileNotFoundException fnfe) { + return null; + } + } + + /** + * Get the status of a path -optimized for paths + * where there is a directory marker or child entries. + * @param path path to probe. + * @return the status + * @throws IOException failure + */ + private S3AFileStatus getPathStatusExpectingDir(final Path path) + throws IOException { + S3AFileStatus status = probePathStatusOrNull(path, + StatusProbeEnum.DIRECTORIES); + if (status == null) { + status = probePathStatusOrNull(path, + StatusProbeEnum.FILE); + } + return status; + } + + /** + * Callbacks used by mkdir. + */ + public interface MkdirCallbacks { + + /** + * Get the status of a path. + * @param path path to probe. + * @param probes probes to exec + * @return the status + * @throws IOException failure + */ + @Retries.RetryTranslated + S3AFileStatus probePathStatus(Path path, + Set probes) throws IOException; + + /** + * Create a fake directory, always ending in "/". + * Retry policy: retrying; translated. + * @param key name of directory object. + * @throws IOException IO failure + */ + @Retries.RetryTranslated + void createFakeDirectory(String key) throws IOException; + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java index 5890ac01a9e74..553d6e6031f8c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java @@ -381,7 +381,7 @@ protected Path renameFileToDest() throws IOException { * Execute a full recursive rename. * There is a special handling of directly markers here -only leaf markers * are copied. This reduces incompatibility "regions" across versions. -Are * @throws IOException failure + * @throws IOException failure */ protected void recursiveDirectoryRename() throws IOException { final StoreContext storeContext = getStoreContext(); @@ -596,7 +596,7 @@ protected CompletableFuture initiateCopy( source.getVersionId(), source.getLen()); // queue the copy operation for execution in the thread pool - return submit(getStoreContext().getExecutor(), () -> + return submit(getStoreContext().getExecutor(), getAuditSpan(), () -> copySourceAndUpdateTracker( childSourcePath, key, diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java new file mode 100644 index 0000000000000..f0a7c79e60188 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java @@ -0,0 +1,651 @@ +/* + * 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 java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; + +import com.amazonaws.AmazonWebServiceRequest; +import com.amazonaws.services.s3.model.AbortMultipartUploadRequest; +import com.amazonaws.services.s3.model.CannedAccessControlList; +import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest; +import com.amazonaws.services.s3.model.CopyObjectRequest; +import com.amazonaws.services.s3.model.DeleteObjectRequest; +import com.amazonaws.services.s3.model.DeleteObjectsRequest; +import com.amazonaws.services.s3.model.GetObjectMetadataRequest; +import com.amazonaws.services.s3.model.GetObjectRequest; +import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest; +import com.amazonaws.services.s3.model.ListMultipartUploadsRequest; +import com.amazonaws.services.s3.model.ListNextBatchOfObjectsRequest; +import com.amazonaws.services.s3.model.ListObjectsRequest; +import com.amazonaws.services.s3.model.ListObjectsV2Request; +import com.amazonaws.services.s3.model.ObjectListing; +import com.amazonaws.services.s3.model.ObjectMetadata; +import com.amazonaws.services.s3.model.PartETag; +import com.amazonaws.services.s3.model.PutObjectRequest; +import com.amazonaws.services.s3.model.SSEAwsKeyManagementParams; +import com.amazonaws.services.s3.model.SSECustomerKey; +import com.amazonaws.services.s3.model.SelectObjectContentRequest; +import com.amazonaws.services.s3.model.UploadPartRequest; +import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.s3a.Retries; +import org.apache.hadoop.fs.s3a.S3AEncryptionMethods; +import org.apache.hadoop.fs.s3a.api.RequestFactory; +import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecretOperations; +import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets; + +import static org.apache.commons.lang3.StringUtils.isNotEmpty; +import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DEFAULT_UPLOAD_PART_COUNT_LIMIT; +import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkArgument; +import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull; + +/** + * The standard implementation of the request factory. + * This creates AWS SDK request classes for the specific bucket, + * with standard options/headers set. + * It is also where custom setting parameters can take place. + * + * All creation of AWS S3 requests MUST be through this class so that + * common options (encryption etc.) can be added here, + * and so that any chained transformation of requests can be applied. + * + * This is where audit span information is added to the requests, + * until it is done in the AWS SDK itself. + * + * All created requests will be passed through + * {@link PrepareRequest#prepareRequest(AmazonWebServiceRequest)} before + * being returned to the caller. + */ +public class RequestFactoryImpl implements RequestFactory { + + public static final Logger LOG = LoggerFactory.getLogger( + RequestFactoryImpl.class); + + /** + * Target bucket. + */ + private final String bucket; + + /** + * Encryption secrets. + */ + private EncryptionSecrets encryptionSecrets; + + /** + * ACL For new objects. + */ + private final CannedAccessControlList cannedACL; + + /** + * Max number of multipart entries allowed in a large + * upload. Tunable for testing only. + */ + private final long multipartPartCountLimit; + + /** Requester Pays. TODO: Wire up. */ + private final boolean requesterPays; + + /** + * Callback to prepare requests. + */ + private final PrepareRequest requestPreparer; + + protected RequestFactoryImpl( + final String bucket, + final EncryptionSecrets encryptionSecrets, + final CannedAccessControlList cannedACL, + final long multipartPartCountLimit, + final boolean requesterPays, + final PrepareRequest requestPreparer) { + this.encryptionSecrets = encryptionSecrets; + this.bucket = bucket; + this.cannedACL = cannedACL; + this.multipartPartCountLimit = multipartPartCountLimit; + this.requesterPays = requesterPays; + this.requestPreparer = requestPreparer; + } + + /** + * Preflight preparation of AWS request. + * @param web service request + * @return prepared entry. + */ + @Retries.OnceRaw + private T prepareRequest(T t) { + return requestPreparer != null + ? requestPreparer.prepareRequest(t) + : t; + } + + /** + * Get the canned ACL of this FS. + * @return an ACL, if any + */ + @Override + public CannedAccessControlList getCannedACL() { + return cannedACL; + } + + /** + * Get the target bucket. + * @return the bucket. + */ + protected String getBucket() { + return bucket; + } + + /** + * Create the AWS SDK structure used to configure SSE, + * if the encryption secrets contain the information/settings for this. + * @return an optional set of KMS Key settings + */ + @Override + public Optional generateSSEAwsKeyParams() { + return EncryptionSecretOperations.createSSEAwsKeyManagementParams( + encryptionSecrets); + } + + /** + * Create the SSE-C structure for the AWS SDK, if the encryption secrets + * contain the information/settings for this. + * This will contain a secret extracted from the bucket/configuration. + * @return an optional customer key. + */ + @Override + public Optional generateSSECustomerKey() { + return EncryptionSecretOperations.createSSECustomerKey( + encryptionSecrets); + } + + /** + * Get the encryption algorithm of this endpoint. + * @return the encryption algorithm. + */ + @Override + public S3AEncryptionMethods getServerSideEncryptionAlgorithm() { + return encryptionSecrets.getEncryptionMethod(); + } + + /** + * Sets server side encryption parameters to the part upload + * request when encryption is enabled. + * @param request upload part request + */ + @Override + public void setOptionalUploadPartRequestParameters( + UploadPartRequest request) { + generateSSECustomerKey().ifPresent(request::setSSECustomerKey); + } + + /** + * Sets server side encryption parameters to the part upload + * request when encryption is enabled. + * @param request upload part request + */ + @Override + public void setOptionalGetObjectMetadataParameters( + GetObjectMetadataRequest request) { + generateSSECustomerKey().ifPresent(request::setSSECustomerKey); + } + + /** + * Set the optional parameters when initiating the request (encryption, + * headers, storage, etc). + * @param request request to patch. + */ + @Override + public void setOptionalMultipartUploadRequestParameters( + InitiateMultipartUploadRequest request) { + generateSSEAwsKeyParams().ifPresent(request::setSSEAwsKeyManagementParams); + generateSSECustomerKey().ifPresent(request::setSSECustomerKey); + } + + private void setOptionalPutRequestParameters(PutObjectRequest request) { + generateSSEAwsKeyParams().ifPresent(request::setSSEAwsKeyManagementParams); + generateSSECustomerKey().ifPresent(request::setSSECustomerKey); + } + + @Override + public void setOptionalObjectMetadata(ObjectMetadata metadata) { + final S3AEncryptionMethods algorithm + = getServerSideEncryptionAlgorithm(); + if (S3AEncryptionMethods.SSE_S3 == algorithm) { + metadata.setSSEAlgorithm(algorithm.getMethod()); + } + } + + + /** + * Create a new object metadata instance. + * Any standard metadata headers are added here, for example: + * encryption. + * + * @param length length of data to set in header; Ignored if negative + * @return a new metadata instance + */ + @Override + public ObjectMetadata newObjectMetadata(long length) { + final ObjectMetadata om = new ObjectMetadata(); + setOptionalObjectMetadata(om); + if (length >= 0) { + om.setContentLength(length); + } + return om; + } + + @Override + public CopyObjectRequest newCopyObjectRequest(String srcKey, + String dstKey, + ObjectMetadata srcom) { + CopyObjectRequest copyObjectRequest = + new CopyObjectRequest(getBucket(), srcKey, getBucket(), dstKey); + ObjectMetadata dstom = newObjectMetadata(srcom.getContentLength()); + HeaderProcessing.cloneObjectMetadata(srcom, dstom); + setOptionalObjectMetadata(dstom); + copyEncryptionParameters(srcom, copyObjectRequest); + copyObjectRequest.setCannedAccessControlList(cannedACL); + copyObjectRequest.setNewObjectMetadata(dstom); + Optional.ofNullable(srcom.getStorageClass()) + .ifPresent(copyObjectRequest::setStorageClass); + return prepareRequest(copyObjectRequest); + } + + /** + * Propagate encryption parameters from source file if set else use the + * current filesystem encryption settings. + * @param srcom source object metadata. + + * @param copyObjectRequest copy object request body. + */ + private void copyEncryptionParameters( + ObjectMetadata srcom, + CopyObjectRequest copyObjectRequest) { + String sourceKMSId = srcom.getSSEAwsKmsKeyId(); + if (isNotEmpty(sourceKMSId)) { + // source KMS ID is propagated + LOG.debug("Propagating SSE-KMS settings from source {}", + sourceKMSId); + copyObjectRequest.setSSEAwsKeyManagementParams( + new SSEAwsKeyManagementParams(sourceKMSId)); + } + switch (getServerSideEncryptionAlgorithm()) { + case SSE_S3: + /* no-op; this is set in destination object metadata */ + break; + + case SSE_C: + generateSSECustomerKey().ifPresent(customerKey -> { + copyObjectRequest.setSourceSSECustomerKey(customerKey); + copyObjectRequest.setDestinationSSECustomerKey(customerKey); + }); + break; + + case SSE_KMS: + generateSSEAwsKeyParams().ifPresent( + copyObjectRequest::setSSEAwsKeyManagementParams); + break; + default: + } + } + /** + * Create a putObject request. + * Adds the ACL and metadata + * @param key key of object + * @param metadata metadata header + * @param srcfile source file + * @return the request + */ + @Override + public PutObjectRequest newPutObjectRequest(String key, + ObjectMetadata metadata, File srcfile) { + Preconditions.checkNotNull(srcfile); + PutObjectRequest putObjectRequest = new PutObjectRequest(getBucket(), key, + srcfile); + setOptionalPutRequestParameters(putObjectRequest); + putObjectRequest.setCannedAcl(cannedACL); + putObjectRequest.setMetadata(metadata); + return prepareRequest(putObjectRequest); + } + + /** + * Create a {@link PutObjectRequest} request. + * The metadata is assumed to have been configured with the size of the + * operation. + * @param key key of object + * @param metadata metadata header + * @param inputStream source data. + * @return the request + */ + @Override + public PutObjectRequest newPutObjectRequest(String key, + ObjectMetadata metadata, + InputStream inputStream) { + Preconditions.checkNotNull(inputStream); + Preconditions.checkArgument(isNotEmpty(key), "Null/empty key"); + PutObjectRequest putObjectRequest = new PutObjectRequest(getBucket(), key, + inputStream, metadata); + setOptionalPutRequestParameters(putObjectRequest); + putObjectRequest.setCannedAcl(cannedACL); + return prepareRequest(putObjectRequest); + } + + @Override + public PutObjectRequest newDirectoryMarkerRequest(String directory) { + String key = directory.endsWith("/") + ? directory + : (directory + "/"); + // an input stream which is laways empty + final InputStream im = new InputStream() { + @Override + public int read() throws IOException { + return -1; + } + }; + // preparation happens in here + final ObjectMetadata md = newObjectMetadata(0L); + md.setContentType(HeaderProcessing.CONTENT_TYPE_X_DIRECTORY); + PutObjectRequest putObjectRequest = + newPutObjectRequest(key, md, im); + return putObjectRequest; + } + + @Override + public ListMultipartUploadsRequest + newListMultipartUploadsRequest(String prefix) { + ListMultipartUploadsRequest request = new ListMultipartUploadsRequest( + getBucket()); + if (prefix != null) { + request.setPrefix(prefix); + } + return prepareRequest(request); + } + + @Override + public AbortMultipartUploadRequest newAbortMultipartUploadRequest( + String destKey, + String uploadId) { + return prepareRequest(new AbortMultipartUploadRequest(getBucket(), + destKey, + uploadId)); + } + + @Override + public InitiateMultipartUploadRequest newMultipartUploadRequest( + String destKey) { + final InitiateMultipartUploadRequest initiateMPURequest = + new InitiateMultipartUploadRequest(getBucket(), + destKey, + newObjectMetadata(-1)); + initiateMPURequest.setCannedACL(getCannedACL()); + setOptionalMultipartUploadRequestParameters(initiateMPURequest); + return prepareRequest(initiateMPURequest); + } + + @Override + public CompleteMultipartUploadRequest newCompleteMultipartUploadRequest( + String destKey, + String uploadId, + List partETags) { + // a copy of the list is required, so that the AWS SDK doesn't + // attempt to sort an unmodifiable list. + return prepareRequest(new CompleteMultipartUploadRequest(bucket, + destKey, uploadId, new ArrayList<>(partETags))); + + } + + @Override + public GetObjectMetadataRequest newGetObjectMetadataRequest(String key) { + GetObjectMetadataRequest request = + new GetObjectMetadataRequest(getBucket(), key); + //SSE-C requires to be filled in if enabled for object metadata + setOptionalGetObjectMetadataParameters(request); + return prepareRequest(request); + } + + @Override + public GetObjectRequest newGetObjectRequest(String key) { + GetObjectRequest request = new GetObjectRequest(bucket, key); + generateSSECustomerKey().ifPresent(request::setSSECustomerKey); + + return prepareRequest(request); + } + + @Override + public UploadPartRequest newUploadPartRequest( + String destKey, + String uploadId, + int partNumber, + int size, + InputStream uploadStream, + File sourceFile, + long offset) throws PathIOException { + checkNotNull(uploadId); + // exactly one source must be set; xor verifies this + checkArgument((uploadStream != null) ^ (sourceFile != null), + "Data source"); + checkArgument(size >= 0, "Invalid partition size %s", size); + checkArgument(partNumber > 0, + "partNumber must be between 1 and %s inclusive, but is %s", + DEFAULT_UPLOAD_PART_COUNT_LIMIT, partNumber); + + LOG.debug("Creating part upload request for {} #{} size {}", + uploadId, partNumber, size); + final String pathErrorMsg = "Number of parts in multipart upload exceeded." + + " Current part count = %s, Part count limit = %s "; + if (partNumber > multipartPartCountLimit) { + throw new PathIOException(destKey, + String.format(pathErrorMsg, partNumber, multipartPartCountLimit)); + } + UploadPartRequest request = new UploadPartRequest() + .withBucketName(getBucket()) + .withKey(destKey) + .withUploadId(uploadId) + .withPartNumber(partNumber) + .withPartSize(size); + if (uploadStream != null) { + // there's an upload stream. Bind to it. + request.setInputStream(uploadStream); + } else { + checkArgument(sourceFile.exists(), + "Source file does not exist: %s", sourceFile); + checkArgument(offset >= 0, "Invalid offset %s", offset); + long length = sourceFile.length(); + checkArgument(offset == 0 || offset < length, + "Offset %s beyond length of file %s", offset, length); + request.setFile(sourceFile); + request.setFileOffset(offset); + } + setOptionalUploadPartRequestParameters(request); + return prepareRequest(request); + } + + @Override + public SelectObjectContentRequest newSelectRequest(String key) { + SelectObjectContentRequest request = new SelectObjectContentRequest(); + request.setBucketName(bucket); + request.setKey(key); + generateSSECustomerKey().ifPresent(request::setSSECustomerKey); + return prepareRequest(request); + } + + @Override + public ListObjectsRequest newListObjectsV1Request( + final String key, + final String delimiter, + final int maxKeys) { + ListObjectsRequest request = new ListObjectsRequest() + .withBucketName(bucket) + .withMaxKeys(maxKeys) + .withPrefix(key); + if (delimiter != null) { + request.setDelimiter(delimiter); + } + return prepareRequest(request); + } + + @Override + public ListNextBatchOfObjectsRequest newListNextBatchOfObjectsRequest( + ObjectListing prev) { + return prepareRequest(new ListNextBatchOfObjectsRequest(prev)); + } + + @Override + public ListObjectsV2Request newListObjectsV2Request( + final String key, + final String delimiter, + final int maxKeys) { + final ListObjectsV2Request request = new ListObjectsV2Request() + .withBucketName(bucket) + .withMaxKeys(maxKeys) + .withPrefix(key); + if (delimiter != null) { + request.setDelimiter(delimiter); + } + return prepareRequest(request); + } + + @Override + public DeleteObjectRequest newDeleteObjectRequest(String key) { + return prepareRequest(new DeleteObjectRequest(bucket, key)); + } + + @Override + public DeleteObjectsRequest newDeleteObjectsRequest( + List keysToDelete, + boolean quiet) { + return prepareRequest( + new DeleteObjectsRequest(bucket) + .withKeys(keysToDelete) + .withQuiet(quiet)); + } + + @Override + public void setEncryptionSecrets(final EncryptionSecrets secrets) { + encryptionSecrets = secrets; + } + + /** + * Create a builder. + * @return new builder. + */ + public static RequestFactoryBuilder builder() { + return new RequestFactoryBuilder(); + } + + /** + * Builder. + */ + public static final class RequestFactoryBuilder { + + /** + * Target bucket. + */ + private String bucket; + + /** + * Encryption secrets. + */ + private EncryptionSecrets encryptionSecrets = new EncryptionSecrets(); + + /** + * ACL For new objects. + */ + private CannedAccessControlList cannedACL = null; + + /** Requester Pays. TODO: Wire up. */ + private boolean requesterPays = false; + + /** + * Multipart limit. + */ + private long multipartPartCountLimit = DEFAULT_UPLOAD_PART_COUNT_LIMIT; + + /** + * Callback to prepare requests. + */ + private PrepareRequest requestPreparer; + + private RequestFactoryBuilder() { + } + + public RequestFactory build() { + return new RequestFactoryImpl(bucket, encryptionSecrets, cannedACL, + multipartPartCountLimit, requesterPays, requestPreparer); + } + + public RequestFactoryBuilder withBucket(final String _bucket) { + bucket = _bucket; + return this; + } + + public RequestFactoryBuilder withEncryptionSecrets( + final EncryptionSecrets _encryptionSecrets) { + encryptionSecrets = _encryptionSecrets; + return this; + } + + public RequestFactoryBuilder withCannedACL( + final CannedAccessControlList _cannedACL) { + cannedACL = _cannedACL; + return this; + } + + public RequestFactoryBuilder withRequesterPays(final boolean _requesterPays) { + requesterPays = _requesterPays; + return this; + } + + public RequestFactoryBuilder withMultipartPartCountLimit( + final long _multipartPartCountLimit) { + multipartPartCountLimit = _multipartPartCountLimit; + return this; + } + + + public RequestFactoryBuilder withRequestPreparer( + final PrepareRequest _requestPreparer) { + this.requestPreparer = _requestPreparer; + return this; + } + } + + /** + * This is the callback to prepare a request. + */ + @FunctionalInterface + public interface PrepareRequest { + + /** + * Post-creation preparation of AWS request. + * @param t request + * @param request type. + * @return prepared entry. + */ + @Retries.OnceRaw + T prepareRequest(T t); + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java index 88231d8af9c04..db667c7033e28 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java @@ -25,6 +25,8 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; +import org.apache.hadoop.fs.s3a.api.RequestFactory; +import org.apache.hadoop.fs.s3a.audit.AuditSpan; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListeningExecutorService; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.MoreExecutors; @@ -146,7 +148,10 @@ public StoreContext( this.configuration = configuration; this.username = username; this.owner = owner; - this.executor = MoreExecutors.listeningDecorator(executor); + // some mock tests have a null executor pool + this.executor = executor !=null + ? MoreExecutors.listeningDecorator(executor) + : null; this.executorCapacity = executorCapacity; this.invoker = invoker; this.instrumentation = instrumentation; @@ -391,4 +396,22 @@ public CompletableFuture submit( LambdaUtils.eval(future, call)); return future; } + + /** + * Return the active audit span. + * This is thread local -it MUST be picked up and passed into workers. + * Collect and cache the value during construction. + * @return active audit span. + */ + public AuditSpan getActiveAuditSpan() { + return contextAccessors.getActiveAuditSpan(); + } + + /** + * Get the request factory. + * @return the factory for requests. + */ + public RequestFactory getRequestFactory() { + return contextAccessors.getRequestFactory(); + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardDynamoTable.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardDynamoTable.java index 20bd250da0c38..4aa30eda5530d 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardDynamoTable.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardDynamoTable.java @@ -35,6 +35,8 @@ import java.util.List; import com.amazonaws.services.dynamodbv2.xspec.ExpressionSpecBuilder; + +import org.apache.hadoop.fs.s3a.audit.AuditSpan; import org.apache.hadoop.thirdparty.com.google.common.collect.Lists; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -347,21 +349,25 @@ protected long listStatusFilesystem( protected long dumpRawS3ObjectStore( final CsvFile csv) throws IOException { S3AFileSystem fs = getFilesystem(); - Path rootPath = fs.qualify(new Path("/")); - Listing listing = fs.getListing(); - S3ListRequest request = listing.createListObjectsRequest("", null); long count = 0; - RemoteIterator st = - listing.createFileStatusListingIterator(rootPath, request, - ACCEPT_ALL, - new Listing.AcceptAllButSelfAndS3nDirs(rootPath)); - while (st.hasNext()) { - count++; - S3AFileStatus next = st.next(); - LOG.debug("[{}] {}", count, next); - csv.entry(next); + Path rootPath = fs.qualify(new Path("/")); + try (AuditSpan span = fs.createSpan("dump", rootPath.toString(), null)) { + Listing listing = fs.getListing(); + S3ListRequest request = listing.createListObjectsRequest("", null); + count = 0; + RemoteIterator st = + listing.createFileStatusListingIterator(rootPath, request, + ACCEPT_ALL, + new Listing.AcceptAllButSelfAndS3nDirs(rootPath), + span); + while (st.hasNext()) { + count++; + S3AFileStatus next = st.next(); + LOG.debug("[{}] {}", count, next); + csv.entry(next); + } + LOG.info("entry count: {}", count); } - LOG.info("entry count: {}", count); return count; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java index b963e7e2532e5..e4542eb1bcfa6 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java @@ -63,6 +63,8 @@ import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughputDescription; import com.amazonaws.services.dynamodbv2.model.TableDescription; import com.amazonaws.services.dynamodbv2.model.WriteRequest; + +import org.apache.hadoop.fs.s3a.impl.InternalConstants; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.apache.hadoop.thirdparty.com.google.common.collect.Lists; @@ -593,7 +595,7 @@ private void innerDelete(final Path path, } // the policy on whether repeating delete operations is based // on that of S3A itself - boolean idempotent = S3AFileSystem.DELETE_CONSIDERED_IDEMPOTENT; + boolean idempotent = InternalConstants.DELETE_CONSIDERED_IDEMPOTENT; if (tombstone) { Preconditions.checkArgument(ttlTimeProvider != null, "ttlTimeProvider " + "must not be null"); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java index 2acae76875a22..188fa258232ff 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java @@ -60,6 +60,8 @@ import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3AUtils; +import org.apache.hadoop.fs.s3a.WriteOperationHelper; +import org.apache.hadoop.fs.s3a.audit.AuditSpan; import org.apache.hadoop.fs.s3a.auth.RolePolicies; import org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens; import org.apache.hadoop.fs.s3a.commit.CommitConstants; @@ -84,6 +86,7 @@ import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; import static org.apache.hadoop.fs.s3a.commit.staging.StagingCommitterConstants.FILESYSTEM_TEMP_PATH; import static org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStoreTableManager.SSE_DEFAULT_MASTER_KEY; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.MULTIPART_UPLOAD_ABORTED; import static org.apache.hadoop.service.launcher.LauncherExitCodes.*; /** @@ -1584,6 +1587,7 @@ public int run(String[] args, PrintStream out) throw invalidArgs("No options specified"); } processArgs(paths, out); + println(out, "Listing uploads under path \"%s\"", prefix); promptBeforeAbort(out); processUploads(out); @@ -1605,8 +1609,13 @@ private void promptBeforeAbort(PrintStream out) throws IOException { } private void processUploads(PrintStream out) throws IOException { - MultipartUtils.UploadIterator uploads; - uploads = getFilesystem().listUploads(prefix); + final S3AFileSystem fs = getFilesystem(); + MultipartUtils.UploadIterator uploads = fs.listUploads(prefix); + AuditSpan span = + fs.createSpan(MULTIPART_UPLOAD_ABORTED, + prefix, null); + final WriteOperationHelper writeOperationHelper + = fs.getWriteOperationHelper(); int count = 0; while (uploads.hasNext()) { @@ -1620,18 +1629,20 @@ private void processUploads(PrintStream out) throws IOException { upload.getKey(), upload.getUploadId()); } if (mode == Mode.ABORT) { - getFilesystem().getWriteOperationHelper() + writeOperationHelper .abortMultipartUpload(upload.getKey(), upload.getUploadId(), true, LOG_EVENT); } } + span.deactivate(); if (mode != Mode.EXPECT || verbose) { println(out, "%s %d uploads %s.", TOTAL, count, mode == Mode.ABORT ? "deleted" : "found"); } if (mode == Mode.EXPECT) { if (count != expectedCount) { - throw badState("Expected %d uploads, found %d", expectedCount, count); + throw badState("Expected upload count under %s: %d, found %d", + prefix, expectedCount, count); } } } @@ -1643,6 +1654,9 @@ private void processUploads(PrintStream out) throws IOException { * @return true iff u was created at least age milliseconds ago. */ private boolean olderThan(MultipartUpload u, long msec) { + if (msec == 0) { + return true; + } Date ageDate = new Date(System.currentTimeMillis() - msec); return ageDate.compareTo(u.getInitiated()) >= 0; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectBinding.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectBinding.java index 20dc00fbc06a1..35bc82be612df 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectBinding.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectBinding.java @@ -20,7 +20,6 @@ import java.io.IOException; import java.util.Locale; -import java.util.Optional; import com.amazonaws.services.s3.model.CSVInput; import com.amazonaws.services.s3.model.CSVOutput; @@ -28,7 +27,6 @@ import com.amazonaws.services.s3.model.InputSerialization; import com.amazonaws.services.s3.model.OutputSerialization; import com.amazonaws.services.s3.model.QuoteFields; -import com.amazonaws.services.s3.model.SSECustomerKey; import com.amazonaws.services.s3.model.SelectObjectContentRequest; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.slf4j.Logger; @@ -96,7 +94,6 @@ public boolean isEnabled() { * @param readContext the read context, which includes the source path. * @param expression the SQL expression. * @param builderOptions query options - * @param sseKey optional SSE customer key * @param objectAttributes object attributes from a HEAD request * @return an FSDataInputStream whose wrapped stream is a SelectInputStream * @throws IllegalArgumentException argument failure @@ -108,7 +105,6 @@ public FSDataInputStream select( final S3AReadOpContext readContext, final String expression, final Configuration builderOptions, - final Optional sseKey, final S3ObjectAttributes objectAttributes) throws IOException { return new FSDataInputStream( @@ -118,8 +114,8 @@ public FSDataInputStream select( buildSelectRequest( readContext.getPath(), expression, - builderOptions, - sseKey))); + builderOptions + ))); } /** @@ -127,7 +123,6 @@ public FSDataInputStream select( * @param path source path. * @param expression the SQL expression. * @param builderOptions config to extract other query options from - * @param sseKey optional SSE customer key * @return the request to serve * @throws IllegalArgumentException argument failure * @throws IOException problem building/validating the request @@ -135,16 +130,13 @@ public FSDataInputStream select( public SelectObjectContentRequest buildSelectRequest( final Path path, final String expression, - final Configuration builderOptions, - final Optional sseKey) + final Configuration builderOptions) throws IOException { Preconditions.checkState(isEnabled(), "S3 Select is not enabled for %s", path); SelectObjectContentRequest request = operations.newSelectRequest(path); buildRequest(request, expression, builderOptions); - // optionally set an SSE key in the input - sseKey.ifPresent(request::withSSECustomerKey); return request; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java index 6d386f250e6cf..bcf65cde1bc82 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java @@ -51,6 +51,7 @@ import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus; import org.apache.hadoop.fs.s3a.UnknownStoreException; +import org.apache.hadoop.fs.s3a.audit.AuditSpan; import org.apache.hadoop.fs.s3a.impl.DirMarkerTracker; import org.apache.hadoop.fs.s3a.impl.DirectoryPolicy; import org.apache.hadoop.fs.s3a.impl.DirectoryPolicyImpl; @@ -417,13 +418,15 @@ ScanResult execute(final ScanArgs scanArgs) minMarkerCount = maxMarkerCount; maxMarkerCount = m; } - ScanResult result = scan(target, - scanArgs.isDoPurge(), - minMarkerCount, - maxMarkerCount, - scanArgs.getLimit(), - filterPolicy); - return result; + try (AuditSpan span = + fs.createSpan("marker-tool-scan", target.toString(), null)) { + return scan(target, + scanArgs.isDoPurge(), + minMarkerCount, + maxMarkerCount, + scanArgs.getLimit(), + filterPolicy); + } } /** diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/audit.png b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/audit.png new file mode 100644 index 0000000000000000000000000000000000000000..b0cbbce265e63ef06c8b4ed36e9112b3365e59b8 GIT binary patch literal 49705 zcmdqJcT|&G*Ds2_gN-6ZRGM_@9kw7{Kzaui>C$@(Hbg~&)JPX0l+dJv7K&1(OD~~F zuK@|2kmRhOdvEvqzUMo4-2091jB)=WJbAL#Ty2)${LS_Hp1R_>)3m3_$jHtqE8Wo| zBRe`wMs~R2$JPrbj~SGDbO-+hV^!?mAQ(YKy2<36{2zNMxZI>78389!3*w{90B z(j4GJ6gUx57?bXYs6Zk1a6SYIjn0L|#hyNE^<>7rQSB8;&_5a3DgBPYgAae~Ur;hh8`8YF~VQd`nWhtmRW^ zMJZSMV6;Np2A*3&Y6qfZYdiFkNk4a-d_JY2>)aA+t1w~<6Pw}SZ;zs*6Uj5L_V@AV8|33fc$Q1rPLeqY z&xtZW~wmBD+$}zK-!NH8G%t9~_D$SJ3aow=gYZG^Pb z1xM)X44qS@Z|x2NG0Kyq%&~-@#?N}{2JLd5a}j*JZZ%qD-T#($w=%6gPAHoKGV}VG zwjsiFWxJqhMHALXKwe5SQa*9j|5IuSzK#NKU>d{S)6J2&V^>PtDlPGyPdQUMT56rN zY%W_k95yo*?Q1>x$F3$e*JP!us`bsx)XcM^R$`8>yu(%q->tIkBtJ)87>i`@>ys=unODql0#u}RX;D+fX|`7EycCG%*LVr%^Ens%Yy|DJ*?-r zaOD9@XUn}X0hKYhe0|>a+zXv{bZHs-kxcgo!^EKdd8T##RNEblqaU3#SzM~@2P7Jy zKAO?=c@1aAg&)iPKK&X5-|8q6QNj#;x${P;0DSt{er8QS50vUfjH*1NB4a?B_Eh60 z3O}u$<%x03G(r}Nhud7~d=t{IlX=f zzE9Cq5g$qkSd!Mj-^tF6&!8*04!xoDQ>VkjQmtj0wQRM`V_bApg!|d(xuK4yfI{Pk zs<^hcwRx}1g;nzHTO$^Gv)~;HDw0cmOVY?#>7>3~WBczTo zQXG}{pss>xjLSzPN17^0vOtoX8>2IZb!%8c=w@SA(w-s${s>$uD)V&FsR=w4lTU$3 z#nkxkV!Ne~jIB*{C9hLErE$(xzOgSAvY-rT>2K{f(icNS-(0&61qOU4dQ@b|&T`kv z)Q#b6dtacPPPAZLY43odYEwTVI;GPoj96T>XE6J;REkW?=7sZp`lbawJ;nQx zoYOB|&9O9fTVW!PFDZIGPL7SyyS>GV8!y5g!5I6i)^IEo&a^hR%v@@U2`IBkn|S#E z*r}kOcB-ph6R&?IR^*;8y^))!;Oae_JUwhGX_WfO(xb(zN412!I)2a!b%%uX(@U3S zdSp|bqwRGVPW$p=4UhEpbSBEaegzDaZEt2$y<~0xq4L7Ty0kPFsj4`Wd)5rPp;tF$ z>t1>+4dk0ZkZjHt)I_|vg_V)#f!|0`Mb4p=4B>_hbP6x7PvRt^zqOM z+V{@`15jeq^=R@10ck%lh+GdqR*69s28j#Lbi<^(vmQ1GE<9MD4Mlo&4@-@~PHaXb zrorLOW5qQ$QWvWh^82z(*9%(_El5-GQkZ}CXqsJL3-$2LIjv>Cm@JR41q;a(o@XOD})_rvg`* zoOe#UI(hT(bJB)*60$`(w~rwb`UV%JHZGkcBl$vPJny~>-k;D$YvhWUP4I!o$NzlX zyNwTKx$U$6;wt{=v!@3wTwK&is%K;8P5zde{KwBu?#U*$XwS@C@&!LVd+^h0FOJSu zwvBnV^FKWd|5cqN4yPp2PktH}t;AjMBtMb#5Iu(hY8_D&tWc;NC9gu-wfV=KL>-jdR~6@ zS8MD4MQcC`;8cJ)rQT(c?$WqMcAyx`zVfcUh8#Y0;G2JzhrHH+EU=B|oVoMY10V3< z$;nr#Jnx3yeZ2YR<4ulRmL^Ny5#{I$E#6Z{hy5QSQv??M*)d-n6{Emw^YOi68?*rt9IN611d&y`TF3Fw%ePl*Z{`ZBR=j-XU*ve$qNI zD5^p4ag4Nc>QAZqa4gK+a=rEPhzWU9qt9d}=g*c6TC|mUgTtTm(SLUp74EpCXnGJb zxzuVUsrZV$rGGm?2CFM@NfKQ=O*(|HF2!JVx?3Mi-#>ezj2;@gUazT7I%$6-e2U75 zFe7I=VIlf?#`$pq8?@pX>7-1)fJ;VixYCGmHT|$oUU|nqPCb+VkDeZuM`fe|Pe@pV zwczjwV{mxgt(P+RNAdf(3>gRq3v~zn;oQFVF)jIq_%<&!pUQvqI{16QRc4%0w4_dN zY~`q-x9PXA*4B1*A$k{aRL8L+0T}8<wVVl#!JJi#i8wQB&#ihx4cJ7Uno3Uv1>X=VwGzi9$1zrQkBuJ@ z17;WpM_5a9l9XHHc{jft4;;n;L~f-L)_J>#lSs=R_wwlKhC-Pbyv!%3+c;xm-8_>l zBP55Ay&j|Br@7c#-IoptFw4Le6LO~D2WN?kA$a0kr#WTd6|>FVH?=~Kk7UzyxPisJq;y4U9w zH3Ibt>0w;7mh|U>`W?@xi8HiBM4Z8<`p~(Z10FXygHfv%!DOxC)D&(Y&ouQ`Al#?O zyLS&6$*GGV%{3Ov8@XA`uVecs6eD=px~WYW*a1*#-s5q z0$#Ra>jmr zN8I9fHW8;d&w<2J;DN_WdweR+Z;Sg@V1X9*uru{?C?fe5emdy!;g!b5&?OkJxtSx2 zkY`&6A^>=z2zk5lB%k3#l~CGU|01hUyIv(264zCys*}3mwqFM6l-bTa8quF?#AWg$ zrUXCjWnPUbiJ#Wo{}C)=6Ixj1E@wIhe4BFWe~FhhQG~%wa~_R2(Ld04&AS*&YxkoE zOa~rM?XeF#tfIC{vb4=kp|=$|tOu4)FqUPjW8%6CAiPp}0cE{d$R^MaG-Ek9m=p1b z`VONpaVq&l;I>*<{^=V0)^>MhUf*P9R5{%Qf9Qm>Ej z*X!tq;&T$w!)!5@tY*y6l+dd#A|8eQrX}WK*a(@;)vizzjvPDP^Lsbh1ReBk5QyBA zwV{;eZW}Z6MNScG=rj)=HYtg9&PbA4i9R#RxT^K=ma_Z_lOKg>BD; zZz%!x9(5LqF=rXN6{)_Ef>fj{x@15jc?(gzI*z)(l{2snk#uUF)bv>Dj}zHGhpfs8 z`Y=d7dU&*P%zAYNz8CVwX%0bf75MnNu%|l5!IeUpmU<< zrG>G0CrLMFVr_A;paVNETIRR#ea1zZV=z%*I|+rh#dzWimTLYHp*vk!a*32&uU!vA zIFK_KL;)Wh`sf8Ap~aXd>#i?Pll5@amnz?}jE+x3qxseJ3gqkhSCofGuzfP{tKdA_ z9!p*t&&e*#w4~=o(~!d&yLQ(VzW#VeL)G7)iQ=;ua1@!vBx$2Cn8wOc10*`|1;hyY z096oG|GPp5p{uz$GqpOw?x~6BR^Max@@q1S`9k$w_>>_!Ig7I_p|hLtYAPP@CEyt^ z8bkWGzo^R)9+bH^;|49ZGZm22x@>{WIb&{}xd>CLk#3$?J^W5b{ZnjePVS6r*otYJ z)B9zxStz6jHE~_o_B_Ptb$!}NJ)qE}HogU+SJ2w7H5U)p!SVh(DU8{k3n}%&l&2n9 zn&%o(_Lcq${q5*bz{Y)t`+;a~mhi%1FL0|ZPLnAa_DHM9YV%9Jq^2{6DzeFUWGuk? zooB!O@_T9)qxsc_5yn&USnrv!&G;auv3xzCD0k1+q_taP352%SHQCccc;{yQE4H_L zd7v>4XMu5LHxf@ z<8KH6+~3hAPA5v$$FN>M*T`_Z8~E!TiRj>(nrUwFs)~mV#fAdGFI6p!2lBj>{?O6| z+Z#mbdhyL#Wu6$?mjaRv^7ch~Z-Jh-Zh|_*3)?z-Wq1yQKvq3k%&{*Uw%h;88p+rL z@W(^}$$dP;aa>%^5k5x8K49mP7g)10j34HnnR%*oSg{5*>&$gti zo9Gr5cUqytx+^6}XqPR%;atq7u@bRb9tOLs|Dm@!ri3(87JZJyu8kgg#dZt!*S~Av zSy5l?z&d=oc8z!LYfx%UB0CH9v?3nq=~8^H*o316zCWwh^N8E{ec;67x4lP3S9CZ{ z@ew3c!M!=pbz(6)Z~j}gM{)+_WAv6~W#Q#&!d~`SudY-->hk9LQ+P`}9$*cv^osB! z0%Lyd`%~L7J7^xBwSR#!-u33HiyQLHfXI}DWaMscVvJ&Om!N}z2VjP<0RS_cmz&fO zZgX35W*BaXEWO+}6K(;sgb_6Sxun**`i+r;ZM5Vfn9fH};DNRT*&CaQd9-kH8wqN< z^g903g#io6znwclykL{&x`E2%Zywoe2z%gGBv-WXrsWky_#+EELesd#iG*vovIkb& zkIr$K>qM2!Iu3qXDQ0Ie5BzA3{fSkm;jf1RZ${;j&??7cWJOGRPx5SaBd)YOA`i{nQ4W=;)5MxcC>)X?8NVfi@YS>)lC#f_QrA! zRqN;DdbjrA{-YI00K7hM)VEyG5PeA04eH2?oD!RCkML4&bRW@COWs;+7b#5;v-k?Y z2tE_bmCmA0X_I`jbRV`0EBc(kciR-D?V%xGnw$qpk!cqJLVphEaOf43CyFnh7w=|Q z9`J(CNlcGvSw>cdyUBUbii^eNH*c}NGD;+BCf=jnfKcyr=q_ z!g!)Retf$g@Ms88J){&I8CEFm0{B**jpsn#<-;wlD3Z9YKt^(rj?2lwOb*Mh0Q6u z>x{oLexEF~yqUqNbF(I3=`CqjVJ3hI9rHx1O*K_Ns%RNZWQ?>$6}~M)K!>0r$ZUsPfy-1aXBtKdjBSAT0MXYRlK^QgS3lY z?8`>lbluTwMwrHQh>VY5!RfYslxA&Z#bwHGglYyDy>6~iIRrmk7!$BH6pU0Dk~1nl|L zq2pvCkHO|VQzD1n-N=~Pb~^i2j$)I^?S&pSVxsaycUVM!X1JM2@$8Q;@&Ko|`S!sP zz@MydSM#}NC|1mGy2wK1QWZ9tFWLZ;s%bf2Q~&(-5x*8ovCD$A6F&5Ibq>VcWX)}iQoy3(cWsK55Z^c9i{T45QC}sdw z-z8U3v-)uCb997I0U@9NWsR|TGN+l>{qe&d>I&u{I@!V`Izh5iOl=thxZ2EZ2jln4 z0#~2i^kO;V-dY2tv2T7ZB$MfEE7Ldr`BgO+E^vQd3!_mp&)Gkv_cFr7VTU#T2YhAV z!St(JFZHRe`hmEF<5kPC-L6Wt%2vm=_h_atpLI7{AKXccvBEYSIK~+qU|*zMl=)56 zix1e7N4k@nQ=XZUGbv8Jn z9UU{C=>s!r-Q=9}AK(4VHN#Gfsr&kchTQodVZ;M=4VEh4!~D;OrkQa6v9%cx9CNgd z&olpf4E%^)hF(dSjT8hwR&V?(I3`Q-*zb`e;xWl@bIBy10HlHx0Y4GJuM!DMn_*^c z;$`2wkC9Pa1Ly!L)cvj&+<^&)54*WB&K+g{ejA`62OyNBQX*|MJnk}^(jf{Gl5p_u z?Zw_q4)u43uigRJ!vTC}zmcIJcKfvVC4i9}z)JaD$+R<|A+^66YVfy5VotR9M(1V3W<^GB6+D?fW0yZvWROgEBgbsJ7;eP#lX&H-peLU2A8 z{SNRvu?C&|Zy=C=!ZN=@MI=b6;M5Zl0H6F!89y=jOIEH9=-~rA=U}w|hVJ~{%)_kX z-c(>3+FX8NM?biJ2VA}zl7Ob4AP@k&4qiQ&F|t>be@fZ~w+hF-BVsGrim+CYWp zeLlR>#LVx)S?9pNg^U5&a$cCN?#9CDb*|;0miE}50t@2+)f(8<&bGFbiJ_i&_lo;{ z;c|>|Xww`JbpuyLsB~%RbM-Nqa-t$a*df8U*)uELNoISNuJgSYrM+4rw&lZ9sdkJe zASV3CI_aX5j%TV#blLG*Rc<~|==&|X&D?w}ydrE+mW`b|c?RZ3N4ZyPZZ-`kf0d_K zvphK9DFtu;hMpwP80$f7hC04IBI$Br*-?mJyz5L0NFUo*bI3Xj37R^>otl5*O?6g^ z6udr|zOk1I%DxyU2iV5*uc>ZOVPBD@dtPi zR%-+M!E6zmm$)k8G-iNO8Kx6L3>4cwe|p5%Cf}c~UoFN#2ZMHVX?Op34h5H?Ci*UC zx%q-Jh}1#0tE^-u+Fq|Fby(j@Z5||^fs2=iDv5RfLpHqxzvF^W3S8+r4jZr?MKshN zbIS5;_9o>seTbeoO-pE^C#LXWq`FEU*hPEg9QKne+sQ@tThA#*bTd5F#IE7Zd`$u% z3XF;}CW6?CrMPBaS-j{pGQ-hs^ae=&N#-qRpIBNo1(cZY+OwjFG4e~-p(?R#Pd}j} zYeuK3N@AmyS$8Mu_FBU7$sw-m>9$^@J|QD2qs;xv-L6De67MC~K-vtIYr;ao{Fr#- z21`%bqsq5$H&7Bt34a@xp*`E7<**o~U%Q1rg_d0P1b*YB4F}VK84smWWC>gxPo*o@G3Y~F z+t5_E8P5BrqiZZTRU{V)?~)9NUmD|K81tWseU76}hr@02qxOi`28h8+QO&%-x&9i` zzmibCJu<3p&)e2HTryL&Y0st_Qdw*x0c#boKAQ!&0CQW0T0Tv@?#e3#02x?Sr}eJ* z?&owO0+(x6H9W7$`)Kw3<`mR{PBIdLRG3~Q81 z)Jg!sNN-V76@nPUdjnnC{*F5T7k|X(J7A#j7|U?uE|mmc@CS(8DIKP9U-hf7dQ}Vr zV^&$a)v2mw7G)vj7%^e)P0&QPryL3C?&kO!{`MOqYHoA|0_Z1y>I}Ym&CN}D*NN$T zs;@y7=IsEe$EFgDpqC_m3~o;VlZ8 z4GRy$k|=#?L!aqn6HUNnDmSk?$F!^%E#^6lT*75hDTGj0JCsF3h=sxtQ#pDbds;3! zF=~b6?>67`OIvN60Z?^6d5Gz+xwd;)y3fJ1dH6xn(Mi5tV|B+1tSqg{W{ZihkIc+2 zhJ^dNuYQz?&Ekkt=6{SN2AvtJST02D#UyVAeCi1cqydu|{>isfsubaQDz-cCs-I{h zL1I?^b=ICA z-*(fAPndI!(chZutc#9X;c&$rzoNVuq>(LW#KYLI2Wn!?pCd^Yi}E`3r7H1T>l&}Ms%_x> zOoWTz)yO~tk#hO>K>7H*<^;TgTL}&!9ua4aG3Kn(2wl!4kEisrzTc%DA-_^2z!yw;m)|e@&4{}!{pH)%%@%khsbj3$`I>a@ni9|o) z(B-gPz4>~?b$)}`PFu1VHoAUw0s>@8=Lf#;tJH5alk?5yoiK5oy;C>-WbN!Nzw^vB zrRP4$4SNHT$AXxeRwQp6!{()ewD)>m`8k_-3F$w>$z!CQBI~oMg>V`mO*3?-B|}9| z3>#50U8nh=M}?#YqmTHxuexsP>fcTHW^PuJycke3!@05LKyBWYk182@yKN>g4jYB& z#9CySqEeDUFr6R_pPo|i#MP9lo)D{@mC7zkE#&mU$aqfI#upYFt#y@=QVbwp1x|-k zwZnVS(>r?8qO`&RE}vPZY$REUpB!DaEydV})UX%z^8gu0&LSuB(V!hMUPLW&sQMOh zGHdum#b{QA!}g6EUcABVHpu2d{D&;8q(GoqKmDaWY9?*V0{RagMg>epj7mYqubXcGQ4 z#N5iN(M=|BI&?xtTNkiTHW{0-Ed$vD0d@j-PLsd!F$nw=4D$4L!yDT@Dmeg?*#n&J zm5A!tkJP{hDW8WvMn-XalXrY{{k*#W=#&0Xvl&RU1Tr#y8!J3;qIO>X$zS$z4>t;s zpYrj#6V&p9#6QurP7bI_@@mD>37N7P@mAh`M!=U$SEGH8t^*!)Dc6+Eml|^u6y0Q| z38fgcLUcEtO{A{3dwXg#w1N0T$#6D5j<3V~x(y0j0~#{2TV@Lx%F|}G4Hk+(=un|W-!jyd$|?#RO^83^rGz-*{PD5 zI?;^VN7*kZQvZ(6?hxSJji{J$eDs-#e8sZ;aZEg0ooof~om8ok#XHW~)=3LEiIf~s ze3EF%-K*H1fC>`$g+yZl zo3>(YZ7I`au-=wI839jIwsaK=@ks9csMc=dT^B$BHCCy4sDHKNQWYUrGu79~dV>_Y z06ycnzWOr?P`&evozl^vLP9%58#*Y)A{DA;6ofvNKgk|+yIsjL9G3;F&V*f(%w zXR?T_nb@BsE#jdZyNyh0THzey8VClYJexz3E91y%2eo9z;;U2%8sTA(j=9RCsf~*B%kYXyTB7{Z*f`$jWD5tG?zv zBIp|rSG~&Q0hp+v=Z*Y(oiQE7mP;9yfy-t`HK=oZH^S8c0s6xX{C9<-)58NiXz1Vp*qCpd09V52lo=EXcvE^G!zh? zJ|5~MRGrx%E^vB~FhS#4-7I~Zp&bxl&Z7FIgS6h)bo=L8Hfw$VnnH0d^uiXCOk8QH zJ+GXM<-+})$>5VLIQF+Y!K1HJpJ*5Y=H_C&M_)pIK#}7!&e~}!5q3j$tf%fvk{2D3 zS}YUvkpml|G{0OC;5Vl@)a+ajG1_X%o4#O3m}xWMT;pv)Jd|>@yxBS@g~ea|D_XhH z1gL8GKpJfr7s-LYFbV8+yvG@7b1pE;Jd~DeHzLIU-k#T?i`g2`w~zDiU7dqQ5N>coZs%W>Hy>LWdsi(vEtoD}p@R3*Cn6&T9}MfR2l|5L9sYXYY& z(aT>e64TMM2hnJKyr*492HQMIres1f9ou;2>$#JFF-O8LT`Wa~GFlW5 zzTJ8DUw!MHynFE5mvbkd0_^j)`Q**XMr>~ahk3%!5zO{DB1Cs*8Gz`_K=i$fiWfv; zGG=vlCx1#_B!NkP6X|63+3m##0H=Q7b-BEDVFEM|mi)5;&x}zWFt9UVV1OJ+y}<&# zA$=nuPJ3qUlKTcDStT#_U}&m5U}!X8XaTQNl`hv%2=7sljj;_i4jv7g5R(NYMAOfw zDNjvpe<|6mx3h@M-@)%3`9tuP_*j$Mi_awZi<0Bw%P%n|^nqkqPco+hfV^jcyap*Z z9_&jGib|X&8&m9ElLnm}2c6{H3%>_=q_V&mP%3IIBWV|{7NQ86| zP|?L_A9#G9?Uj+G9k#l)b9s#nFa_nh!9hRYE3a{nkvahgRvC8XthBu3A2;~M-noNC zJ?hbQskiCa91YzMX$5xwH{MU~JgmIF0H*2W;zzd3urnQe!ruelYW=6?EFOQx(ZGMO zPaXm!yO(Jomj6?@rokuMd@2*#^+y1m@tm?E*q-;OrEUk9I715#ty)CY{=3lM=>Y=0 zYH|Fz#s!LZN^eD&8Ng!SHr+V7>2gTvSlj*o*v*ipH=_W}|5d3ij*KE zQjayAJ)V5y>a}nhCna&Pr=L~KpIV$c{O*uk%a=!c?bQcD5El?qM{V-*-LofHr zwMgZ7^z)Nn-lrA2ipG3D?-Hc~xu6&jT$GyiQZ(+#4XBpR-I-gmw^-S8>7gv5cj(~} z=JlH66aRZzNPH{~02$eE5>M05kB>X}@B+}ZNo__qu z6?~yqk~J%e`{#>zO?kj%qdTHa5;opLTkZ2?rigWK@9WfLNTC`5-@_;q9dTw3Qs4%X zg;^gb)n@A^^Er5gOPEWajWPybxFJg)okH`TZ$Zz;Vc5M?hP>$_y-KdA?;fnRm}Ad9 z$ma;}UqsBksts=|D~0)G@%6?pU`*d#Vq65}t$P0|jq`;o(c>G+&y7QHxO|A)#*$3H zR)$P~=X{7p2fkCrk6||*H{p~djooaiwU}j^f*BQB;J(X?&6%i~bPcTShpy(d_m|cW zCl^_ZT)ymhs}!YMur(I})@86k^U4eU%a(r}zkfF-w^GKTaX48Oq&trWYjHxa?seK4 z&ERK`tRIhwW;7}h5HuuZ1VZnD04Uw0+!f_E8<+p)p0ThR%y_}rwa~gVfv^745OCGZ z{E(&9EUP<;u66NFj#&_TnY`qE(PH2$>xQ~?EUqX%mr7M(7%cq&uoVvdJe^#l zXcA^bJGg##f}>8DNAFF&W|nzC0g<@Bn?1=}mn}_PFOVVFFO>7HcQ{7U!nPNff{EfF zU0v9@%*V)>+j8p#_KZmN&X<+QV|4BM%DG8&Y9VhZW)*mbue{2&?G|6qx5&t+EF1H z_iQjF-_m+gxJXp8xYB~0%Bj|AeN714pY+ep!!bw*W*Ydxi^ib14;CA%30Hyl9Al~3 z1p%WQD6QeG{_%ddq(hYzCU=60`@PI$D%_%dpb@xT2M4K<2YTFFA$qTiLBwlegfRVj zhJUI(Y15Yo83oLq_fAWCa4q^#uazaIS^upt4^=F&sxJ2rM+UbS2XoGhDD+j zB|UDy@>4~k9eAk$%kOqM>@;NyV1cxz31BgEy}B3GEL$QQe?sX6cJ$IY`R=QIUo;%Y z1TlFMPRbhj!U2o-F`72N)R7A(bk9a1I-)g_mHeAh=~6N?yc zJ6g)-L0%eC2&Gz^Vv2iuDSphPM76K^S;(qav`HzXv%djdN@1EHVAqgm8qE7h9fenN z7%ugwU0mK|bYduVg+XLm3waZ|B|Ys@M?yTKiz!Sa1RmRVbT2y?(Ib~SccO<_y!O60 zdh}IDQLpImIE~mRLRdIdg=5s!&EgXS_iFR>W*sBrxyF%d{68e@J;{yUp)83pMe=lG zM%7E-UxzT3B$PPL&Ei=yg=CyA?~8I|v*0CKZorI; zZ2tov_iEE@MX74A$hi0HcS4Y@t?h757k-ULL%K`Xw4XTF^DB?R`{TDfM#Q^v2x3q}8 zqKn&d5J)5bq5@+-OB6fh4*#lFjWq{V6YM=EzlgwFH;ZpT_sCrY`)phUJ*;%t%bSLg z_LUO0AADP1>)FlHODzU1GZaBa3LHQUTA$+kWh-@5Z2vv>i;8t{b4zAU9Je`MdojE& zFj4PEIUzLMZq%uKgLnOWVMr>P;&II9BCG9^fF=>u{rFFdvTc!MV!jP|Q@eEQr$wNx zEiqqd%kj%Yn~GRr22nFt2=@JoT1k<}>N~vkjAP^FCnhPFu-x_0(d=MSj$zBL`yb}A!Jr^u+)6Uq8$k)Wf0s_ocA>pC+*Ha`b9Kz`r^zfanEC zkwU1yS&SE{Ok=<05+En)0PxBpY&4Z88^6@4l=;RZ{al6?{W%y9KDil(6JR6c>FqAs zTNKOJjCR+w$$jO4Ld~P4+@^xwdllY){+Q{y8cz*g6Y5!^CEN9tl6vx;DGE^}&*BHd zb_Zfa8FNxr8A#a}VK&^Lsr-CBXP9@ny*K$&d%TBS10C~43jD3P z=%8{&%&^{u>CHrxPt&xAMBhy1vj~gKfRDQTpZ0Hx>G5<9RpO^CSlA#e-|Va%2QWh% ze7b?WvxFSHcf6!oxfkAQ;uVQMM{*xGDOdT}him~6AR;bl(UjlN)}{DY>ZrHCNu&9h zO0C%oPt?L!`yQ*DGY$WmI8r=Z{(FnSV}+}bN~veUxBB^8?UCJ9u?LwW>)3L)m-aRC zphN`bpUH!6KirqCA89V>hT>RvG99#|@&*h){&;$^jR&9to>VUC;%{!lW;*!pW#rWb6;ftV`c2&vEz@hAsSl&hPE;Yuken|@S%kA=c|t+upJ zm%mjh@GMc;>gHtT>Ul-&(Cbjd_{javOe4xfGb!82C-!-^zBAG1bl>)l=bIRVQv94) zxw;6#j zcp$HZzI1~R`?WcujD{mL)e{%)H!wL8k%(pkMHuCtF7DrIBedgw@!uyi5tEY<@5Clk z6DCx7{fW8tqYF!3DxY*%ykP62xKPX1FClK{thGBYJJ8&c*`Cje@bkBJ*oqq+$8|8f z1x~wE747+j>*@kDA69oVe1vqCL8|!CZ1|vOx^o|;ey=`Xj#<@R!y4OoI4ChcbxRgl znrWo1K!4CAP}{l)ot81pi{+LI47>e7Zr2fQ|6qZd;vp#bExhQ9*v7{cb%Uf8wX!F=RhrJkY%V-lVsp_aS@JWuN+R2Hs7wymACR>7 zTF+QSLtnp{%MHJ&Z<|;Q@VV6=?G;&#V2vLr20#n( zJc9Lt*LjvfFp6&F9+Q)>G~Ie*ihjffjytXNy6sA5uLnz5Z)M~Gr}e^(ato~n&2ndl zR?ZA{AGmx7D+jd2pC6Ro7q=IKgO5pP%2$iGG`Q)xG6MOUU1iAzv?!ndY|lutAIklm zqTQJC!KMHK(KANHPDlj2UtzD%L{hoAajz?@o&O+HmSFiPF+4VQZKRXY$6}{br!Pxe z6>lBgcDTxSG()(DSTgWe?LAmM=Y`;MkiqRKJqt>ws;;M4hxzfMauDuQpyIlDb)tOI z0~@~&>hA4|4&if&k#~)mHSlBDTjhU1nWteD#fu;Vef@a*HfEr!JbxR#Mzgqdj+3UgmsG6vr^ zT4=iToC-K>CfIWdKhL6Sjc(?6rhBAl)8F&KnpY0t<<5wR2NvfAQ`|3mS<@HV-lzVjS4OGW$JqDkUQD1)>SMW^ z`=<2KpY@!5QxQe*Kz;4P`FK^gw5m*-=hKZB7-teVMGsLy;IJIN7ZPVLO7B-53!*8BQYM~;eWVa_e-L(8{|c_i0@qet zeA+c?TF=ij@WY?YlAsO|=wgj`UFA~*sEVvh@d=i7Lq;&dyDH(;fXk2`@H_|s?Crq6 z(D3tKkP1BQYcSxpny4$_ZUhbxkVcN1d_UVzF^T_U=GI85i5|yT}rO|GhlC3mJrjKPs)dYj)(s zMj6P}0*a71sO_S2EHkzssLQYpP{EpgJ#yUfZuU? zT%u1GZ>VeZ?#_!=@fWq@4VnSDZQRPz)}4;;k+zC8KIIo7q+rrdF*B_iK7?jY3pI_! zx73b=w|Zr5Ck`X68HaE9y^D~dFuLb@JJAwJ(ju{?7}=d(J7Dj1SRd%-F(4lfyrXbq zsdpUI#xls-J=YGRQ{WyKx=&b1QFUcjup`woAh)I*fEMw=)Q#v<sp?%+a~tMO16 zz2`*HmN2v7zBlZYa`~a394jv@Sbj9Ib;jBTt|=VT%Td}*@D0_w_#W4Z&JA#InkFc* z!V;C9MOUerqKY!-Yje3)2kwzG-EnP7`&Ht=&z&ji3wW!kVeSY6J~h_NgJKLWq;<>>HYFTKG;?V4mjm846iDyqVBp4pPNy)l#LQoMN7MT zj)E{Qb(y5r4?Jnu+tfoznqiEl2>iJtx*TRLjo@CZTjL%%0=arwREoN}T#{QG;rV?Z z%@3}tnl!zrvYG5ENo`bv`)cJ8Cup*F)kqm}acgB{;6dp zb+WK-v`-36fO4L2V8ATP>Hx{XVW(BhD%3?YnuzRV5^Z=M^{qx)+2& zucc9wqCd$ZzVVSIg;|t^J_JmXWgopI*+a((h<^ggMl@tpYo0*XE+3H`57@a6v+r{4KLeE$o_ z{#TIw^0a6i$ji=z{F(_REpT6n6=4Q$lJmK#>)C$ptfHExCC}f30XVqIPO5tWh!Qt~x z^_m&!j#s$>3yMU{4%9-wVmF|Gxc%PcQIYM zi2ZG9a~h|2wW4}t7eUhSYcF@I?&C9quj@%O5LA@&n-KSy)qocl{ z=%PHFReLn~Rxc=cQ5ifCp#BlK6-ifR+za8Cr=nt%7GMHbX{1ZL1*Lpae+?QFK5_XY zf-FJl?Bdij6e+D7KB1F>IGS)vFo9M#`pKoVX5Y$JePu?-<&GU8WmUg?7mKNLS~N|k z|9E&_vGuF-X-!=+b>`!nE?-PR%J|h+XT&|-!|F7&BmqAE6u{0DR7oLp3f>w#H%1N< zYR~-Eeb5Y$zgqf0{((&vXl>F=maMLe)B_tII~`pUjd!d|Yah=Na3Q>`I{9OTu9lS9 z#ZDm@*LI=R{{O|^TZcv2b#3F=3L+{cQlb))N=SnN43g3vk|Ny=iV9ML(v3KD_s}EK zE!~XLF~dj=F~qkAz3=CKp5Oa^@A3QVJHF%i{z8wru4~6yd+oK(bFIB|n&HhWcuAkP zFJos{WXRdr{w)qD(4e!?nS^T1+wW56vz5U29>?dk2AAxKFq*$iC1H-ug5M^7125Xw0yj%9W?og>+d2`+Yl$HIfD#pMNC^OOD4dAnXOpk*dNY<% zKh}9jZG)^G;SlHO?E>DO3H|rCXWo>;ib(PEDUftO#10H7Q2{@oZ{!HF@#vccI(aDfg%NrEJmU!hz?A1;{B>hK*iGaylzl<-YPPaIypL5 z`~9wEBomtt?D7ZAzFEyvA4r)fDeqeLow-Y7OxGt{E`T2`_bnsIus=KulgC_Sx`dDVptTx7D}slk%>;`CR+H_^zAmu+f!i z%E1R=x4;9@dAp(ipB|nAY9!cl{!2`rTMJ7w1t-wGq%W>if>;N9aB2w_SoAhvk zMg(Y`9Py9VgNee+mrjMhe-O5mu1zBkECkinf4wd}{DpZMWFH7WrrrDyVA5P_(rpM7 zo8`qAM)?M+c;bj~iMOB(CHS8b|tWhrY25poJ=al=kY6*Ax|M|hgUSA)$nf(9!AbPcs+1%`=&Z)0FUOXutRLG?p|IzgB4BJU3_glpk@^wZ= z2?>&aChUn z>j3pb-oKv`EB((wx>>>&T|<9`M@*r|E`97(T*2~EnaOHyAA<(Q=IO9kWm=(~?99UIX#wmL`9`>We)#ncOC z{OebuRLlCMaKp2gTTG13&IT7IZYt3n&T{!YSRA(u{i8Cc2-Y8cB`}%?%4D$E<89YS z+@enK>oT${Q@`c7M{}@Yi2d=n5xHp?WEXoxcmJI&voV#P_ z<|ubw@z1yrZ0x#rogDw_5;u$bbQtlpyF#6NUah0oRxS9$b>mcHf-$$7-vjNF5u&qI zh$brd{_bJ)$=aiCB>ubM{9Oq zHJFF^u3d%Pa|QVnjJY3RS0s;6EZ5qagT$U_&iwf%Zf&Yl*!AQ|3GMf~*X<+w|9biw zgjBr`a#2=ax@(& zz>7FUI`^Y2a8yCkX|@TdOJ$gO`Hgk|;)}pVzkb&J;;m&YI=9;S#3=sNe8={m^bA)C zC4ca7ReynwLh~{FTkRWFpOs6$(95P+O;ztVTsYDAPwEUYWDHOWa@%;{oJaUf1^I4P zp}O(O8?oT@U7!8-hkHAUz6Yu@oQb$&RNW;vGPUs?`HO@%DvBC9JoCU!xLGw{Ll#Wxi&-@Qvx6t@mNO;G7iSPYxkP`E6F)kC1@ zB}#5_p-pq(+&t>rsIf*GDlt@5TbsY+xf=BQ+yFmQha4~}mWN{n zu|}s>)@o`1=E}xqTz<&GB_pHBn!*j??pLoz99YX1&yFe0w$#6b&LvLXrbkWJm3j!8 zW9{7cu!WtKDh(yDFTGr}`>i_$|55V zJ+7KtM}X<1)ZLq)d&+lzrwDdxX2cc3=If@um3lPoS-s=23mwQby33(a5PRRq4e{+D z32XHD6Ncj3wHjKE9gCgW+9ZRX#Etg#8=u(QWId^BI;8;wPYCX$tN4eeOw9ChsIvQ8G? zDe(2RBqR9B=a%q3GO!ui5!Z`zS27V#3nC15Nxm1P-l-+o=@2Y)OthxQ4RK!&tE-uw zd{)QXtCXSWS?7&=7otEGOj^B)UwxZY($hqkC!*zd>HaEd$i0oAPe3^_u=@{4*+{a zdzWn;&qpxTvv*SqFN6hNl3hb!hCh7Rm!%AQQ1R_~5o)QGa^`4n)_}t8DCuA^1?qM9 z^k~st{Af4y>`)DPq%57hK`?i}hS*-yVR!t)XwipWqmC=Gu~6^DzAfkJF>C(G?_xf^ zw_!aH`YOxyFYv*VS+Sj$<4@NFPu7(e9L7MEOzxc(Lz5tGn`WA>-?7@S!%x$NH*iL9 z*t{6p$AM*kliYErkIpG>&8`!GDKwLYg#cSu12aYMSNPY5%VCO zi8wW@E6(eUnb({p5XWBdycPlA7yr6KGb2RFq%hQD#9qzPKlH#5kQ`q=(gBP`5XzO9%65z-yGl}rJ19Dhl0#jFGd@DS?71H{oT8n^6diw$+czpZ zrezfnN+GMm-J!N)qW!Bq z?U-7Rx$pbGe%To-+?hY#{bu}X%!~0=-#7j{cc?qcnAnWBpOQ?QHEs5^bV&Gug@65; zEzB?ZddlX}27Gn!5;CUxNu*d=c7Y@>>KnJM z%KBjBeE(6l%6VpH&M0DjY;T~k(kzmW#q6^VyoY;drEj(+UAE<>&WkoK;r+eBJ^Be) z$X&JY`u%5j+e`26lm_)MhGi6{6W#4j<7YIFX1~ev`53lR5@?Kc-}2^&7)dSMQR{?| z*j^+lyo(!W5n97VU-h@{a>{V zZ$+5TI&RFEw`K&IZqsqGj5@M2|NPZHvl_uH$3IpeH_5i$YL<~GO39oO73Nk5DRbMY zN5|bW-;75IH2_jBNdh!(51tIYS*IL6C+Kdg@5=>!{xe_^|bkI-YM zHLq-yC_d;|Y@FS#?9V?mkhE%Ot!)z(hDrB z!wS~|ZjrtWvw%9ZqWQ(QRTb4h3KJjqwU|rrn942&aHKN>xrU0@wt-z-{9vJhMn+G` zWO`@0<6_8jQ>W-hW?d`F;fw{5;QY+y>u!lQvAPJGSBpL9CRUN%uh!fn?zKC0ARp`a zSEFle=LZu>MKP^vj()m%Dv5rFVj*n6aBL^i&f9Y~Jn{y?7;pLH@g4-!)4G(i^Ke+? zkGu5wxx?nt<;G3+_I&DSFp-;`Ao-N<*cg{e`7BDq@o0M`+gDA2b9rCe?8|j70gN4W zLLKJv7;eAr94wSAuGFlap+bgZ6ko#aG42vWq*&h`nR z)ciL#hl#4pjnt_0+)EE=uhi>YgwWu=vD4#bln2Ab!>G+aokcTp^pEukP7F+kah1Dg zI?kqr#PVwSxXSw5S2;k`H_GzUAF5Gmu(oj(G#Eib_nX(R7(}F_5)0_;-@=B?a%@9o z?yoK8#s~=u-FfI8s#06_P{&gwKEXMdaJK1s@PwD8_d@hm| z^ekEe5V zqF%m^Y=lPN^W=~6#QPW1AG%WbK`!68o};{_=^f{J=jD9rD`tB(3^bS24Q{w>iAQ%+ zoU?(nT6ayZmvWGW>c>>=yeyuIN8GWtcCdIml7=FWNso56XRJBonP@Z43`Qqb7piH5 z-aP?)UJ*a|5PN%;oMMA_DqQiisrQ~)T&=pp3abOFRV5Cdp%w%pzY=ey0#|_`SgBO z*r3#{PbW!8FMojrxE$;T4|`b5Y6qLU9WRm_z+Hmax4akLR9Nd^-4De2tl?7t^_*Wp zEI^|_-}f~~kH2`;Zw#g0=)2w($0tWcV1EZa0S`o$O5cV%q37;Z@j6)t_XwK^={a$;r;E^o5;{X^ob3Iynvi(Dr8>u zVD&zc5j$0?e8R(^ao#CecRi0)|F>HBmE|`?=}&vOWmv{yC%nI~-(@mTY91N`QD6)~ z#unf9L9V3D*t*e)sqMebs;&&5IyxLrCWk*B~xrjGN`T3rAE+eEVhUuT-->aCUH#ntXG2J<&2e z$-30tvwK4{#)02T+8xbz;wo0QL7B+ADWZ&_+Lgen;zC(S*9UbS&RQCmB-H>h<+TRm5{0hsdbzcFOwM2m9 zH{q>OG#qbuKPx2JIhyo0#SG~MAEx-ufz#uj@kDdbGOMHzy{~C*spa>LWPRxHZj+qH z_{v;*ZCSo-jAaZbT`9J)ec^&M7Bw~-DAqIFNh$k-Rd_{%(|vih;cbdd-|haRYhfx# zDhJdm*i&6F(|d>Wwn%SR{yR&Ry>(>)la6E{vbz+0jurW9OZ8XkbVA34t{pABmu16bQS6T%>797k-+LqT;M+p!YpKTA64YlQu z5;>j?itTO|ueD!kT9BvC$gTuQus>pG%f%g9j38`0czZZKxYO8ZDZ4P26gri%ow~Q4 zfUW158y!)8>sAZwuW3)-3#sq+DV`Q2Rf{ZirewE{^l9_EelBPKczXCIYc3_+fY9T4p)jtiRLE!btaJxA@)^x!(GyfId*hwl^?fb zUIkGm(Ihdf9~=m*D*+>?%W^Npjm1J)Rmveyud64Jp9aQRU;_6Q@?6mDg|uErj(@bk z+Om<#Afl^y*xaS8_TPsKo5if{&3}?qMhL77RM~H2&)a)?7sKA4^EBbOph#A;R_oXt zqwnZ2yNH$+R-S1zg*TaHd_aq+Z_sLI8(;%b1qXhWe9t^HTO+5X*+Z4Y^;y@PT87K2 zCfJ&w$$P&V>l*#F{^-Tv<|f8|qW6QoPgqUSOLLdLc$j1^al0(ui!*M0=;W6NZuR}BtN{wNrFjmw>Bj{J6@gv4OX zxmh|%#r8%A6BovaBeU-KHj~x)$BJ9NN7ptzfTK-HeTS%|7})445JUo_FKWB?Aw!-z z9fSJ{OF6UL5gb7UZd&qOTN{S8h61moayi4&GFbXw~tCorF@(*ze&)Y zs1~c~nA@EoI+Rf(h52l!O~w^lkbVh5-YjLfv${wRi8_suFqfMO7ju{On0%A=fSN}! z@!qs^HJYz;M^~uMVZQ#_ofF9Ahc$&VlZ$Xp83(56voSK8=L*1dAH1g-?698)7n;}EBN+z91(zD4myE830-1B8~cF@>BGn(8(qiVHXl+xC# zgWaO`u;&G)`uT=22Ss75t60zOG9{xaSNgBG$r(>4j;I^L`Jd;m=y%o#fp|fXO+77) z?gUo+D_&6YdGT$Wr|Py=5n`1!?l@hKba0lRpB(@mwWY(ZUU5{b>SJjWqV4WieKrgmEe%06eX=Ya zbQN&6_a)EsVAYO>P@R*kU+mAG@8p^G=)8H3z>av@jKOrK8LG}eOfzae_rVA2=Bx&s z&Z!z#Xo%J5+KRfOm_1_s48y4$aE4*d!<^5PjnMPbw_M5y+mTP}5cA%6km(ca2qjMO zZdZsrbt!|fvTDo5u6K*Er_-NyEtRoS7&crKc zX#?6-Vb}m6W!A=9tW&~4es@qTgi%E>azYB04|p8piNUR>z)Qr|x?f;YtiZeWXI#k6 zK|t%Qz?piN3yWg8^Q-Q%;=}mNh zx$pE}_HXpxo@@#Zx>NplJn`X7yCBuukH9T^RY=NUfFC(G`X#nc0%rr6|M-vZDvu#J)Z^Dol?%1=!_JO==+^zB5B^Z!PKYu z^jYk>C55^*zFn0j25N$pLWFAPoGnxXiPFTv*N1g7qq*RJ*P&a>10i8Td>Njh8D~4&h3F)^$p0@4MbxONr!cC${P<``MHC%e2c^ zq9@;K3k~dPUAMS-?E&0IZ}kEz(Z2?vb_K z%b9u`Q*Bqj9c8x|LK+T7L6YE`L5Q+}?;&cu{BNOk0pZ3g!EJD0w$|rHi@kB2At6rf z5ynq1;xcc(r9HP}jNt7!r%eCM9bG<2O#Rm&KS%a*b1z6pTz)mBeOR#2s8^l0Gytik zK;&i{9;<4&m2g(lOr1Q{2mvo)VPNt#=MsTH#ruF7&QdG*ySq(@-w;O^tr|FzB*WLOpWgoea z-xH>CTHy86#uZ$qkWw%P`XuVfK_*bp83VV}TMvg%)Vu2m0nJw%_kcr{q_5B?L#KsT zDD4k%lxw3ePGUea27)Vw@Y=?sYz0F|>Z>=j+rgr}q8eqEF@GdN@K*zGXy50Ya~b4v z5y#~*bN2jq9~VyX*`IxP_C&Mq6P>ansF|n5Dmwcyul{aaIN56>V$S6E@0n+R|4u^i zju4e)V1*f292ivn_us&~xv=NuU!nWW6(E4Gv`zWXp*Ohx>G>#x{*e7nXjiMNe!v@X6rGs=MhOP>@dowgz5(8#ejIfY*6w>bZc zXh9tB>={yWfM1J0&wsS8-QhN(hheZ+9&(PARa??LdK zu_G8z)=M=<56;xU2oM?Ifo)Nb0}PM(zxp93>-;{s9B6dW^CZ+#wT|tf5Z79PfNP_n&Y7GSU0}_3ER2gwH|oe~gXbITOC=+O=w1F8#6-W5mNlaN$13 z5V^Cf>`0R|C|2=C28R^weddGi)aZ}_((CYQoA0ow%C;uh0C?bd;=h@;o+~8Bh~vfMe~moAysOCnv-Tq(nD%M{ z3bQT&D`GV*fu&QgI~$kYJ(g@07(E9>VH^3Q6w zTA#-e-f#Wv|C-i06-eyr#s4Ee6)lc7EA9lL*q-pIpIW#vc$Egt(i-ud5S%OX<~8lP z3%Q0Yi2E{9HHL(&w>sPv4tr?CO^Nm;&Qy9^C=TANc^AMScWOT z7Cg^)77*Sotr6dzST>b^gPwkL%MkL|5(1x3y5WSX@fj1x6WQaPXJr>1;Pk@P?Q|L) zU7ACs<|O5{0Lp_ww&5|^?cCVjpKn*`Fy4jAPY#Sc+a1^*b70=~`XU_GI*Uf^g7 zP6@6In(32G(cQ%}_;%{s3_J$ z9{?vsMMXl+->CV5mZs}AJ4l$uJ5@`d=8#Zo+!w~t_3;Y82i(_}c?)mpD)t@bqQWs_ zP!F@^2g@vA&X$>@ve7!fC*EiIyLkO^&M(AuDKmFDeDzxkDyuUkod zd$l6Ia9<~3)bb;R@svOr$B8a{i7=m0xuT;n7t=G>(mj^^(8zi?t31v&zjrb*T3geQ zgOvRR^^TnDZ8*CFx+}kxzyoCw|VJox+VtZ`n`xaG5LY&AEk0Q_Q8; zh3M@}57dn`x+aQ|-{F~xVq*%qF~1 zQ%Yg;;>nS-i69SQe0XbS;w^t#1{4{oq*X9s4bc-9Ev5a|Tbgv#z&n zx~6eVmtzk01OhQ9bUNQJ%Y31KO_9R})&?SpE&a3<@w6|cFg7>Z6u%Behroqi;7+CS z*)RMMl9vM+k~J>*pt^Uu4{9@hAMH;)-uZ6AxCAD%(XehyqT0!wALH%?02dN5I~{DF z!O=Oft?haMS=b=!yFH1dWO-DdFHDx`mnYU^&fHl`PlmVFImwPF0R93n^cKrA4_MuB z)_UU=)>IS}One{Ay1kw@JG(gPpeB<=J;C*+bN;InvS*sOvrqRY3zh~T#L{eY*%F=| zu8zp_iLF-pM$(|OjB(8JURv?|E?x8hGMw~9=9iGSFvSAHb|0CC>!m*~`MQaj6F+vE zTczI+@5QaR+TP8Y=HO|SU=FjLqQJeuXReG70lMn^Wj9p{lZgagi%S0O!$U3l;arWK zCqHebDm<5i$9Munk%cjXj^2xwE^C@UMCm{pz>0TY!=Yse)AGKm`c_A$qRjYU0WH(i zT^qacnx}(0L`cugf*PAo4z%tGh1XV zwC7$w`U>?cr6nJ;9p;;M5TK?F**7}cD$G#WH|mr7#O_sPSdAyv;tzUq0a4&#dz9r^ zg{UU6`@>^f3H;%);jkLlx{=S1y=TSbN@>9vmrvc?teu^C`AmJ@$bh>qAJ09Lc&u7A z$m87cW3DMZY^!>+N4PqNn@vOR0U{W_`vC)p9djB4}drgcx@S4q8Pm0Z1q9*#$9e(sFx`%?0`4JfGH=VO9%Gm)UdY*YHRb{e(Ff6WI{;~r$`fg9m#2y2N-T629K>u`{nTB1rpYCFUh+EvCGW&!k;`d zIJ_I4rX7xp3lm(pxBjIRoQ$L7$G$rwjilS@$H>%+xDTmu_#Q9m<*F-$K;uFAy>OV% zMBPqv$nw(xY{5q%9k@+F{hRFI#iUUAXWA9!BTtrT+b_VLRf|Q<>)2Fr@)!>_oUvpS z@E%`nfuwbsq|peLi2XVbpY16nwcdg-WoSxZj5QS^dy6`!8oN4d zN#R09aQc)mp|)^N8l9=hrSV>=m@m->?vnz+cVBW@C^hUQaigQD@?#qx*w%gYWKwT# zX86#k+KEkYHl<@HowrbtEu?jqK`q(Z)IHAl0k&G#DQC0^MZpZ7j!6x$(dgt>iuN$6 zF6-ESbeX)AUTf5ACDKmtk8n;KMJC?9=-oXGFX~1yVa+(pbXv4ESq0mwz!<> zlNs&0rg`h3j{KoB4$^J|GD;{pV)pAGOtSx_UPT zo1->*Y)U7TTE*&q?qgf#zk@sr*JX!%Z3=!(gpb~k!FArFq|{C|Ow|k9Fi1phb@8&d zXOvg_i{R5;53(3WeV5X`S|c`+N|moL2kwca(C_ptY{`4781aPfpcM6MYeqlEmor&l zHg*mC)-QxN>ja5M&Z`Ax(fHNkDLlm2nJ*^f#j-Mk)=zH#N3D06BYBbbr@n2 z5`LS3?MS)q=K7xaJ^ygk8vDbV>zK)^BzwjIO5vQ`MbvWs%vC_x)!+ZI+QJpdT?%DreE9i4dJG1vrQ6@hw3rdO3G@ghu)Gzj$1f-rqz}~rp{zfh|2(E*bz*oQd6auKl4diq6Ux-YVKe$GXsTv9f)+P@pj09rP(SV3X4#wNN(-Ngss9*~ zufpurjGL`%ib7!(M>2sE;r;!q7^|3_A^p9KXpO>nlgSUlMsDJ**jUG}o;ITsvwd?p z;okVA8gpnit^Zqi0^@z$roC_=JZQ3v8$jAqRW1>^q%+b;U(cQ{uksQ7cCO$5&^$Lgc(mN`@>5u* zOK{V8ApeJP-n!qcG&hlkJ#I)TZmMH$x2f2`CXd@@*n57S8SLQgu^9b{pE<&qC81yG zt;CGX6uu*>>_M@lzt7A}P zQQf-LN_73!`z$5c#qx*~?%xrmq6I~kW(`GkC%f;zIdw=66v}jR?R}0M(wy#Nj5i%Z z(P_e(e<;_rs`~kIY_92oKj4|=3*t>fp?UBPRENj_|*WY%EJP=NM)5dL^K*;?Kqrx{vLt>V$w|Gm&!zc$5XD`w;0 zDe#YUV)MV==I-s;&jj#E*C?-Tq?r!zGGum@`1<%SdbOl*r znR1-`mZP(u?j#nzj?3Jr zK7@gcO#N}p%XW6c@2qDTTO$v=BR^$5i|{8Ab*d5Q2kdK!IX*oM$Tb&^npiV7_|OT~ zs(1wHEF+#u1bdecs(*ewJ-9XR(5__tvW`0&cgfF>BgyXO{cJMb@q7;oI)oN-b|#D9 zt*~~r(k|lxKqyYyjw)UJ_<2GJxcoejOn!UWk{*qyX)%-+!ZmC1C|Al$!(BbxVSl6TV?>)FZ~nI z7HOZ`e#)gkxBLA-_CcqNBAV&hcm4jnGxYgYxHX$~PYQRLq->DtVG;r>H*PL!oF$PTS#wOX02gUcM5VLT~C$UVczLUT$gvf z!EJr2lPVjIv_|hvxM}xSP(q<`CB^pcGTWE~x-(DrQ`+ z`uWNUy(6H-U30pO@Ns*wgDf`F-q8Ujb*&`P+vKfBr~O3TRHB)0}-jtDxe0|N8*>u6C@UC$D zUyL(tcTb=Ol$V*`XBt{|hfza@h!FdmD|V^gT3L$ZpqfeFB$^q%i%vRnY;;18$D2EtKv#NW}k!27P)C8^z;cR*21#@S`XohK?&Q1jeCjO!I zAYgVAVCFaHC3!MPtf#80yd*lZwCa1H9sY1zq|{}XOa+bJ1mP#IWRV&N&C*k<`f%UK zCTkg8ZRY{`^svVDm*ZTI$M$~y{Mr9xip$RJbj67rb6qh;xfw^I48k0@ztjhgNltZ-Xy5gV~LtTw!(Q+8-{Lc^sP<>Uu|BCYOy?F(Pa34As}z z4}NI@xsD9%iLGYz-iY3+JoC)>p;@wkimsWXc95pJ{aZlW#``i#ns4?)nF2e>pGmOB zob<;Fs~$DJWCPe;%Sg2_-2}b|D;kO&NmL0B8FF^>A5e3b z$oQlU%Kl?x0;tdDzu4P|>J#?XaCvV1!z4i=Wb5TyjQECw0m&SQ{?j}Y({+xZmH=!M9vgwUPG3qZHh~;N&igYZHm)X;nw5*W%bzDg6wdFD< zQ`@b21J0E5l79xKN}BtcRTbbw4ws{{2^=rK0*E5zUqD{oX;wUh^nds#fRcW65BxGV(%|nmJ&?9<1={ru3U>C^=o_#!?$tvEPYL! zu!Ls9IWATAhXr42E;C=cy(zCyP-VmbnaxR%c z=H{J|pcj508T`-Woc(0R^1(q-sEP)p3WQGbNIWy1(|#a#mAlFAQPi#e%jcBVo-xO~ zrIk$m)n$t~Hr$aT#hA9$G`YSSO>v`8EZE{u|%;WzPiPw!Ek~?YO{b--H zJM(#3T!8_WcKS<$fh1!&~T7I}fue;-h@r=4YLpedD8V_bD*XX83Su$}83@3bF625O#kx z#XNH~_`P~m%K7O1DaXNllXKqUHC_U4Ip4JIurxAYP!$oaF!cHg17*h8{|>)bj<3qE znHr|U;6;~sYof1g69-A&-H$^ZCnjxLC_E0r1ITCRIXuOwgmlW>4f>woq4@1DHJCMZ zaCsnrxK~U;LN$l}R&=@?08s%H-j?_SI$>K6Jd2f)!ih@zw|%hvl->zWO#N=_^otiS zoLAoQzas0a^IVLgmn$nw4|f(3elwq>!DSe5bP!Iov<+Kv&amUP5mpIwrjS6Psj z8t>SNL0I2gs1#49e03NfnPwWzT|Y?rc~V49L;>Vl6FwXb)>4JO_@cUJVYkv`CVR7X zzat`=+eK@o@Ey!Xu%A!UVbWFxiW9tAF-=R-6^DCg6rhuXqkKQz6C7mcZoLo^;?t#nfZg^WD$FyHcn0A%6C>l|5~gZ z2%9#0(JWm^M#uHH7lx-j6?gPWkoNnkA(h|%!ufqOGfnD`2315&e}W20;KJqW30o~< z_pr`?`QYCSfb&NWP_O3p>&-+-mp6dSX$1d0G*mz>bSJ;*^zFlX1e0KwAVpvGOI= zwc@oE&Tc(n%B*M>XUy~%%WS0{zFw}m1-fT!^er5RsZp!w8RabnCgy2Ei;!@R>=`}X z{zunmK|#u0bRov8yN8xYbrQLyFi~UFTH)SriBa&Dr%+ajrcQBYFI=_Jn?!nu?3axY z^9OdCRs1do3=Px(@PgD8`bk6!OWs7vD<4}jo-}HkY7ZKCXQy3hTX~~IyAi~@idVb- z#mpt8n=HbtZ@;l*rwLcDtZ1Bo&9nnjb@KvVPRp#BIp)bek2oNZ?vDeZR-ZiLKGko4 z4CNToIICS?@~Dt>PI(5dZhdHeF5C;yk@>-~ETwmI2kR&nm{02q(@+1>A&~vt>*(t? z%3>Ccm|7W`gYZu}ZLhBM<&WI{ZTkKd`rH>m z3rQkIfa{Ljsas0~^ZK_`tG_G6@;T|-B}4$EFnI4R2wXdY)@ot7tfdak?^;5syAp(N|DhOY#&Dt| zvQkOf50B3%YlKz1vibCW0jA&davRE^*);kWD4c7KM#<_ibO`Qs78KbQYTJT(8GJ3= zTgZ9+Fy!Rt84x){WYF!tlm|u>+(M{Tx=#zE3ma%!dV@Sv2cK5Q{ruJ7oE&~>%zqoh zI%W417eHVCG^)>f1f;SXxtS1x>nP0y9k-V3VY9^KV3SUjH%_7*N`J#y&~MnXmCEfL z&G5b2+0prZ6LO1AWY9J8@vmGcRKcJqHZ!PU)+V~XWHO^@wo!W+?$|aU14!xvk-g1OFi24%zegef9)gdY; z!XR@7G*7GNyv;TiRt;dMekp#u*45;`Xk;^?Q;$WIs$}NQzh(6S<=aq^67wcaPAy)u zN5_z%x+m59&HQi(j8yYhHgtup^A86H-kqhv%Xg%kw zm!@8QawnX@r%IXj4N$juQq=ZJfvepfFm-df6RA%VUf3SI__q7T(Tq5vDI8ANgGmIX zVuE2oaWp5DVlwo{ESuZNNsqTQ*#RV;YoHFPy+OMfCw|=alw_(PdR~B~)v}!P-mw+( zg4(24(@4@Kr|qu92W8-hPAQ%)huTwL?L9>qSH1VMk=dqHq1=V#$ZSSypN)0{&QI=% zOj1M(q2b-2f@;7dMUz#l+D#5v$j}Rqmn&q|7`f(6?sfL=teQ@Gzl!$sfBAAilj*&~ zUF*Kv%FnPmdp5&Syl&w9zrhk2ap!Z5$Sw77CP_&5{Si!hZ9xs~?9uQnBbqiKK@Mpe zfwoVtEu8?RqkgkTk>w)=w8J2gfsyJO>zZk$;P7mpl}22nd^_GC9ZmK5CMEr{|Fpep zoHf~aSJmP(DvL(Mz@r%6wu=eT!oqTm59u;t1qFw6^5aYXVVruRjLh+P<;;B2{@{*A zvu4Lpq~jMQxn~-PwO=!RD1Jtq4>-gyA@mqZJ87Vckee0J4~Wt#puH01HGHw%PQcT{ zQd2W_FZs&TBbQu61bt~&qr3*9fR1aGud~|%`>F?_FS=bQBhBMmqhn=0k|69ZbF^a* z*kKVyPu$M{%y+y-@J_ARdOB@q#Xy%+h4~LDX{zu`p3Q51n zF=0EqDbs+sSovG{vf~K>397qbufP-hEFpOLw zWd!i&v50VU5zJt&5KhUXy7uCF4&E672mrUVB&aY^w#Ps;TK7d2AbNT@(fG1HVtT#m zAWpFH_svReb9rrgU}|>M9lNJjM`-m$;tw-SkR%-15;sREBmkp7B-q?p#NAom*x&;# z>Z})3vlPhkK^9vl3uevP!uN&u`iS}U4y!$C^G4YFyJme|sVf}YzdV>BF@1sUdNH%+ zWUmL1c=-MVou24Qy0OO+)Q7HHNmV0E5+5PW-WHD6A-l6-pY>s1Z1Z(jDNe*EWvODq zCPxFaG09U5NCN2|`2-;ermGXii&H*2G1GOJI3CZ+`BDefa31f$jAgoksgT0jgS5-k zcMJ;Ac0)<7PPDuIv;N_vF&;tYD;gjA)J(mKVbvay6*h5)bEG>Xg$Ax*mks@F93*U0 zXj+$j4{^B)@PFZIcu=neoI{P}9DoMl3-Lk8*lp6!;1)(#P+u*j>;J{dZ@$2rYl4w9 z8_#ECPyUhXnwef06S0%MVzS1O8C9ojdx>v&R-W6t-d}qTs7ZX80Ww)t}Fcy`8uJQ%*s%+%eI9I6r(*b})&{e}yAdY_<=Rb9K zdjhI=bAOl9QN{J%Y^Mmo$K(!P8gty2&I3!9$>03M*i#Y`R6y`Q&u z>51e-rfhAR1_9Piq_6eoyQ$Qt)U#U06v8>lUrYj5S02c=m;*Z8X$lL zBGN>p3swk4h_r-`%AsQrLaz!)5d>+WgVfNG5)cAJ=`DmNA=JA9ddeO5-XHhJ8*hyF z_P;%{v&&j@%{ABjzWJ@N&SFq*SL7fr?{+-q@`G2BnybOEDbq4O;oF=gNI#$RFX1MX zbT~_SU7N+z^*d@#PHWL|BLt4M*nbQ$58@~s_*Wyq7+~;V`1S1KoM7{sLz~04#m*Uo z(E{U}rC-e=K#XtD3>38_lDGA$b6DBX;i#xT&$IAR%%lH{A$5RtZu#>g+A&XJk^%cD zVg0}nk5T|cyFLKpqc#wcPY#>gxPmvYS?h9@OiB3g;XJ*VF#E5uWpi1MM5{JYP>`$1 zQYS3I`P)S70zh@z;%n%F+I%eleBZ$nS^AjZq>l(TmE1nxC25VctYe|CgCI_ zJ8&=+qSGNmJ4x`pC2+>@y^pT}Ba?2Wx^gx*e&QMl1dsae$%+%`2BlTymnR>?HX41t zgC#Ru$AA(+K-F$VTFF|*n+asaStA(@R=16O`IRsa_&TVI%a=Qi3uRQrT?~zUcw+3c zyFAJ@kwBHe&V4V^gE>q9P}tuJU{g2DTKoZoi*|nSkxDACcr$V`!vL=_6!COKKa4QD zI^z>3IY9{}mxnle&EaIJXSWRTU2qIWyZt)6`l?E=4oKivxRRGwiz9EqEBn|}u7v(rD zlOJjU!l^uglu)mYzQ(k%o{^Ti7F*Am2^u4w+3j*{ZQDuUQuD(gTeBr2$(fiR-JPD6 z^lsuXzq zvJYJ>df`D+f47k-pGtfl<(g{XKdmrJ^u)!Mmtk{=cm$S17>D&8^15I;R-vy7r4K2qWcfm zB*phDSz$TfOP_)=LiGrzt1aDvx;ZhYCbTt{FIM0RwC!zRR?63VXkmj)3IJCJC2@YT z9QKZmB(ayi`5;732<7%#d>g}5+ zP2{hp!oI$~sx6Kkv$SH1A0wX-alm9G^Z+PBitsArV@PfAn04)$g0ZtPW_ARO{nHdM zi?(xLoG}ejbc_|a_{!EVI!%=zhmw-w;QPj#S~0!o;$xphuq`{> z?;ERR!!P7hhuz+2E!K2f6E9n0!5vL~-j1EYW)II62{<}ZXra>BTEYe6db1mX9?^0j_YxWsP-of{VZiz(J4&B5?Z>(gv6VB{ffieMY3_ zy_e;)4l@!l0|tVB)?PV<^N0>nwYxdd_*gNNJnC7rE=6CQ@{P^*FoqBJS@}syjJf`B z4x@!e?x!+ukPD0YP`9nC z1TwS*E-8y2pSvq#r;x#dm}bJ)hVxf!Ua7J|85!G?ZMTk&Cpg%_KouDeXt>aZ#=?hD ze?0d4`b_jn3iqL;L7c`pCQv&j_78j7l9<^>=-Ef6pHL2G%N>X zdDLz{d{`HV&(UfAMFHyY8*?n_^-p~_e%ig@@Q38;$ND6Eg$Mq1&DrjySF03lU)Dm8 z-5>Y8PJIDk5gN7Xvw4*F+Dz8P_tETXT5`6lF1vU*^ou(@AetC1!f9 zck=g3C1lCsQDpCs5IF;`-3vbMljV6V-_jVx&UPW)(dt`S@)68*UXrH3V$$aP-UmP9 zo-!=$mEgn^AGTA>LXO=4}kma9+0)ShkgZ|BF6Fhm#RC@LE+2piNlvj z(W3y)XB^L1|NldF8)D!8vE~|E2;%NNy7$nqFF>^B;9dX1#J%&MYhR*!v(9jlK%((q^%}hiO$=B2S)Bdd<-afM!+N-#m?sES(Psij$?mR; zBovgtVx`HuYDe9tW2S) zG zuMpZmXE3yGYnf1Wc41SN-k|`YulsIxEQ8t=-h=NjEV80`Bw?)wo$V4zotCgna|0W# zAW#|M1JSYUP6b;pFufZGg&%{q8*MDS$_y!}e#9jnQ=iJ=B7^P*Ab;SDs-D6^NbkP* zecc-7gDZ-s0R;p<*NjOpg98l>?d#XK0HMaP)G6rDFj$rY{Tm-|Zxxm#0}1l^X4km6 z^ejyK+}E5V(CEx|3LB`I)4RD((_$wJoekIb^{xaA3 z!sLWd1h+p)ZGA?#Q=)&TX` zl?4RhM55Lr3dlvIX>iBI6|aPtV1}I$dOBv~0SAF(L)0bAb=`&;U;gs^u0EE zUuN3&+c;#^+z5i3Tlo$WafIW25H!YOSI_Eq+4Wc>RFnd03_u0O7-L!6DAC~`B3y1D zE%6X4A)%YG)UoP*tG*4djYK&9NMZs>6{3+D-7!Bqi?&ODu741)!$4Qd#o8qR*QjI@ zQbd08nd-F>ND@r#yQ}!ORa4L_C0;wnBR1dO;5>V-hf6lbpy2vgB;;R2UJl3EUV1KJ zK3Zy-2(kZS{PsMQ(o4nE&H9)UN^*lb*6YVJ1@q zkyxW=;*ke*KMq4NKtc&12D2X{`7edV758XAS;%%mZHwqMFgV2si`j^Nphe)hweARZ z9vb>Bf{aZKvqpegsq;G{f#Yva{A*BkP}WK1bTiT!Y|kQ3#*jQWS~9~NpU#I~0o-H& z89D%cAf0l6S8J}6beL3L^Oer2TuL#-fA0M07yd%RKPj0F6chb4jpB+ZVW4%@z$>eE z{GJQj+Db+gSj!m1mpuFZ`^@_S>q?nIB8oghiapHQ>zR`y{Oi!|L9iX=W<5^m%-S_5 zB0x8`O6uLIT`Ld1Xoz>X3%-Vibe}roKGwl;Ar{%Rl))Z*$vP^v`7qz-G$7b2(WM8< z#fAaQewYNK44oJ%b*#KqEW0y^1@*!NVCNuX9aOAqaFoVKb{1K5U#(ZVZI2UT!90l- zZV|?A%`_D~CuuDv?X6t>Kcnbhz!DoL9m_28w^JX&HrHTKZR<}~PL@F+&Lgt`QKt+y zX)lRE|F!N68TU1rFDWAdC9!uA-?dTz8Y=2xx9}xR=ltLFJg}wC#WiBuVNx_8J2Du0 z!+_ApC|(;Wb*Zbo$pQpadqL3}Kip=J58f>Tjk(}%4p8B7!Q0BP+Xi|=rIEn3^Mkr0 z<`F#|6;I04o~(@GOh|(RC=zg+{9y1;4>MzCT+3$NB5P~DZswRj5A;NuRLiX1nR9T= z$vfSD-~Z)|{~7UCvPIi_h%<1UA+oTWc za+BZm6n3R$&GDpxNMrClS0TNydr|j`)J{YK&0yZYFC74vW`}s7FDsv72D=)F0-2fD z2Khp(PEtqzvJDbN#Lnw4ukpW`x4|-^nNga-I9_0RDdz1>$H#zl#rN62xD)$=Id_$P z_jkqn#qB>RKkEBr*LR9AlUldKkw5!#H;IUd$Fkk|sCC%pLePoyPTSK_my@&}-D>sI z7;JzGX;nk6s4MbuT|I_j3u;Nff(+s*e4c87Kfu$>$#xy)7vPul>{-|H046I;XGop* zes;KAhjZ{`FQ%&0k#}c3{@u(05R2@$1Mm6lSR*6!y!{8`6>mJ}1Hs7ulRs1+7&lkA z9lo>k3kbZOo|t2Gk^B-FL5%hYh~X3O4FO@iq82W>UWUd;hV-p5lI{aj`a$`wAQqeN zMwN|^tWt`comZ2+^=zD6b_ymtG-Q&VR)l>2$_w@mAIC}*_^W30IGFEcxtznm`5PX_ z1+aJi(zLhGS+rt#hrl+7h(Wgbw6({zZwrr<(EX^=n&HPvn?^{QOg99Eve>Tr%yx*CJI$|5FW0iW(A2P) z0gH{F`r^X*kY@0K)V5whYL&F}R4@=Mj5c$v9R^QNOz4bBZ?7t8WGD0ICar+FAY z^|4}OJXy|ij5c+&r{@0_f>hTF1Ssm?P@=nczSJ|Zb+HR6zM8eO>a+aF{;O)8+%mZcP%zATyu3nv z7k~HV8x)7}!)HqI)`rSq4U65HLxZJ`D&_N0!l(dV5{g_A3TGD{bY0BZQ1(Mp2F7cc z*Bxf@8|v3Q4UA)x+hVLe+K9;dZ?1g3@NAr$N@3+jEAQ@TC{zzkZaP)fEP~2CpLtWF zmcQe0W32w!!RSj`!ELdq#liflz)?7DkY0LavZ4A_HJ1=IY;`JElZ1c2J=_Zl88cerKs4UE>x9lZ#qS7jwFFj4f>Lm-PW17nS#hW zfo||Aqn78Df~@0@w~tD8)Oet4YQIv_`YEbma%zG6E%1pdwga9_7YA!K2T{) zx;*vyjd3Eor_+|>r1G!)o|}`zOV-U^TJPT*MaK0^BKmK&Y7m1I2A^&@lh<|AAeg~( z&P5W@$V6_=3;oWMtbMF&!E+<4Bu@#DeqZ&f#9)h08UVqV7B&RmQBtTyXOeShB(V_2 zFgj;y<%Bz?Amwq(mA(Yy3eeQpT}L(0Cjw;i1RwDvCKDj8=Dw_?RhgxaQS(DI^WG=R z%^AsN8lF?UWOEtd3fNwG9Z2XdLgnCgDEBtTH#%^ zT&X28D7Z8gKdWFWkkD16S2vdL$)kHaPiM8`BV%B0P|EbkxZu`yU??{=l(6%a8ye1i ztu39vkXRv0kvaa-Zi&QQ?`3x{nQ!Zn*?l!Zdj~X8kkK%<>$zJSTdo&Dv|IWuA~4e` zf1z1SLt(`47yj*9RML?nuwtWCme#q$d{5E9J269JqrXj|N!IpsQ@SUsL*p0b(eb<^ z0(>xYk~QhBOVb&IfcsPQ;uo(n2yymfxv)_FFdx`RntaKkE9J?cv9s$;S%>AWS(dzv zPHqx#vA938c?~*xvDUsyDn{)r=X`5C(|j3TCo|L!7?Ni2&XwPNeQATJ#Cg~4ks4fR zkqqz`3&4e%920S)Ux_ur^_B&f;^4p(Z&2)G&QPV|Q+oU&q z(+Godrd^Gr!Bn>J<_!ElG!_eC+r-xf+HS8N6x@=3n5ob0Vo zsng$*` zS0ejUIi0ftD|engq|^^v{Im8E9JL6%gQOLnjwj*$x!{z7M|pv^i@<3$WR6v8^1=p2 zKg`F=;88=RrzVEWcBt$%|M;9SK6SH|75_9fCxwHZm;HRbUAtoP7BQ1jk1ukJ)6}!m zM{`<&1YGq_*?e_z#{qLtg${>o;H_)fGvt6>tDK+IaHmD@&BP4fb3rFRvG|dtsT$w0 z1Swwmy?HcN6w#K-|DR@SAw;0#7)Y_*>KQZ7-5XK=QI~CCsL0w7mPe z5vn4*%1~Rvd1;I`>vk)YBC^K3Fs|uT$IGoiPVt-4bCsHkWJP+$R- zN9(KCvl`x>+#Jd4bOm<*>(jf!E|_ott8 zY4IFMCXYC6Z_?RNY`9TE@Xp$3uwM4vlKa&!=8XJug<~Tf=T(M-x5jXqp>P&A{YwWY z&ZCCQ)r8oYI&oU{O^84-F|lQNOQXsDH5I|LVpiii=61Ix)q8s;%r|>%i#(kfAwN$~ zI^WBNjfL7H`HAYrd5=|6EYR2LkV{R(-^v@yMT)?d} z-l}rbyH$pE)(C%==FQXCIp@d;HQ|HC@O8vUiMd?1`NL3#e#kt#Cf{42x2>g929=kJ z?|5`g4lX?=4*?&)E-8x0r^1C!s;jE1;)3&gdptZGLElIfq9iKm6S0frn1RoMqhs){ z@s>zLhEQT-TX<}U_tx1whJFr|HJFl)xcC+InZV?19t-B zVr4Nlp_W#C+3MN2*pP3OGf-%-YJGowAQxhe7@4ecTls`$yo{7}(%*@QuswQZ8o4e;lx_ddy zwdxd=@qww?u|eBv+h-o0R&fB`*K4%}8+ZmfiOWMc|0iJC22`X0Gp(+6u&VR62z~bT zn}zr$H6h%V*Ew$-Nuxks8n+yJHKDgVH+n%GaQn0Jb zCJ`#>#tFVeeFg<%th?3*LK!GQiXvl0@#<%WTT(p4@I?F0y!!8cSi9FUVuNUX8nFZ} P+oz(WaXtI${TKfMx8ZJ) literal 0 HcmV?d00001 diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/auditing.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/auditing.md new file mode 100644 index 0000000000000..c3824e8be0996 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/auditing.md @@ -0,0 +1,270 @@ + + +# S3A Auditing + + +The S3A connector provides an extension point for auditing requests to S3. +The auditing can take place at the entry point to every FS operation, +and inside the AWS S3 SDK, immediately before the request is executed. + + +## Auditing workflow + +1. An audit service can be instantiated for each S3A FileSystem instance, + created during FS initialization and closed when the FS instance is closed. +1. The S3A FS will request an `AuditSpan` for each Hadoop FileSystem API call. +1. The audit span will have its callbacks invoked during each of the S3 operations + invoked during the execution of the API call. +1. This allows the audit service to log requests made and associate with users and operations. +1. And/or reject operations. + +Thus: auditing mechanisms can be plugged in to provide (best-effort) auditing as well +as hinted allow/deny security. + +* Why best effort: coverage is not complete. See limitations below. +* Why "hinted" security? Any custom code running in the JVM could retrieve the AWS + credential chain and so bypass this auditing mechanism. + +## Architecture + +The auditing subsystem is implemented in the package `org.apache.hadoop.fs.s3a.audit`. + +This package is declared `LimitedPrivate`; some classes inside are explicitly +declared `@Public` (e.g `AuditConstants`) while others `@Private`. If declared +`@Private`, external auditing modules MUST NOT use them. + +```java +@InterfaceAudience.LimitedPrivate("auditing extensions") +@InterfaceStability.Unstable +package org.apache.hadoop.fs.s3a.audit; +``` + +![image](audit.png) + + +### Interface `AuditSpan` + +This is the central class as far as actual FS operations are concerned. + +1. One `AuditSpan` is created per Hadoop FS API call; the S3A FileSystem's `AuditManager` provides this. +1. Each span has the name of the operation and optionally source and destination paths. +1. A span may be _activated_ or _deactivated_. Between these two operations a span is _active_. +1. Activation is on a per-thread basis. A single span can be active in multiple threads + simultaneously; other spans may be active in other threads. +1. A single filesystem can have only one active span per thread, but different filesystem + instances MAY have different active spans. +1. All S3 operations performed on a thread are considered _within_ + the active span. +1. Spans do not explicitly terminate; they just stop being invoked; eventually + Garbage Collection should dispose of them. +1. The `AuditSpan` class does extend `Closeable`; calling `close()` simply deactivates + the span _for that thread_. +1. FS API calls which return objects which go on to perform FS operations + (`create()`, `open()`, list calls which return Remote Iterators etc.) pass + the span into those objects. +1. And Any S3 IO performed by the objects (GET, POST, PUT, LIST,...) activate + the span before that IO. +1. There is also the "Unbonded Span" which is the effective span of an FS when there + is no active span. +1. Calling S3 within the unbonded span is considered an error. In the S3A codebase + this should never happen outside copy/rename operations, and will be logged at + warning level in the Logging Auditor. + +### interface `AuditSpanSource` + +This interface is implemented by sources of audit spans. + +``` +public interface AuditSpanSource { + + AuditSpan startOperation(String name, + @Nullable String path1, + @Nullable String path2) + throws IOException; +} +``` + +All Auditors implement this interface, as does the `AuditManager`. + +Implementation note: so do `S3AFileSystem` and `WriteOperationHelper`. + +When a Hadoop FS API call is made of an S3AFileSystem instance, it +calls `startOperation` on its audit manager; this will relay it to +the auditor is bound to. + +The auditor then creates and returns a span for the specific operation. +The AuditManager will automatically activate the span returned by the auditor +(i.e. assign it the thread local variable tracking the active span in each thread) + +### class `NoopAuditor` + +This auditor creates spans which perform no auditing. +It is very efficient and reliable. + +### class `LoggingAuditor` + +The logging auditor logs operations to the console at DEBUG level (to keep the noise down), +and attaches the operation details in the HTTP "referer" header. + +It can be configured to raise an exception whenever an S3 API call is made +from within the unbonded span. +This is primarily for development, as it is how we can verify that all +calls are audited/identify where this is not possible. + + +### class `ActiveAuditManager` interface `ActiveAuditManager` + +The class `ActiveAuditManager` provides all the support needed for the +S3AFileSystem to support spans, including +* Loading and starting the auditor declared in a Hadoop configuration. +* Maintaining a per-thread record of the active audit span +* Switching spans on `AuditSpan.activate()` and reverting to the unbonded span in `deactivate()` and `close(). +* Providing binding classes to be passed into the AWS SDK so as to + invoke audit operations prior to requests being issued. This is essential to + guarantee that all AWS S3 operations will be audited. + +It's a YARN composite service which follows the standard lifecycle. +The actual auditor is instantiated initialized and started in its service +start phase; closed when the Audit Manager is stopped. + +```java +public interface AuditManager extends Service, AuditSpanSource, + AuditSpanCallbacks { + + /** + * Get the wrapped active span. + * @return the currently active span. + */ + AuditSpan getActiveThreadSpan(); + + /** + * Create the request handler(s) for this audit service. + * The list returned is mutable; new handlers may be added. + * @return list of handlers for the SDK. + */ + List createRequestHandlers(); + + /** + * Return a transfer state change callback which + * fixes the active span context to be that in which + * the state change listener was created. + * This ensures that copy operations get bounded + * to the correct span. + * @return a state change listener. + */ + TransferStateChangeListener createStateChangeListener(); +} +``` + +### Integration with S3A Committers + +Work submitted through the S3A committer will have the job (query) ID associated +with S3 operations taking place against all S3A filesystems in that thread. + +For this to be useful, the work performed in a task MUST be in the same thread +which called `jobSetup()` or `taskSetup()` on the committer. + +## Using Auditing + +### Using the Logging Auditor + + +```xml + + fs.s3a.audit.service.classname + org.apache.hadoop.fs.s3a.audit.LoggingAuditororg.apache.hadoop.fs.s3a.audit.LoggingAuditor + +``` + + +``` +# Auditing +log4j.logger.org.apache.hadoop.fs.s3a.audit=DEBUG +``` + +### Integration with S3 Logging + +In the logging auditor the HTTP `referer` field of every AWS S3 request is built up into a URL +which provides context and span information. +As this field is saved in the S3 logs, if S3 bucket logging is enabled, the +logs will be able to correlate access by S3 clients to the actual operations +taking place. + + +## Debugging + +The `org.apache.hadoop.fs.s3a.audit` log context contains logs for the +different components implementing auditing. + +Logging of requests audited with the `LoggingAuditService` can be enabled +by setting that log to debug. + +``` +# Log before a request is made to S3 +log4j.logger.org.apache.hadoop.fs.s3a.audit.LoggingAuditor=DEBUG +``` + +This adds one log line per request -and does provide some insight into communications +between the S3A client and AWS S3. + +For low-level debugging of the Auditing system, set the log to `TRACE`: +``` +# log request creation, span lifecycle and other low-level details +log4j.logger.org.apache.hadoop.fs.s3a.audit=TRACE +``` + +This is very noisy and not recommended in normal operation. + +## Implementing a custom Auditor + +A custom auditor is a class which implements the interface `org.apache.hadoop.fs.s3a.audit.OperationAuditor`. +This SHOULD be done by subclassing `org.apache.hadoop.fs.s3a.audit.AbstractOperationAuditor`. + +* It is a YARN service and follows the lifecycle: configured in `serviceInit()`, SHALL start +any worker threads/perform startup operations in `serviceInit()` and shutdown in `serviceStop()`. + +* In use, it will be instantiated in `S3AFileSystem.initialize()` and shutdown when the FS instance +is closed. + +## Limitations + +This is not a means of controlling access to S3 resources. It is a best-effort +attempt at supporting logging of FileSystem operations API calls, and, +in particular, correlating S3 object requests with those FS API calls, and ideally the jobs. + +* Low-level code using public S3A methods intended only for internal use may not create spans. +* Code which asks for the AWS S3 client may bypass span creation +* Application code can also create a new S3 client (reusing any existing credentials) + and so have unaudited access to S3. +* There's (currently) not tie-up with OpenTelemetry. +* Uploads and copy operations through the TransferManager do not pick up an active + span because work is executed in threads which the S3A code cannot update. +* There's a limit to how long an http referer header can be; operations on long paths may be incompletely logged. +* There's no guarantee that audit spans will be closed/deactivated + +## Outstanding TODO items + +* Statistics for auditing +* thread ID to go into span from common context (and so travel into helper spans) +* Maybe: parse AWS S3 log lines for use in queries, with some test data. LineRecordReader would be the start +* log auditor to log AWS Request ID in responses, especially error reporting. + + verify +* javadocs for RequestFactory +* Opportunities to simplify + +Tests for + +* RequestFactoryImpl +* callback from AWS SDK, including handling of no request handler diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java index bc8b71b3b3bf4..17136244487e8 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java @@ -141,11 +141,11 @@ protected AbstractFSContract createContract(Configuration conf) { @Override public void teardown() throws Exception { + super.teardown(); S3AFileSystem fs = getFileSystem(); if (fs.getConf().getBoolean(FS_S3A_IMPL_DISABLE_CACHE, false)) { fs.close(); } - super.teardown(); } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java index b1ea45f686528..5e0c052ea2200 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java @@ -25,6 +25,9 @@ import org.apache.hadoop.fs.contract.AbstractFSContractTestBase; import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.contract.s3a.S3AContract; +import org.apache.hadoop.fs.s3a.audit.AuditSpan; +import org.apache.hadoop.fs.s3a.audit.AuditSpanSource; +import org.apache.hadoop.fs.s3a.audit.NoopSpan; import org.apache.hadoop.fs.s3a.tools.MarkerTool; import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; import org.apache.hadoop.io.IOUtils; @@ -59,6 +62,27 @@ public abstract class AbstractS3ATestBase extends AbstractFSContractTestBase protected static final IOStatisticsSnapshot FILESYSTEM_IOSTATS = snapshotIOStatistics(); + /** + * Source of audit spans. + */ + private AuditSpanSource spanSource; + + /** + * Get the source. + * @return span source + */ + protected AuditSpanSource getSpanSource() { + return spanSource; + } + + /** + * Set the span source. + * @param spanSource new value. + */ + protected void setSpanSource(final AuditSpanSource spanSource) { + this.spanSource = spanSource; + } + @Override protected AbstractFSContract createContract(Configuration conf) { return new S3AContract(conf, false); @@ -75,6 +99,7 @@ public void setup() throws Exception { // static initializers. See: HADOOP-17385 S3AFileSystem.initializeClass(); super.setup(); + setSpanSource(getFileSystem()); } @Override @@ -213,4 +238,34 @@ protected void writeThenReadFile(Path path, int len) throws IOException { protected String getTestTableName(String suffix) { return getTestDynamoTablePrefix(getConfiguration()) + suffix; } + + /** + * Create a span from the source; returns a no-op if + * creation fails or the source is null. + * Uses the test method name for the span. + * @return a span. + */ + protected AuditSpan span() { + return span(getSpanSource()); + } + + /** + * Create a span from the source; returns a no-op if + * creation fails or the source is null. + * Uses the test method name for the span. + * @param source source of spans; can be an S3A FS + * @return a span. + */ + protected AuditSpan span(AuditSpanSource source) { + + if (source != null) { + try { + return source.createSpan(getMethodName(), null, null); + } catch (Exception e) { + LOG.info("Span collection failure", e); + return NoopSpan.INSTANCE; + } + } + return NoopSpan.INSTANCE; + } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEmptyDirectory.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEmptyDirectory.java index f736be0c17331..c86c475ba480f 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEmptyDirectory.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEmptyDirectory.java @@ -20,6 +20,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.s3a.audit.AuditSpan; import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum; import org.junit.Test; @@ -79,8 +80,10 @@ public void testDirectoryBecomesNonEmpty() throws Exception { private S3AFileStatus getS3AFileStatus(S3AFileSystem fs, Path p) throws IOException { - return fs.innerGetFileStatus(p, true, - StatusProbeEnum.ALL); + try (AuditSpan span = span()) { + return fs.innerGetFileStatus(p, true, + StatusProbeEnum.ALL); + } } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java index c0feb85de1584..a7acc361c0ecb 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java @@ -28,7 +28,9 @@ import org.apache.hadoop.fs.Path; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.fs.s3a.audit.AuditSpan; import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport; +import org.apache.hadoop.fs.statistics.StoreStatisticNames; import org.junit.Test; import org.slf4j.Logger; @@ -78,7 +80,9 @@ public void testMultiObjectDeleteNoFile() throws Throwable { private void removeKeys(S3AFileSystem fileSystem, String... keys) throws IOException { - fileSystem.removeKeys(buildDeleteRequest(keys), false, null); + try (AuditSpan span = span()) { + fileSystem.removeKeys(buildDeleteRequest(keys), false, null); + } } private List buildDeleteRequest( @@ -119,6 +123,9 @@ public void testMultiObjectDeleteNoPermissions() throws Throwable { Path csvPath = maybeGetCsvPath(); S3AFileSystem fs = (S3AFileSystem) csvPath.getFileSystem( getConfiguration()); + // create a span, expect it to be activated. + fs.getAuditSpanSource().createSpan(StoreStatisticNames.OP_DELETE, + csvPath.toString(), null); List keys = buildDeleteRequest( new String[]{ @@ -163,7 +170,9 @@ public void testMultiObjectDeleteMissingEntriesSucceeds() throws Throwable { S3AFileSystem fs = getFileSystem(); List keys = keysToDelete( Lists.newArrayList(new Path(base, "1"), new Path(base, "2"))); - fs.removeKeys(keys, false, null); + try (AuditSpan span = span()) { + fs.removeKeys(keys, false, null); + } } private String join(final Iterable iterable) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java index e6ebfba922d5f..c35b4bd96f5f6 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java @@ -38,6 +38,7 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.audit.AuditSpan; import org.apache.hadoop.fs.store.EtagChecksum; import org.apache.hadoop.test.LambdaTestUtils; @@ -111,16 +112,18 @@ public void testCreateNonRecursiveParentIsFile() throws IOException { @Test public void testPutObjectDirect() throws Throwable { final S3AFileSystem fs = getFileSystem(); - ObjectMetadata metadata = fs.newObjectMetadata(-1); - metadata.setContentLength(-1); - Path path = path("putDirect"); - final PutObjectRequest put = new PutObjectRequest(fs.getBucket(), - path.toUri().getPath(), - new ByteArrayInputStream("PUT".getBytes()), - metadata); - LambdaTestUtils.intercept(IllegalStateException.class, - () -> fs.putObjectDirect(put)); - assertPathDoesNotExist("put object was created", path); + try (AuditSpan span = span()) { + ObjectMetadata metadata = fs.newObjectMetadata(-1); + metadata.setContentLength(-1); + Path path = path("putDirect"); + final PutObjectRequest put = new PutObjectRequest(fs.getBucket(), + path.toUri().getPath(), + new ByteArrayInputStream("PUT".getBytes()), + metadata); + LambdaTestUtils.intercept(IllegalStateException.class, + () -> fs.putObjectDirect(put)); + assertPathDoesNotExist("put object was created", path); + } } private FSDataOutputStream createNonRecursive(Path path) throws IOException { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMultipartUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMultipartUtils.java index 4746ad5588bd0..24f3ca65b527e 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMultipartUtils.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMultipartUtils.java @@ -21,6 +21,8 @@ import com.amazonaws.services.s3.model.MultipartUpload; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.audit.AuditSpan; + import org.junit.Test; import java.io.IOException; @@ -57,7 +59,7 @@ protected Configuration createConfiguration() { public void testListMultipartUploads() throws Exception { S3AFileSystem fs = getFileSystem(); Set keySet = new HashSet<>(); - try { + try (AuditSpan span = span()) { // 1. Create NUM_KEYS pending upload parts for (int i = 0; i < NUM_KEYS; i++) { Path filePath = getPartFilename(i); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java index adcf578b05862..770a99599f508 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java @@ -1352,7 +1352,7 @@ private void skipIfVersionPolicyAndNoVersionId(Path testpath) // enabled Assume.assumeTrue( "Target filesystem does not support versioning", - fs.getObjectMetadata(fs.pathToKey(testpath)).getVersionId() != null); + fs.getObjectMetadata(testpath).getVersionId() != null); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardEmptyDirs.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardEmptyDirs.java index bd69ef28669bb..b0a86a2cb78d7 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardEmptyDirs.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardEmptyDirs.java @@ -34,6 +34,8 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.api.RequestFactory; +import org.apache.hadoop.fs.s3a.audit.AuditSpan; import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum; import org.apache.hadoop.fs.s3a.impl.StoreContext; import org.apache.hadoop.fs.s3a.s3guard.DDBPathMetadata; @@ -43,6 +45,7 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.assertRenameOutcome; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; +import static org.apache.hadoop.fs.s3a.Statistic.INVOCATION_MKDIRS; import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume; import static org.apache.hadoop.fs.s3a.S3ATestUtils.assumeFilesystemHasMetadatastore; @@ -73,7 +76,9 @@ public void testRenameEmptyDir() throws Throwable { String destDirMarker = fs.pathToKey(destDir) + "/"; // set things up. mkdirs(sourceDir); - // there's source directory marker + // create a span for all the low level operations + span(); + // there's source directory marker+ fs.getObjectMetadata(sourceDirMarker); S3AFileStatus srcStatus = getEmptyDirStatus(sourceDir); assertEquals("Must be an empty dir: " + srcStatus, Tristate.TRUE, @@ -89,6 +94,7 @@ public void testRenameEmptyDir() throws Throwable { // and verify that there's no dir marker hidden under a tombstone intercept(FileNotFoundException.class, () -> Invoker.once("HEAD", sourceDirMarker, () -> { + span(); ObjectMetadata md = fs.getObjectMetadata(sourceDirMarker); return String.format("Object %s of length %d", sourceDirMarker, md.getInstanceLength()); @@ -99,11 +105,14 @@ public void testRenameEmptyDir() throws Throwable { assertEquals("Must not be an empty dir: " + baseStatus, Tristate.FALSE, baseStatus.isEmptyDirectory()); // and verify the dest dir has a marker + span(); fs.getObjectMetadata(destDirMarker); } private S3AFileStatus getEmptyDirStatus(Path dir) throws IOException { - return getFileSystem().innerGetFileStatus(dir, true, StatusProbeEnum.ALL); + try (AuditSpan span = span()) { + return getFileSystem().innerGetFileStatus(dir, true, StatusProbeEnum.ALL); + } } @Test @@ -128,7 +137,7 @@ public void testEmptyDirs() throws Exception { fs.setMetadataStore(configuredMs); // "start cluster" Path newFile = path("existing-dir/new-file"); touch(fs, newFile); - + span(); S3AFileStatus status = fs.innerGetFileStatus(existingDir, true, StatusProbeEnum.ALL); assertEquals("Should not be empty dir", Tristate.FALSE, @@ -137,6 +146,7 @@ public void testEmptyDirs() throws Exception { // 3. Assert that removing the only file the MetadataStore witnessed // being created doesn't cause it to think the directory is now empty. fs.delete(newFile, false); + span(); status = fs.innerGetFileStatus(existingDir, true, StatusProbeEnum.ALL); assertEquals("Should not be empty dir", Tristate.FALSE, status.isEmptyDirectory()); @@ -144,6 +154,7 @@ public void testEmptyDirs() throws Exception { // 4. Assert that removing the final file, that existed "before" // MetadataStore started, *does* cause the directory to be marked empty. fs.delete(existingFile, false); + span(); status = fs.innerGetFileStatus(existingDir, true, StatusProbeEnum.ALL); assertEquals("Should be empty dir now", Tristate.TRUE, status.isEmptyDirectory()); @@ -198,11 +209,9 @@ public void testTombstonesAndEmptyDirectories() throws Throwable { createEmptyObject(fs, childKey); // Do a list - ListObjectsV2Request listReq = new ListObjectsV2Request() - .withBucketName(bucket) - .withPrefix(baseKey) - .withMaxKeys(10) - .withDelimiter("/"); + span(); + ListObjectsV2Request listReq = ctx.getRequestFactory() + .newListObjectsV2Request(baseKey, "/", 10); ListObjectsV2Result listing = s3.listObjectsV2(listReq); // the listing has the first path as a prefix, because of the child @@ -243,6 +252,7 @@ public void testTombstonesAndEmptyDirectories() throws Throwable { } finally { // try to recover from the defective state. + span(); s3.deleteObject(bucket, childKey); fs.delete(lastPath, true); ddbMs.forgetMetadata(firstPath); @@ -272,19 +282,13 @@ private DynamoDBMetadataStore getRequiredDDBMetastore(S3AFileSystem fs) { * @param fs filesystem * @param key key */ - private void createEmptyObject(S3AFileSystem fs, String key) { - final InputStream im = new InputStream() { - @Override - public int read() { - return -1; - } - }; + private void createEmptyObject(S3AFileSystem fs, String key) + throws IOException { - PutObjectRequest putObjectRequest = fs.newPutObjectRequest(key, - fs.newObjectMetadata(0L), - im); - AmazonS3 s3 = fs.getAmazonS3ClientForTesting("PUT"); - s3.putObject(putObjectRequest); + try (AuditSpan span = fs.getAuditSpanSource() + .createSpan(INVOCATION_MKDIRS.getSymbol(), key, null)) { + fs.createMkdirOperationCallbacks().createFakeDirectory(key); + } } @Test diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java index e2915884cefa3..72b1a8a3d2a19 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java @@ -22,9 +22,8 @@ import java.net.URI; import com.amazonaws.AmazonClientException; +import com.amazonaws.AmazonWebServiceRequest; import com.amazonaws.services.s3.AmazonS3; -import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest; -import com.amazonaws.services.s3.model.UploadPartRequest; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -37,8 +36,12 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.fs.s3a.api.RequestFactory; +import org.apache.hadoop.fs.s3a.audit.NoopAuditor; +import org.apache.hadoop.fs.s3a.audit.NoopSpan; import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets; import org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase; +import org.apache.hadoop.fs.s3a.impl.RequestFactoryImpl; 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; @@ -81,6 +84,16 @@ public class MockS3AFileSystem extends S3AFileSystem { private final Path root; + /** + * This is a request factory whose preparation is a no-op. + */ + public static final RequestFactory REQUEST_FACTORY = + RequestFactoryImpl.builder() + .withRequestPreparer(MockS3AFileSystem::prepareRequest) + .withBucket(BUCKET) + .withEncryptionSecrets(new EncryptionSecrets()) + .build(); + /** * This can be edited to set the log level of events through the * mock FS. @@ -99,6 +112,15 @@ public MockS3AFileSystem(S3AFileSystem mock, root = new Path(FS_URI.toString()); } + private static T prepareRequest(T t) { + return t; + } + + @Override + public RequestFactory getRequestFactory() { + return REQUEST_FACTORY; + } + public Pair getOutcome() { return outcome; @@ -147,8 +169,11 @@ public Path qualify(final Path path) { public void initialize(URI name, Configuration originalConf) throws IOException { conf = originalConf; - writeHelper = new WriteOperationHelper(this, conf, - new EmptyS3AStatisticsContext()); + writeHelper = new WriteOperationHelper(this, + conf, + new EmptyS3AStatisticsContext(), + NoopAuditor.newInstance(conf), + NoopSpan.INSTANCE); } @Override @@ -305,17 +330,6 @@ public void incrementPutCompletedStatistics(boolean success, long bytes) { public void incrementPutProgressStatistics(String key, long bytes) { } - @Override - protected void setOptionalMultipartUploadRequestParameters( - InitiateMultipartUploadRequest req) { -// no-op - } - - @Override - protected void setOptionalUploadPartRequestParameters( - UploadPartRequest request) { - } - @Override @SuppressWarnings("deprecation") public long getDefaultBlockSize() { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MultipartTestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MultipartTestUtils.java index 861824277aca9..abbf87607c12c 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MultipartTestUtils.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MultipartTestUtils.java @@ -22,6 +22,9 @@ import com.amazonaws.services.s3.model.PartETag; import com.amazonaws.services.s3.model.UploadPartRequest; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.audit.AuditSpan; +import org.apache.hadoop.io.IOUtils; + import org.junit.Assert; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -54,7 +57,9 @@ private MultipartTestUtils() { } static void cleanupParts(S3AFileSystem fs, Set keySet) { boolean anyFailure = false; for (IdKey ik : keySet) { - try { + try (AuditSpan span = + fs.createSpan("multipart", ik.key, null)) { + LOG.debug("aborting upload id {}", ik.getUploadId()); fs.abortMultipartUpload(ik.getKey(), ik.getUploadId()); } catch (Exception e) { @@ -69,31 +74,39 @@ static void cleanupParts(S3AFileSystem fs, Set keySet) { public static IdKey createPartUpload(S3AFileSystem fs, String key, int len, int partNo) throws IOException { - WriteOperationHelper writeHelper = fs.getWriteOperationHelper(); - byte[] data = dataset(len, 'a', 'z'); - InputStream in = new ByteArrayInputStream(data); - String uploadId = writeHelper.initiateMultiPartUpload(key); - UploadPartRequest req = writeHelper.newUploadPartRequest(key, uploadId, - partNo, len, in, null, 0L); - PartETag partEtag = fs.uploadPart(req).getPartETag(); - LOG.debug("uploaded part etag {}, upid {}", partEtag.getETag(), uploadId); - return new IdKey(key, uploadId); + try (AuditSpan span = fs.createSpan("multipart", key, null)) { + WriteOperationHelper writeHelper = fs.getWriteOperationHelper(); + byte[] data = dataset(len, 'a', 'z'); + InputStream in = new ByteArrayInputStream(data); + String uploadId = writeHelper.initiateMultiPartUpload(key); + UploadPartRequest req = writeHelper.newUploadPartRequest(key, uploadId, + partNo, len, in, null, 0L); + PartETag partEtag = writeHelper.uploadPart(req).getPartETag(); + LOG.debug("uploaded part etag {}, upid {}", partEtag.getETag(), uploadId); + return new IdKey(key, uploadId); + } } /** Delete any uploads under given path (recursive). Silent on failure. */ public static void clearAnyUploads(S3AFileSystem fs, Path path) { + String key = fs.pathToKey(path); + AuditSpan span = null; try { - String key = fs.pathToKey(path); MultipartUtils.UploadIterator uploads = fs.listUploads(key); + span = fs.createSpan("multipart", path.toString(), null); + final WriteOperationHelper helper + = fs.getWriteOperationHelper(); while (uploads.hasNext()) { MultipartUpload upload = uploads.next(); - fs.getWriteOperationHelper().abortMultipartUpload(upload.getKey(), - upload.getUploadId(), true, LOG_EVENT); LOG.debug("Cleaning up upload: {} {}", upload.getKey(), truncatedUploadId(upload.getUploadId())); + helper.abortMultipartUpload(upload.getKey(), + upload.getUploadId(), true, LOG_EVENT); } } catch (IOException ioe) { LOG.info("Ignoring exception: ", ioe); + } finally { + IOUtils.closeStream(span); } } @@ -131,13 +144,15 @@ public static int countUploadsAt(S3AFileSystem fs, Path path) throws public static List listMultipartUploads(S3AFileSystem fs, String prefix) throws IOException { - return fs - .listMultipartUploads(prefix).stream() - .map(upload -> String.format("Upload to %s with ID %s; initiated %s", - upload.getKey(), - upload.getUploadId(), - S3ATestUtils.LISTING_FORMAT.format(upload.getInitiated()))) - .collect(Collectors.toList()); + try (AuditSpan span = fs.createSpan("multipart", prefix, null)) { + return fs + .listMultipartUploads(prefix).stream() + .map(upload -> String.format("Upload to %s with ID %s; initiated %s", + upload.getKey(), + upload.getUploadId(), + S3ATestUtils.LISTING_FORMAT.format(upload.getInitiated()))) + .collect(Collectors.toList()); + } } @@ -146,7 +161,7 @@ private static String truncatedUploadId(String fullId) { } /** Struct of object key, upload ID. */ - static class IdKey { + public static class IdKey { private String key; private String uploadId; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java index 599f18cd9ad62..6a920972bab78 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java @@ -67,7 +67,6 @@ import java.io.Closeable; import java.io.File; -import java.io.FileNotFoundException; import java.io.IOException; import java.net.URI; import java.net.URISyntaxException; @@ -91,7 +90,6 @@ import static org.apache.hadoop.fs.s3a.S3ATestConstants.*; import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.S3AUtils.propagateBucketOptions; -import static org.apache.hadoop.test.LambdaTestUtils.eventually; import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.junit.Assert.*; @@ -853,21 +851,6 @@ public static void callQuietly(final Logger log, } } - /** - * Call a void operation; any exception raised is logged at info. - * This is for test teardowns. - * @param log log to use. - * @param operation operation to invoke - */ - public static void callQuietly(final Logger log, - final Invoker.VoidOperation operation) { - try { - operation.execute(); - } catch (Exception e) { - log.info(e.toString(), e); - } - } - /** * Deploy a hadoop service: init and start it. * @param conf configuration to use @@ -1449,35 +1432,6 @@ public static void checkListingContainsPath(S3AFileSystem fs, Path filePath) listStatusHasIt); } - /** - * Wait for a deleted file to no longer be visible. - * @param fs filesystem - * @param testFilePath path to query - * @throws Exception failure - */ - public static void awaitDeletedFileDisappearance(final S3AFileSystem fs, - final Path testFilePath) throws Exception { - eventually( - STABILIZATION_TIME, PROBE_INTERVAL_MILLIS, - () -> intercept(FileNotFoundException.class, - () -> fs.getFileStatus(testFilePath))); - } - - /** - * Wait for a file to be visible. - * @param fs filesystem - * @param testFilePath path to query - * @return the file status. - * @throws Exception failure - */ - public static S3AFileStatus awaitFileStatus(S3AFileSystem fs, - final Path testFilePath) - throws Exception { - return (S3AFileStatus) eventually( - STABILIZATION_TIME, PROBE_INTERVAL_MILLIS, - () -> fs.getFileStatus(testFilePath)); - } - /** * This creates a set containing all current threads and some well-known * thread names whose existence should not fail test runs. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java index baa4a542c855a..21d3b57c8799b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java @@ -20,6 +20,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.s3a.audit.NoopAuditor; +import org.apache.hadoop.fs.s3a.audit.NoopSpan; import org.apache.hadoop.fs.s3a.commit.PutTracker; import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext; import org.apache.hadoop.util.Progressable; @@ -68,9 +70,14 @@ public void testFlushNoOpWhenStreamClosed() throws Exception { public void testWriteOperationHelperPartLimits() throws Throwable { S3AFileSystem s3a = mock(S3AFileSystem.class); when(s3a.getBucket()).thenReturn("bucket"); + when(s3a.getRequestFactory()) + .thenReturn(MockS3AFileSystem.REQUEST_FACTORY); + final Configuration conf = new Configuration(); WriteOperationHelper woh = new WriteOperationHelper(s3a, - new Configuration(), - new EmptyS3AStatisticsContext()); + conf, + new EmptyS3AStatisticsContext(), + NoopAuditor.newInstance(conf), + NoopSpan.INSTANCE); ByteArrayInputStream inputStream = new ByteArrayInputStream( "a".getBytes()); // first one works diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditManagerLifecycle.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditManagerLifecycle.java new file mode 100644 index 0000000000000..690b012902e73 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditManagerLifecycle.java @@ -0,0 +1,174 @@ +/* + * 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.audit; + +import java.util.List; + +import com.amazonaws.handlers.RequestHandler2; +import org.assertj.core.api.Assertions; +import org.junit.After; +import org.junit.Before; +import org.junit.Ignore; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; +import org.apache.hadoop.test.AbstractHadoopTestBase; + +import static org.apache.commons.io.IOUtils.closeQuietly; + +/** + * Unit tests related to audit manager and span lifecycle. + */ +public class TestAuditManagerLifecycle extends AbstractHadoopTestBase { + + private static final Logger LOG = + LoggerFactory.getLogger(TestAuditManagerLifecycle.class); + + private static final IOStatisticsStore STORE + = IOStatisticsBinding.emptyStatisticsStore(); + + private Configuration conf; + + private ActiveAuditManager auditManager; + + private ActiveAuditManager.WrappingAuditSpan resetSpan; + + @Before + public void setup() throws Exception { + // conf is always bonded to no-op service + conf = new Configuration(false); + conf.set(AuditConstants.AUDIT_SERVICE_CLASSNAME, + AuditConstants.NOOP_AUDIT_SERVICE); + auditManager = (ActiveAuditManager) + ActiveAuditManager.createAuditManager(conf, STORE); + resetSpan = (ActiveAuditManager.WrappingAuditSpan) + auditManager.getActiveThreadSpan(); + + } + + @After + public void teardown() throws Exception { + closeQuietly(auditManager); + } + + /** + * Core lifecycle (remember: the service has already been started). + */ + @Test + public void testStop() throws Throwable { + auditManager.stop(); + } + + @Test + public void testCreateRequestHandlers() throws Throwable { + List handlers + = auditManager.createRequestHandlers(); + Assertions.assertThat(handlers) + .isNotEmpty(); + } + + @Test + public void testInitialSpanIsInvalid() throws Throwable { + Assertions.assertThat(resetSpan) + .matches(f -> !f.isValidSpan(), "is invalid"); + } + + @Test + public void testCreateCloseSpan() throws Throwable { + AuditSpan span = auditManager.createSpan("op", null, null); + Assertions.assertThat(span) + .matches(AuditSpan::isValidSpan, "is valid"); + assertActiveSpan(span); + // activation when already active is no-op + span.activate(); + assertActiveSpan(span); + // close the span + span.close(); + // the original span is restored. + assertActiveSpan(resetSpan); + } + + @Test + @Ignore + public void testSpanActivation() throws Throwable { + // real activation switches spans in the current thead. + + AuditSpan span1 = auditManager.createSpan("op1", null, null); + AuditSpan span2 = auditManager.createSpan("op2", null, null); + assertActiveSpan(span2); + // switch back to span 1 + span1.activate(); + assertActiveSpan(span1); + // then to span 2 + span2.activate(); + assertActiveSpan(span2); + span2.close(); + // because span2 was active at time of close, + // we revert to whatever span was active when it was started. + assertActiveSpan(span1); + } + + @Test + @Ignore + public void testSpanDeactivation() throws Throwable { + AuditSpan span1 = auditManager.createSpan("op1", null, null); + AuditSpan span2 = auditManager.createSpan("op2", null, null); + assertActiveSpan(span2); + + span1.close(); + // because span2 was active at time of close, + // we revert to whatever span was active when it was started. + assertActiveSpan(span2); + // when span2 is closed, it will go back to the reset + // span because span1 is now closed + // that is closed + span2.close(); + assertActiveSpan(resetSpan); + } + + @Test + @Ignore + public void testResetSpanCannotBeClosed() throws Throwable { + + Assertions.assertThat(resetSpan) + .matches(f -> !f.isValidSpan(), "is invalid"); + // create a new span + AuditSpan span1 = auditManager.createSpan("op1", null, null); + // switch to the reset span and then close it. + resetSpan.activate(); + resetSpan.close(); + assertActiveSpan(resetSpan); + span1.close(); + } + + /** + * Ass that the supplied parameter is the active span + * for this thread. + * @param span span expected to be active + */ + private void assertActiveSpan(AuditSpan span) { + Assertions.assertThat(auditManager.getActiveThreadSpan()) + .isSameAs(span); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditOperations.java new file mode 100644 index 0000000000000..c19a6d4110ac4 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditOperations.java @@ -0,0 +1,43 @@ +/* + * 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.audit; + +import java.nio.file.AccessDeniedException; + +import org.junit.Test; + +import org.apache.hadoop.test.AbstractHadoopTestBase; + +import static org.apache.hadoop.fs.s3a.S3AUtils.translateException; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; + +/** + * Unit tests for auditing. + */ +public class TestAuditOperations extends AbstractHadoopTestBase { + + @Test + public void testExceptionTranslation() throws Throwable { + intercept(AccessDeniedException.class, + () -> { + throw translateException("test", "/", + new AuditFailureException("should be translated")); + }); + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java index 814292c45d83b..771578001235e 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java @@ -255,8 +255,7 @@ public void testAssumeRoleBadInnerAuth() throws Exception { conf.set(SECRET_KEY, "not secret"); expectFileSystemCreateFailure(conf, AWSBadRequestException.class, - "not a valid " + - "key=value pair (missing equal-sign) in Authorization header"); + "IncompleteSignature"); } @Test diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java index 1df4f9b9cc15c..de544e986ca71 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java @@ -40,6 +40,7 @@ import org.apache.hadoop.fs.s3a.InconsistentAmazonS3Client; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.WriteOperationHelper; +import org.apache.hadoop.fs.s3a.audit.AuditSpan; import org.apache.hadoop.fs.s3a.commit.files.SuccessData; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.RecordWriter; @@ -289,10 +290,13 @@ protected int abortMultipartUploadsUnderPath(Path path) throws IOException { S3AFileSystem fs = getFileSystem(); if (fs != null && path != null) { String key = fs.pathToKey(path); - WriteOperationHelper writeOps = fs.getWriteOperationHelper(); - int count = writeOps.abortMultipartUploadsUnderPath(key); - if (count > 0) { - log().info("Multipart uploads deleted: {}", count); + int count = 0; + try (AuditSpan span = span()) { + WriteOperationHelper writeOps = fs.getWriteOperationHelper(); + count = writeOps.abortMultipartUploadsUnderPath(key); + if (count > 0) { + log().info("Multipart uploads deleted: {}", count); + } } return count; } else { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java index 14207e8359788..0e7fc7db64074 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java @@ -41,6 +41,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.fs.s3a.audit.AuditSpan; import org.apache.hadoop.fs.s3a.commit.files.SuccessData; import org.apache.hadoop.fs.s3a.commit.magic.MagicS3GuardCommitter; import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; @@ -177,7 +178,7 @@ public void teardown() throws Exception { describe("teardown"); abortInTeardown.forEach(this::abortJobQuietly); if (outDir != null) { - try { + try (AuditSpan span = span()) { abortMultipartUploadsUnderPath(outDir); cleanupDestDir(); } catch (IOException e) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperations.java index 4d7f81d019b74..199ca1865cdb0 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperations.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperations.java @@ -195,7 +195,8 @@ private void fullThrottle() { setThrottling(FULL_THROTTLE, STANDARD_FAILURE_LIMIT); } - private CommitOperations newCommitOperations() { + private CommitOperations newCommitOperations() + throws IOException { return new CommitOperations(getFileSystem()); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestXAttrCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestXAttrCost.java index aa3cecaf1eb7a..415dcba0f575f 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestXAttrCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestXAttrCost.java @@ -38,6 +38,7 @@ import static org.apache.hadoop.fs.s3a.Statistic.INVOCATION_XATTR_GET_NAMED; import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.CONTENT_TYPE_OCTET_STREAM; import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.CONTENT_TYPE_APPLICATION_XML; +import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.CONTENT_TYPE_X_DIRECTORY; import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.XA_CONTENT_LENGTH; import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.XA_CONTENT_TYPE; import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.XA_STANDARD_HEADERS; @@ -163,7 +164,7 @@ public void testXAttrDir() throws Throwable { assertHeader(XA_CONTENT_LENGTH, bytes) .isEqualTo("0"); assertHeaderEntry(xAttrs, XA_CONTENT_TYPE) - .isEqualTo(CONTENT_TYPE_OCTET_STREAM); + .isEqualTo(CONTENT_TYPE_X_DIRECTORY); } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestHeaderProcessing.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestHeaderProcessing.java index e0c6feeb256cc..5fcd25eced769 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestHeaderProcessing.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestHeaderProcessing.java @@ -33,7 +33,11 @@ import org.junit.Test; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.MockS3AFileSystem; import org.apache.hadoop.fs.s3a.S3ATestUtils; +import org.apache.hadoop.fs.s3a.api.RequestFactory; +import org.apache.hadoop.fs.s3a.audit.AuditSpan; +import org.apache.hadoop.fs.s3a.audit.NoopSpan; import org.apache.hadoop.fs.s3a.test.OperationTrackingStore; import org.apache.hadoop.test.HadoopTestBase; @@ -291,6 +295,16 @@ public Path makeQualified(final Path path) { return path; } + @Override + public AuditSpan getActiveAuditSpan() { + return NoopSpan.INSTANCE; + } + + @Override + public RequestFactory getRequestFactory() { + return MockS3AFileSystem.REQUEST_FACTORY; + } + @Override public ObjectMetadata getObjectMetadata(final String key) throws IOException { @@ -303,6 +317,7 @@ public ObjectMetadata getObjectMetadata(final String key) } else { throw new FileNotFoundException(key); } + } public void setHeader(String key, String val) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java index 42714cb1555e4..5cc94fae7d159 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java @@ -37,7 +37,11 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.commons.lang3.tuple.Triple; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.MockS3AFileSystem; import org.apache.hadoop.fs.s3a.S3ATestUtils; +import org.apache.hadoop.fs.s3a.api.RequestFactory; +import org.apache.hadoop.fs.s3a.audit.AuditSpan; +import org.apache.hadoop.fs.s3a.audit.NoopSpan; import org.apache.hadoop.fs.s3a.test.OperationTrackingStore; import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.ACCESS_DENIED; @@ -261,6 +265,17 @@ public ObjectMetadata getObjectMetadata(final String key) throws IOException { return new ObjectMetadata(); } + + @Override + public AuditSpan getActiveAuditSpan() { + return NoopSpan.INSTANCE; + } + + @Override + public RequestFactory getRequestFactory() { + return MockS3AFileSystem.REQUEST_FACTORY; + } + } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java index c4f8db71937d0..25b0ac6421489 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java @@ -37,6 +37,7 @@ import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.Statistic; import org.apache.hadoop.fs.s3a.Tristate; +import org.apache.hadoop.fs.s3a.audit.AuditSpan; import org.apache.hadoop.fs.s3a.impl.DirectoryPolicy; import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum; import org.apache.hadoop.fs.s3a.statistics.StatisticTypeEnum; @@ -99,6 +100,8 @@ public class AbstractS3ACostTest extends AbstractS3ATestBase { */ private Statistic deleteMarkerStatistic; + private AuditSpan auditSpan; + public AbstractS3ACostTest( final boolean s3guard, final boolean keepMarkers, @@ -108,6 +111,18 @@ public AbstractS3ACostTest( this.authoritative = authoritative; } + /** + * Constructor for tests which don't include + * any for S3Guard. + * @param keepMarkers should markers be tested. + */ + public AbstractS3ACostTest( + final boolean keepMarkers) { + this.s3guard = false; + this.keepMarkers = keepMarkers; + this.authoritative = false; + } + @Override public Configuration createConfiguration() { Configuration conf = super.createConfiguration(); @@ -183,6 +198,29 @@ public void setup() throws Exception { deleteMarkerStatistic = isBulkDelete() ? OBJECT_BULK_DELETE_REQUEST : OBJECT_DELETE_REQUEST; + + setSpanSource(fs); + // there's lots of internal calls going on; to avoid problems + // initiate a span for each test case. + if (shouldCreateSpanInSetup()) { + auditSpan = span(); + } + } + + @Override + public void teardown() throws Exception { + if (auditSpan != null) { + auditSpan.close(); + } + super.teardown(); + } + + /** + * Should this test suite create a span during setup? + * @return true for a span to be created in setup and closed in teardown? + */ + protected boolean shouldCreateSpanInSetup() { + return false; } public void assumeUnguarded() { @@ -357,6 +395,7 @@ private void resetStatistics() { protected T verifyMetrics( Callable eval, OperationCostValidator.ExpectedProbe... expected) throws Exception { + span(); return costValidator.exec(eval, expected); } @@ -379,6 +418,7 @@ protected E verifyMetricsIntercepting( String text, Callable eval, OperationCostValidator.ExpectedProbe... expected) throws Exception { + span(); return costValidator.intercepting(clazz, text, eval, expected); } @@ -476,6 +516,8 @@ protected OperationCostValidator.ExpectedProbe whenDeleting( /** * Execute a closure expecting a specific number of HEAD/LIST calls * on raw S3 stores only. The operation is always evaluated. + * A span is always created prior to the invocation; saves trouble + * in tests that way. * @param cost expected cost * @param eval closure to evaluate * @param return type of closure @@ -525,12 +567,14 @@ public void interceptRawGetFileStatusFNFE( boolean needEmptyDirectoryFlag, Set probes, OperationCost cost) throws Exception { - interceptRaw(FileNotFoundException.class, "", - cost, () -> - innerGetFileStatus(getFileSystem(), - path, - needEmptyDirectoryFlag, - probes)); + try (AuditSpan span = span()) { + interceptRaw(FileNotFoundException.class, "", + cost, () -> + innerGetFileStatus(getFileSystem(), + path, + needEmptyDirectoryFlag, + probes)); + } } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java index ed56802ddfec1..0444869b25df1 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java @@ -46,6 +46,7 @@ import org.apache.hadoop.fs.s3a.AbstractS3ATestBase; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3AUtils; +import org.apache.hadoop.fs.s3a.audit.AuditSpan; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; import static org.apache.hadoop.fs.s3a.Constants.AUTHORITATIVE_PATH; @@ -58,6 +59,7 @@ import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.apache.hadoop.util.functional.RemoteIterators.foreach; /** * This is a test suite designed to verify that directory markers do @@ -223,7 +225,6 @@ public void setup() throws Exception { assume("unguarded FS only", !fs.hasMetadataStore()); s3client = fs.getAmazonS3ClientForTesting("markers"); - bucket = fs.getBucket(); Path base = new Path(methodPath(), "base"); @@ -653,7 +654,9 @@ private void head404(final String key) throws Exception { } /** - * Execute an operation; transate AWS exceptions. + * Execute an operation; translate AWS exceptions. + * Wraps the operation in an audit span, so that low-level + * calls can be safely made. * @param op operation * @param call call to make * @param returned type @@ -662,7 +665,7 @@ private void head404(final String key) throws Exception { */ private T exec(String op, Callable call) throws Exception { ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer(); - try { + try (AuditSpan span = getSpanSource().createSpan(op, null, null)) { return call.call(); } catch (AmazonClientException ex) { throw S3AUtils.translateException(op, "", ex); @@ -749,9 +752,7 @@ private List toList( RemoteIterator status) throws IOException { List l = new ArrayList<>(); - while (status.hasNext()) { - l.add(status.next()); - } + foreach(status, st -> l.add(st)); return dump(l); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AMkdirCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AMkdirCost.java new file mode 100644 index 0000000000000..04fff91c56a44 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AMkdirCost.java @@ -0,0 +1,190 @@ +/* + * 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.performance; + +import java.util.Arrays; +import java.util.Collection; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.FileAlreadyExistsException; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.S3AFileSystem; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; +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.performance.OperationCost.FILESTATUS_DIR_PROBE_L; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.FILESTATUS_FILE_PROBE_H; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.FILESTATUS_FILE_PROBE_L; + +/** + * Use metrics to assert about the cost of mkdirs. + * Parameterized on guarded vs raw. and directory marker keep vs delete + */ +@RunWith(Parameterized.class) +public class ITestS3AMkdirCost extends AbstractS3ACostTest { + + private static final Logger LOG = + LoggerFactory.getLogger(ITestS3AMkdirCost.class); + + /** + * Parameterization. + */ + @Parameterized.Parameters(name = "{0}") + public static Collection params() { + return Arrays.asList(new Object[][]{ + {"keep-markers", true}, + {"delete-markers", false} + }); + } + + public ITestS3AMkdirCost(final String name, + final boolean keepMarkers) { + super(false, true, false); + } + + protected boolean shouldCreateSpanInSetup() { + return false; + } + + /** + * Common operation which should be low cost as possible. + */ + @Test + public void testMkdirOverDir() throws Throwable { + describe("create a dir over a dir"); + S3AFileSystem fs = getFileSystem(); + // create base dir with marker + Path baseDir = dir(methodPath()); + + // create the child; only assert on HEAD/GET IO + verifyMetrics(() -> fs.mkdirs(baseDir), + // full probe on dest plus list only on parent. + with(OBJECT_METADATA_REQUESTS, + 0), + + with(OBJECT_LIST_REQUEST, FILESTATUS_DIR_PROBE_L)); + } + + /** + * Mkdir with a parent dir will check dest (list+HEAD) + * then do a list on the parent to find the marker. + * Once the dir is created, creating a sibling will + * have the same cost. + */ + @Test + public void testMkdirWithParent() throws Throwable { + describe("create a dir under a dir with a parent"); + S3AFileSystem fs = getFileSystem(); + // create base dir with marker + Path baseDir = dir(methodPath()); + Path childDir = new Path(baseDir, "child"); + + // create the child; only assert on HEAD/GET IO + verifyMetrics(() -> fs.mkdirs(childDir), + // full probe on dest plus list only on parent. + with(OBJECT_METADATA_REQUESTS, + FILESTATUS_FILE_PROBE_H), + + with(OBJECT_LIST_REQUEST, + FILESTATUS_FILE_PROBE_L + 2 * FILESTATUS_DIR_PROBE_L)); + + // now include a sibling; cost will be the same. + Path sibling = new Path(baseDir, "sibling"); + verifyMetrics(() -> fs.mkdirs(sibling), + // full probe on dest plus list only on parent. + with(OBJECT_METADATA_REQUESTS, + FILESTATUS_FILE_PROBE_H), + + with(OBJECT_LIST_REQUEST, + FILESTATUS_FILE_PROBE_L + 2 * FILESTATUS_DIR_PROBE_L)); + } + + /** + * Mkdir with a grandparent dir will check dest (list+HEAD) + * then do a list + HEAD on the parent and ultimately find the + * marker with a list of the parent. + * That's three list calls and two head requsts. + * Once the dir is created, creating a sibling will + * cost less as the list of the parent path will find + * a directory. + */ + @Test + public void testMkdirWithGrandparent() throws Throwable { + describe("create a dir under a dir with a parent"); + S3AFileSystem fs = getFileSystem(); + // create base dir with marker + Path baseDir = dir(methodPath()); + Path subDir = new Path(baseDir, "child/grandchild"); + + // create the child; only assert on HEAD/GET IO + verifyMetrics(() -> fs.mkdirs(subDir), + // full probe on dest plus list only on parent. + with(OBJECT_METADATA_REQUESTS, + 2 * FILESTATUS_FILE_PROBE_H), + + with(OBJECT_LIST_REQUEST, + 3 * FILESTATUS_DIR_PROBE_L)); + + + // now include a sibling; cost will be less because + // now the immediate parent check will succeed on the list call. + Path sibling = new Path(baseDir, "child/sibling"); + + verifyMetrics(() -> fs.mkdirs(sibling), + + // full probe on dest plus list only on parent. + with(OBJECT_METADATA_REQUESTS, + FILESTATUS_FILE_PROBE_H), + + with(OBJECT_LIST_REQUEST, + FILESTATUS_FILE_PROBE_L + 2 * FILESTATUS_DIR_PROBE_L)); + } + + + /** + * When calling mkdir over a file, the list happens first, so + * is always billed for. + * @throws Throwable failure. + */ + @Test + public void testMkdirOverFile() throws Throwable { + describe("create a dir over a file; expect dir and file probes"); + S3AFileSystem fs = getFileSystem(); + // create base dir with marker + Path baseDir = dir(methodPath()); + Path childDir = new Path(baseDir, "child"); + touch(fs, childDir); + + // create the child; only assert on HEAD/GET IO + verifyMetricsIntercepting( + FileAlreadyExistsException.class, "", + () -> fs.mkdirs(childDir), + // full probe on dest plus list only on parent. + with(OBJECT_METADATA_REQUESTS, + FILESTATUS_FILE_PROBE_H), + with(OBJECT_LIST_REQUEST, FILESTATUS_DIR_PROBE_L)); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java index c4e903df7390f..28eb52ff41ada 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java @@ -44,7 +44,6 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; 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.awaitFileStatus; import static org.apache.hadoop.fs.s3a.S3ATestUtils.metadataStorePersistsAuthoritativeBit; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; @@ -207,7 +206,6 @@ public void testIDetectDirInS3FileInMs() throws Exception { try { // create a file with guarded fs mkdirs(cwd); - awaitFileStatus(guardedFs, cwd); // modify the cwd metadata and set that it's not a directory final S3AFileStatus newParentFile = MetadataStoreTestBase .basicFileStatus(cwd, 1, false, 1); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java index 798bdd693081f..72af1bef14935 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java @@ -27,6 +27,7 @@ import java.util.Arrays; import java.util.List; +import org.apache.hadoop.fs.s3a.MultipartTestUtils; import org.apache.hadoop.test.LambdaTestUtils; import org.apache.hadoop.util.StringUtils; @@ -272,35 +273,28 @@ public void testUploads() throws Throwable { Path path = path(UPLOAD_PREFIX + "/" + UPLOAD_NAME); describe("Cleaning up any leftover uploads from previous runs."); - // 1. Make sure key doesn't already exist - clearAnyUploads(fs, path); + final String key = fs.pathToKey(path); + try { + // 1. Make sure key doesn't already exist + clearAnyUploads(fs, path); - // 2. Confirm no uploads are listed via API - assertNoUploadsAt(fs, path.getParent()); + // 2. Confirm no uploads are listed via API + assertNoUploadsAt(fs, path.getParent()); - // 3. Confirm no uploads are listed via CLI - describe("Confirming CLI lists nothing."); - assertNumUploads(path, 0); + // 3. Confirm no uploads are listed via CLI + describe("Confirming CLI lists nothing."); + assertNumUploads(path, 0); - // 4. Create a upload part - describe("Uploading single part."); - createPartUpload(fs, fs.pathToKey(path), 128, 1); + // 4. Create a upload part + describe("Uploading single part."); + createPartUpload(fs, key, 128, 1); - try { - // 5. Confirm it exists via API.. - LambdaTestUtils.eventually(5000, /* 5 seconds until failure */ - 1000, /* one second retry interval */ - () -> { - assertEquals("Should be one upload", 1, countUploadsAt(fs, path)); - }); + assertEquals("Should be one upload", 1, countUploadsAt(fs, path)); // 6. Confirm part exists via CLI, direct path and parent path describe("Confirming CLI lists one part"); - LambdaTestUtils.eventually(5000, 1000, - () -> { assertNumUploads(path, 1); }); - LambdaTestUtils.eventually(5000, 1000, - () -> { assertNumUploads(path.getParent(), 1); }); - + assertNumUploads(path, 1); + assertNumUploads(path.getParent(), 1); // 7. Use CLI to delete part, assert it worked describe("Deleting part via CLI"); assertNumDeleted(fs, path, 1); @@ -331,15 +325,15 @@ public void testUploadListByAge() throws Throwable { // 2. Create a upload part describe("Uploading single part."); - createPartUpload(fs, fs.pathToKey(path), 128, 1); + final String key = fs.pathToKey(path); + createPartUpload(fs, key, 128, 1); + //try (AuditSpan span = fs.startOperation("multipart", key, null)) { try { + // 3. Confirm it exists via API.. may want to wrap with // LambdaTestUtils.eventually() ? - LambdaTestUtils.eventually(5000, 1000, - () -> { - assertEquals("Should be one upload", 1, countUploadsAt(fs, path)); - }); + assertEquals("Should be one upload", 1, countUploadsAt(fs, path)); // 4. Confirm part does appear in listing with long age filter describe("Confirming CLI older age doesn't list"); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java index 2b3043f39ac73..3e00917a2dbe5 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java @@ -128,8 +128,7 @@ protected Configuration createScaleConfiguration() { @Test public void test_010_CreateHugeFile() throws IOException { - assertFalse("Please run this test sequentially to avoid timeouts" + - " and bandwidth problems", isParallelExecution()); + long filesizeMB = filesize / _1MB; // clean up from any previous attempts diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java index 61f6ef3c76473..d73ce6e6c1965 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java @@ -28,7 +28,11 @@ import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.Statistic; +import org.apache.hadoop.fs.s3a.WriteOperationHelper; +import org.apache.hadoop.fs.s3a.api.RequestFactory; +import org.apache.hadoop.fs.s3a.audit.AuditSpan; import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.util.functional.RemoteIterators; import org.junit.Test; import org.assertj.core.api.Assertions; @@ -39,7 +43,7 @@ import java.io.InputStream; import java.util.ArrayList; import java.util.List; -import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -47,13 +51,17 @@ import com.amazonaws.services.s3.model.PutObjectRequest; import com.amazonaws.services.s3.model.PutObjectResult; +import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY; +import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_KEEP; import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL; import static org.apache.hadoop.fs.s3a.Statistic.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; import static org.apache.hadoop.fs.contract.ContractTestUtils.*; +import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit; +import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletion; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.lookupCounterStatistic; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue; -import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToString; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString; import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics; import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_CONTINUE_LIST_REQUEST; import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_LIST_REQUEST; @@ -164,6 +172,13 @@ public void testListOperations() throws Throwable { } } + /** + * This is quite a big test; it PUTs up a number of + * files and then lists them in a filesystem set to ask for a small number + * of files on each listing. + * The standard listing API calls are all used, and then + * delete() is invoked to verify that paged deletion works correctly too. + */ @Test public void testMultiPagesListingPerformanceAndCorrectness() throws Throwable { @@ -180,7 +195,15 @@ public void testMultiPagesListingPerformanceAndCorrectness() .isEqualTo(0); final Configuration conf = getConfigurationWithConfiguredBatchSize(batchSize); - removeBaseAndBucketOverrides(conf, S3_METADATA_STORE_IMPL); + + removeBaseAndBucketOverrides(conf, + S3_METADATA_STORE_IMPL, + DIRECTORY_MARKER_POLICY); + // force directory markers = keep to save delete requests on every + // file created. + conf.set(DIRECTORY_MARKER_POLICY, DIRECTORY_MARKER_POLICY_KEEP); + S3AFileSystem fs = (S3AFileSystem) FileSystem.get(dir.toUri(), conf); + final InputStream im = new InputStream() { @Override public int read() throws IOException { @@ -188,37 +211,48 @@ public int read() throws IOException { } }; final List originalListOfFiles = new ArrayList<>(); - List> putObjectRequests = new ArrayList<>(); ExecutorService executorService = Executors .newFixedThreadPool(numOfPutThreads); NanoTimer uploadTimer = new NanoTimer(); - S3AFileSystem fs = (S3AFileSystem) FileSystem.get(dir.toUri(), conf); try { assume("Test is only for raw fs", !fs.hasMetadataStore()); fs.create(dir); + + // create a span for the write operations + final AuditSpan span = fs.getAuditSpanSource() + .createSpan(OBJECT_PUT_REQUESTS.getSymbol(), dir.toString(), null); + final WriteOperationHelper writeOperationHelper + = fs.getWriteOperationHelper(); + final RequestFactory requestFactory + = writeOperationHelper.getRequestFactory(); + List> futures = + new ArrayList<>(numOfPutRequests); + for (int i=0; i - fs.getWriteOperationHelper().putObject(put)); + PutObjectRequest put = requestFactory + .newPutObjectRequest(fs.pathToKey(file), om, im); + futures.add(submit(executorService, () -> + writeOperationHelper.putObject(put))); } - executorService.invokeAll(putObjectRequests); + LOG.info("Waiting for PUTs to complete"); + waitForCompletion(futures); uploadTimer.end("uploading %d files with a parallelism of %d", numOfPutRequests, numOfPutThreads); RemoteIterator resIterator = fs.listFiles(dir, true); List listUsingListFiles = new ArrayList<>(); NanoTimer timeUsingListFiles = new NanoTimer(); - while(resIterator.hasNext()) { - listUsingListFiles.add(resIterator.next().getPath().toString()); - Thread.sleep(eachFileProcessingTime); - } + RemoteIterators.foreach(resIterator, st -> { + listUsingListFiles.add(st.getPath().toString()); + sleep(eachFileProcessingTime); + }); + LOG.info("Listing Statistics: {}", ioStatisticsToPrettyString( + retrieveIOStatistics(resIterator))); + timeUsingListFiles.end("listing %d files using listFiles() api with " + "batch size of %d including %dms of processing time" + " for each file", @@ -226,7 +260,7 @@ public int read() throws IOException { Assertions.assertThat(listUsingListFiles) .describedAs("Listing results using listFiles() must" + - "match with original list of files") + " match with original list of files") .hasSameElementsAs(originalListOfFiles) .hasSize(numOfPutRequests); List listUsingListStatus = new ArrayList<>(); @@ -234,7 +268,7 @@ public int read() throws IOException { FileStatus[] fileStatuses = fs.listStatus(dir); for(FileStatus fileStatus : fileStatuses) { listUsingListStatus.add(fileStatus.getPath().toString()); - Thread.sleep(eachFileProcessingTime); + sleep(eachFileProcessingTime); } timeUsingListStatus.end("listing %d files using listStatus() api with " + "batch size of %d including %dms of processing time" + @@ -247,12 +281,12 @@ public int read() throws IOException { .hasSize(numOfPutRequests); // Validate listing using listStatusIterator(). NanoTimer timeUsingListStatusItr = new NanoTimer(); - RemoteIterator lsItr = fs.listStatusIterator(dir); List listUsingListStatusItr = new ArrayList<>(); - while (lsItr.hasNext()) { - listUsingListStatusItr.add(lsItr.next().getPath().toString()); - Thread.sleep(eachFileProcessingTime); - } + RemoteIterator lsItr = fs.listStatusIterator(dir); + RemoteIterators.foreach(lsItr, st -> { + listUsingListStatusItr.add(st.getPath().toString()); + sleep(eachFileProcessingTime); + }); timeUsingListStatusItr.end("listing %d files using " + "listStatusIterator() api with batch size of %d " + "including %dms of processing time for each file", @@ -265,24 +299,58 @@ public int read() throws IOException { // now validate the statistics returned by the listing // to be non-null and containing list and continue counters. IOStatistics lsStats = retrieveIOStatistics(lsItr); - String statsReport = ioStatisticsToString(lsStats); + String statsReport = ioStatisticsToPrettyString(lsStats); LOG.info("Listing Statistics: {}", statsReport); verifyStatisticCounterValue(lsStats, OBJECT_LIST_REQUEST, 1); long continuations = lookupCounterStatistic(lsStats, OBJECT_CONTINUE_LIST_REQUEST); // calculate expected #of continuations - int expectedContinuations = numOfPutRequests / batchSize -1; + int expectedContinuations = numOfPutRequests / batchSize - 1; Assertions.assertThat(continuations) .describedAs("%s in %s", OBJECT_CONTINUE_LIST_REQUEST, statsReport) .isEqualTo(expectedContinuations); + + List listUsingListLocatedStatus = new ArrayList<>(); + + RemoteIterator it = fs.listLocatedStatus(dir); + RemoteIterators.foreach(it, st -> { + listUsingListLocatedStatus.add(st.getPath().toString()); + sleep(eachFileProcessingTime); + }); + final IOStatistics llsStats = retrieveIOStatistics(it); + LOG.info("Listing Statistics: {}", ioStatisticsToPrettyString( + llsStats)); + verifyStatisticCounterValue(llsStats, OBJECT_CONTINUE_LIST_REQUEST, + expectedContinuations); + Assertions.assertThat(listUsingListLocatedStatus) + .describedAs("Listing results using listLocatedStatus() must" + + "match with original list of files") + .hasSameElementsAs(originalListOfFiles); + // delete in this FS so S3Guard is left out of it. + // and so that the incremental listing is tested through + // the delete operation. + fs.delete(dir, true); } finally { executorService.shutdown(); - // delete in this FS so S3Guard is left out of it. + // in case the previous delete was not reached. fs.delete(dir, true); + LOG.info("FS statistics {}", + ioStatisticsToPrettyString(fs.getIOStatistics())); fs.close(); } } + /** + * Sleep briefly + * @param eachFileProcessingTime time to sleep. + */ + private void sleep(final int eachFileProcessingTime) { + try { + Thread.sleep(eachFileProcessingTime); + } catch (InterruptedException ignored) { + } + } + private Configuration getConfigurationWithConfiguredBatchSize(int batchSize) { Configuration conf = new Configuration(getFileSystem().getConf()); S3ATestUtils.disableFilesystemCaching(conf); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/MinimalListingOperationCallbacks.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/MinimalListingOperationCallbacks.java index f683a0a008caf..04e19bf3ceb58 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/MinimalListingOperationCallbacks.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/MinimalListingOperationCallbacks.java @@ -26,6 +26,7 @@ import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus; import org.apache.hadoop.fs.s3a.S3ListRequest; import org.apache.hadoop.fs.s3a.S3ListResult; +import org.apache.hadoop.fs.s3a.audit.AuditSpan; import org.apache.hadoop.fs.s3a.impl.ListingOperationCallbacks; import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider; import org.apache.hadoop.fs.statistics.DurationTrackerFactory; @@ -39,7 +40,8 @@ public class MinimalListingOperationCallbacks @Override public CompletableFuture listObjectsAsync( final S3ListRequest request, - final DurationTrackerFactory trackerFactory) throws IOException { + final DurationTrackerFactory trackerFactory, AuditSpan span) + throws IOException { return null; } @@ -47,7 +49,8 @@ public CompletableFuture listObjectsAsync( public CompletableFuture continueListObjectsAsync( final S3ListRequest request, final S3ListResult prevResult, - final DurationTrackerFactory trackerFactory) throws IOException { + final DurationTrackerFactory trackerFactory, AuditSpan span) + throws IOException { return null; } diff --git a/hadoop-tools/hadoop-aws/src/test/resources/core-site.xml b/hadoop-tools/hadoop-aws/src/test/resources/core-site.xml index a90edbe24fc4a..7cc3a0e7e69d0 100644 --- a/hadoop-tools/hadoop-aws/src/test/resources/core-site.xml +++ b/hadoop-tools/hadoop-aws/src/test/resources/core-site.xml @@ -166,6 +166,13 @@ 0 + + + fs.s3a.audit.reject.out.of.span.operations + true + + hk9LQ+P`}9$*cv^osB! z0%Lyd`%~L7J7^xBwSR#!-u33HiyQLHfXI}DWaMscVvJ&Om!N}z2VjP<0RS_cmz&fO zZgX35W*BaXEWO+}6K(;sgb_6Sxun**`i+r;ZM5Vfn9fH};DNRT*&CaQd9-kH8wqN< z^g903g#io6znwclykL{&x`E2%Zywoe2z%gGBv-WXrsWky_#+EELesd#iG*vovIkb& zkIr$K>qM2!Iu3qXDQ0Ie5BzA3{fSkm;jf1RZ${;j&??7cWJOGRPx5SaBd)YOA`i{nQ4W=;)5MxcC>)X?8NVfi@YS>)lC#f_QrA! zRqN;DdbjrA{-YI00K7hM)VEyG5PeA04eH2?oD!RCkML4&bRW@COWs;+7b#5;v-k?Y z2tE_bmCmA0X_I`jbRV`0EBc(kciR-D?V%xGnw$qpk!cqJLVphEaOf43CyFnh7w=|Q z9`J(CNlcGvSw>cdyUBUbii^eNH*c}NGD;+BCf=jnfKcyr=q_ z!g!)Retf$g@Ms88J){&I8CEFm0{B**jpsn#<-;wlD3Z9YKt^(rj?2lwOb*Mh0Q6u z>x{oLexEF~yqUqNbF(I3=`CqjVJ3hI9rHx1O*K_Ns%RNZWQ?>$6}~M)K!>0r$ZUsPfy-1aXBtKdjBSAT0MXYRlK^QgS3lY z?8`>lbluTwMwrHQh>VY5!RfYslxA&Z#bwHGglYyDy>6~iIRrmk7!$BH6pU0Dk~1nl|L zq2pvCkHO|VQzD1n-N=~Pb~^i2j$)I^?S&pSVxsaycUVM!X1JM2@$8Q;@&Ko|`S!sP zz@MydSM#}NC|1mGy2wK1QWZ9tFWLZ;s%bf2Q~&(-5x*8ovCD$A6F&5Ibq>VcWX)}iQoy3(cWsK55Z^c9i{T45QC}sdw z-z8U3v-)uCb997I0U@9NWsR|TGN+l>{qe&d>I&u{I@!V`Izh5iOl=thxZ2EZ2jln4 z0#~2i^kO;V-dY2tv2T7ZB$MfEE7Ldr`BgO+E^vQd3!_mp&)Gkv_cFr7VTU#T2YhAV z!St(JFZHRe`hmEF<5kPC-L6Wt%2vm=_h_atpLI7{AKXccvBEYSIK~+qU|*zMl=)56 zix1e7N4k@nQ=XZUGbv8Jn z9UU{C=>s!r-Q=9}AK(4VHN#Gfsr&kchTQodVZ;M=4VEh4!~D;OrkQa6v9%cx9CNgd z&olpf4E%^)hF(dSjT8hwR&V?(I3`Q-*zb`e;xWl@bIBy10HlHx0Y4GJuM!DMn_*^c z;$`2wkC9Pa1Ly!L)cvj&+<^&)54*WB&K+g{ejA`62OyNBQX*|MJnk}^(jf{Gl5p_u z?Zw_q4)u43uigRJ!vTC}zmcIJcKfvVC4i9}z)JaD$+R<|A+^66YVfy5VotR9M(1V3W<^GB6+D?fW0yZvWROgEBgbsJ7;eP#lX&H-peLU2A8 z{SNRvu?C&|Zy=C=!ZN=@MI=b6;M5Zl0H6F!89y=jOIEH9=-~rA=U}w|hVJ~{%)_kX z-c(>3+FX8NM?biJ2VA}zl7Ob4AP@k&4qiQ&F|t>be@fZ~w+hF-BVsGrim+CYWp zeLlR>#LVx)S?9pNg^U5&a$cCN?#9CDb*|;0miE}50t@2+)f(8<&bGFbiJ_i&_lo;{ z;c|>|Xww`JbpuyLsB~%RbM-Nqa-t$a*df8U*)uELNoISNuJgSYrM+4rw&lZ9sdkJe zASV3CI_aX5j%TV#blLG*Rc<~|==&|X&D?w}ydrE+mW`b|c?RZ3N4ZyPZZ-`kf0d_K zvphK9DFtu;hMpwP80$f7hC04IBI$Br*-?mJyz5L0NFUo*bI3Xj37R^>otl5*O?6g^ z6udr|zOk1I%DxyU2iV5*uc>ZOVPBD@dtPi zR%-+M!E6zmm$)k8G-iNO8Kx6L3>4cwe|p5%Cf}c~UoFN#2ZMHVX?Op34h5H?Ci*UC zx%q-Jh}1#0tE^-u+Fq|Fby(j@Z5||^fs2=iDv5RfLpHqxzvF^W3S8+r4jZr?MKshN zbIS5;_9o>seTbeoO-pE^C#LXWq`FEU*hPEg9QKne+sQ@tThA#*bTd5F#IE7Zd`$u% z3XF;}CW6?CrMPBaS-j{pGQ-hs^ae=&N#-qRpIBNo1(cZY+OwjFG4e~-p(?R#Pd}j} zYeuK3N@AmyS$8Mu_FBU7$sw-m>9$^@J|QD2qs;xv-L6De67MC~K-vtIYr;ao{Fr#- z21`%bqsq5$H&7Bt34a@xp*`E7<**o~U%Q1rg_d0P1b*YB4F}VK84smWWC>gxPo*o@G3Y~F z+t5_E8P5BrqiZZTRU{V)?~)9NUmD|K81tWseU76}hr@02qxOi`28h8+QO&%-x&9i` zzmibCJu<3p&)e2HTryL&Y0st_Qdw*x0c#boKAQ!&0CQW0T0Tv@?#e3#02x?Sr}eJ* z?&owO0+(x6H9W7$`)Kw3<`mR{PBIdLRG3~Q81 z)Jg!sNN-V76@nPUdjnnC{*F5T7k|X(J7A#j7|U?uE|mmc@CS(8DIKP9U-hf7dQ}Vr zV^&$a)v2mw7G)vj7%^e)P0&QPryL3C?&kO!{`MOqYHoA|0_Z1y>I}Ym&CN}D*NN$T zs;@y7=IsEe$EFgDpqC_m3~o;VlZ8 z4GRy$k|=#?L!aqn6HUNnDmSk?$F!^%E#^6lT*75hDTGj0JCsF3h=sxtQ#pDbds;3! zF=~b6?>67`OIvN60Z?^6d5Gz+xwd;)y3fJ1dH6xn(Mi5tV|B+1tSqg{W{ZihkIc+2 zhJ^dNuYQz?&Ekkt=6{SN2AvtJST02D#UyVAeCi1cqydu|{>isfsubaQDz-cCs-I{h zL1I?^b=ICA z-*(fAPndI!(chZutc#9X;c&$rzoNVuq>(LW#KYLI2Wn!?pCd^Yi}E`3r7H1T>l&}Ms%_x> zOoWTz)yO~tk#hO>K>7H*<^;TgTL}&!9ua4aG3Kn(2wl!4kEisrzTc%DA-_^2z!yw;m)|e@&4{}!{pH)%%@%khsbj3$`I>a@ni9|o) z(B-gPz4>~?b$)}`PFu1VHoAUw0s>@8=Lf#;tJH5alk?5yoiK5oy;C>-WbN!Nzw^vB zrRP4$4SNHT$AXxeRwQp6!{()ewD)>m`8k_-3F$w>$z!CQBI~oMg>V`mO*3?-B|}9| z3>#50U8nh=M}?#YqmTHxuexsP>fcTHW^PuJycke3!@05LKyBWYk182@yKN>g4jYB& z#9CySqEeDUFr6R_pPo|i#MP9lo)D{@mC7zkE#&mU$aqfI#upYFt#y@=QVbwp1x|-k zwZnVS(>r?8qO`&RE}vPZY$REUpB!DaEydV})UX%z^8gu0&LSuB(V!hMUPLW&sQMOh zGHdum#b{QA!}g6EUcABVHpu2d{D&;8q(GoqKmDaWY9?*V0{RagMg>epj7mYqubXcGQ4 z#N5iN(M=|BI&?xtTNkiTHW{0-Ed$vD0d@j-PLsd!F$nw=4D$4L!yDT@Dmeg?*#n&J zm5A!tkJP{hDW8WvMn-XalXrY{{k*#W=#&0Xvl&RU1Tr#y8!J3;qIO>X$zS$z4>t;s zpYrj#6V&p9#6QurP7bI_@@mD>37N7P@mAh`M!=U$SEGH8t^*!)Dc6+Eml|^u6y0Q| z38fgcLUcEtO{A{3dwXg#w1N0T$#6D5j<3V~x(y0j0~#{2TV@Lx%F|}G4Hk+(=un|W-!jyd$|?#RO^83^rGz-*{PD5 zI?;^VN7*kZQvZ(6?hxSJji{J$eDs-#e8sZ;aZEg0ooof~om8ok#XHW~)=3LEiIf~s ze3EF%-K*H1fC>`$g+yZl zo3>(YZ7I`au-=wI839jIwsaK=@ks9csMc=dT^B$BHCCy4sDHKNQWYUrGu79~dV>_Y z06ycnzWOr?P`&evozl^vLP9%58#*Y)A{DA;6ofvNKgk|+yIsjL9G3;F&V*f(%w zXR?T_nb@BsE#jdZyNyh0THzey8VClYJexz3E91y%2eo9z;;U2%8sTA(j=9RCsf~*B%kYXyTB7{Z*f`$jWD5tG?zv zBIp|rSG~&Q0hp+v=Z*Y(oiQE7mP;9yfy-t`HK=oZH^S8c0s6xX{C9<-)58NiXz1Vp*qCpd09V52lo=EXcvE^G!zh? zJ|5~MRGrx%E^vB~FhS#4-7I~Zp&bxl&Z7FIgS6h)bo=L8Hfw$VnnH0d^uiXCOk8QH zJ+GXM<-+})$>5VLIQF+Y!K1HJpJ*5Y=H_C&M_)pIK#}7!&e~}!5q3j$tf%fvk{2D3 zS}YUvkpml|G{0OC;5Vl@)a+ajG1_X%o4#O3m}xWMT;pv)Jd|>@yxBS@g~ea|D_XhH z1gL8GKpJfr7s-LYFbV8+yvG@7b1pE;Jd~DeHzLIU-k#T?i`g2`w~zDiU7dqQ5N>coZs%W>Hy>LWdsi(vEtoD}p@R3*Cn6&T9}MfR2l|5L9sYXYY& z(aT>e64TMM2hnJKyr*492HQMIres1f9ou;2>$#JFF-O8LT`Wa~GFlW5 zzTJ8DUw!MHynFE5mvbkd0_^j)`Q**XMr>~ahk3%!5zO{DB1Cs*8Gz`_K=i$fiWfv; zGG=vlCx1#_B!NkP6X|63+3m##0H=Q7b-BEDVFEM|mi)5;&x}zWFt9UVV1OJ+y}<&# zA$=nuPJ3qUlKTcDStT#_U}&m5U}!X8XaTQNl`hv%2=7sljj;_i4jv7g5R(NYMAOfw zDNjvpe<|6mx3h@M-@)%3`9tuP_*j$Mi_awZi<0Bw%P%n|^nqkqPco+hfV^jcyap*Z z9_&jGib|X&8&m9ElLnm}2c6{H3%>_=q_V&mP%3IIBWV|{7NQ86| zP|?L_A9#G9?Uj+G9k#l)b9s#nFa_nh!9hRYE3a{nkvahgRvC8XthBu3A2;~M-noNC zJ?hbQskiCa91YzMX$5xwH{MU~JgmIF0H*2W;zzd3urnQe!ruelYW=6?EFOQx(ZGMO zPaXm!yO(Jomj6?@rokuMd@2*#^+y1m@tm?E*q-;OrEUk9I715#ty)CY{=3lM=>Y=0 zYH|Fz#s!LZN^eD&8Ng!SHr+V7>2gTvSlj*o*v*ipH=_W}|5d3ij*KE zQjayAJ)V5y>a}nhCna&Pr=L~KpIV$c{O*uk%a=!c?bQcD5El?qM{V-*-LofHr zwMgZ7^z)Nn-lrA2ipG3D?-Hc~xu6&jT$GyiQZ(+#4XBpR-I-gmw^-S8>7gv5cj(~} z=JlH66aRZzNPH{~02$eE5>M05kB>X}@B+}ZNo__qu z6?~yqk~J%e`{#>zO?kj%qdTHa5;opLTkZ2?rigWK@9WfLNTC`5-@_;q9dTw3Qs4%X zg;^gb)n@A^^Er5gOPEWajWPybxFJg)okH`TZ$Zz;Vc5M?hP>$_y-KdA?;fnRm}Ad9 z$ma;}UqsBksts=|D~0)G@%6?pU`*d#Vq65}t$P0|jq`;o(c>G+&y7QHxO|A)#*$3H zR)$P~=X{7p2fkCrk6||*H{p~djooaiwU}j^f*BQB;J(X?&6%i~bPcTShpy(d_m|cW zCl^_ZT)ymhs}!YMur(I})@86k^U4eU%a(r}zkfF-w^GKTaX48Oq&trWYjHxa?seK4 z&ERK`tRIhwW;7}h5HuuZ1VZnD04Uw0+!f_E8<+p)p0ThR%y_}rwa~gVfv^745OCGZ z{E(&9EUP<;u66NFj#&_TnY`qE(PH2$>xQ~?EUqX%mr7M(7%cq&uoVvdJe^#l zXcA^bJGg##f}>8DNAFF&W|nzC0g<@Bn?1=}mn}_PFOVVFFO>7HcQ{7U!nPNff{EfF zU0v9@%*V)>+j8p#_KZmN&X<+QV|4BM%DG8&Y9VhZW)*mbue{2&?G|6qx5&t+EF1H z_iQjF-_m+gxJXp8xYB~0%Bj|AeN714pY+ep!!bw*W*Ydxi^ib14;CA%30Hyl9Al~3 z1p%WQD6QeG{_%ddq(hYzCU=60`@PI$D%_%dpb@xT2M4K<2YTFFA$qTiLBwlegfRVj zhJUI(Y15Yo83oLq_fAWCa4q^#uazaIS^upt4^=F&sxJ2rM+UbS2XoGhDD+j zB|UDy@>4~k9eAk$%kOqM>@;NyV1cxz31BgEy}B3GEL$QQe?sX6cJ$IY`R=QIUo;%Y z1TlFMPRbhj!U2o-F`72N)R7A(bk9a1I-)g_mHeAh=~6N?yc zJ6g)-L0%eC2&Gz^Vv2iuDSphPM76K^S;(qav`HzXv%djdN@1EHVAqgm8qE7h9fenN z7%ugwU0mK|bYduVg+XLm3waZ|B|Ys@M?yTKiz!Sa1RmRVbT2y?(Ib~SccO<_y!O60 zdh}IDQLpImIE~mRLRdIdg=5s!&EgXS_iFR>W*sBrxyF%d{68e@J;{yUp)83pMe=lG zM%7E-UxzT3B$PPL&Ei=yg=CyA?~8I|v*0CKZorI; zZ2tov_iEE@MX74A$hi0HcS4Y@t?h757k-ULL%K`Xw4XTF^DB?R`{TDfM#Q^v2x3q}8 zqKn&d5J)5bq5@+-OB6fh4*#lFjWq{V6YM=EzlgwFH;ZpT_sCrY`)phUJ*;%t%bSLg z_LUO0AADP1>)FlHODzU1GZaBa3LHQUTA$+kWh-@5Z2vv>i;8t{b4zAU9Je`MdojE& zFj4PEIUzLMZq%uKgLnOWVMr>P;&II9BCG9^fF=>u{rFFdvTc!MV!jP|Q@eEQr$wNx zEiqqd%kj%Yn~GRr22nFt2=@JoT1k<}>N~vkjAP^FCnhPFu-x_0(d=MSj$zBL`yb}A!Jr^u+)6Uq8$k)Wf0s_ocA>pC+*Ha`b9Kz`r^zfanEC zkwU1yS&SE{Ok=<05+En)0PxBpY&4Z88^6@4l=;RZ{al6?{W%y9KDil(6JR6c>FqAs zTNKOJjCR+w$$jO4Ld~P4+@^xwdllY){+Q{y8cz*g6Y5!^CEN9tl6vx;DGE^}&*BHd zb_Zfa8FNxr8A#a}VK&^Lsr-CBXP9@ny*K$&d%TBS10C~43jD3P z=%8{&%&^{u>CHrxPt&xAMBhy1vj~gKfRDQTpZ0Hx>G5<9RpO^CSlA#e-|Va%2QWh% ze7b?WvxFSHcf6!oxfkAQ;uVQMM{*xGDOdT}him~6AR;bl(UjlN)}{DY>ZrHCNu&9h zO0C%oPt?L!`yQ*DGY$WmI8r=Z{(FnSV}+}bN~veUxBB^8?UCJ9u?LwW>)3L)m-aRC zphN`bpUH!6KirqCA89V>hT>RvG99#|@&*h){&;$^jR&9to>VUC;%{!lW;*!pW#rWb6;ftV`c2&vEz@hAsSl&hPE;Yuken|@S%kA=c|t+upJ zm%mjh@GMc;>gHtT>Ul-&(Cbjd_{javOe4xfGb!82C-!-^zBAG1bl>)l=bIRVQv94) zxw;6#j zcp$HZzI1~R`?WcujD{mL)e{%)H!wL8k%(pkMHuCtF7DrIBedgw@!uyi5tEY<@5Clk z6DCx7{fW8tqYF!3DxY*%ykP62xKPX1FClK{thGBYJJ8&c*`Cje@bkBJ*oqq+$8|8f z1x~wE747+j>*@kDA69oVe1vqCL8|!CZ1|vOx^o|;ey=`Xj#<@R!y4OoI4ChcbxRgl znrWo1K!4CAP}{l)ot81pi{+LI47>e7Zr2fQ|6qZd;vp#bExhQ9*v7{cb%Uf8wX!F=RhrJkY%V-lVsp_aS@JWuN+R2Hs7wymACR>7 zTF+QSLtnp{%MHJ&Z<|;Q@VV6=?G;&#V2vLr20#n( zJc9Lt*LjvfFp6&F9+Q)>G~Ie*ihjffjytXNy6sA5uLnz5Z)M~Gr}e^(ato~n&2ndl zR?ZA{AGmx7D+jd2pC6Ro7q=IKgO5pP%2$iGG`Q)xG6MOUU1iAzv?!ndY|lutAIklm zqTQJC!KMHK(KANHPDlj2UtzD%L{hoAajz?@o&O+HmSFiPF+4VQZKRXY$6}{br!Pxe z6>lBgcDTxSG()(DSTgWe?LAmM=Y`;MkiqRKJqt>ws;;M4hxzfMauDuQpyIlDb)tOI z0~@~&>hA4|4&if&k#~)mHSlBDTjhU1nWteD#fu;Vef@a*HfEr!JbxR#Mzgqdj+3UgmsG6vr^ zT4=iToC-K>CfIWdKhL6Sjc(?6rhBAl)8F&KnpY0t<<5wR2NvfAQ`|3mS<@HV-lzVjS4OGW$JqDkUQD1)>SMW^ z`=<2KpY@!5QxQe*Kz;4P`FK^gw5m*-=hKZB7-teVMGsLy;IJIN7ZPVLO7B-53!*8BQYM~;eWVa_e-L(8{|c_i0@qet zeA+c?TF=ij@WY?YlAsO|=wgj`UFA~*sEVvh@d=i7Lq;&dyDH(;fXk2`@H_|s?Crq6 z(D3tKkP1BQYcSxpny4$_ZUhbxkVcN1d_UVzF^T_U=GI85i5|yT}rO|GhlC3mJrjKPs)dYj)(s zMj6P}0*a71sO_S2EHkzssLQYpP{EpgJ#yUfZuU? zT%u1GZ>VeZ?#_!=@fWq@4VnSDZQRPz)}4;;k+zC8KIIo7q+rrdF*B_iK7?jY3pI_! zx73b=w|Zr5Ck`X68HaE9y^D~dFuLb@JJAwJ(ju{?7}=d(J7Dj1SRd%-F(4lfyrXbq zsdpUI#xls-J=YGRQ{WyKx=&b1QFUcjup`woAh)I*fEMw=)Q#v<sp?%+a~tMO16 zz2`*HmN2v7zBlZYa`~a394jv@Sbj9Ib;jBTt|=VT%Td}*@D0_w_#W4Z&JA#InkFc* z!V;C9MOUerqKY!-Yje3)2kwzG-EnP7`&Ht=&z&ji3wW!kVeSY6J~h_NgJKLWq;<>>HYFTKG;?V4mjm846iDyqVBp4pPNy)l#LQoMN7MT zj)E{Qb(y5r4?Jnu+tfoznqiEl2>iJtx*TRLjo@CZTjL%%0=arwREoN}T#{QG;rV?Z z%@3}tnl!zrvYG5ENo`bv`)cJ8Cup*F)kqm}acgB{;6dp zb+WK-v`-36fO4L2V8ATP>Hx{XVW(BhD%3?YnuzRV5^Z=M^{qx)+2& zucc9wqCd$ZzVVSIg;|t^J_JmXWgopI*+a((h<^ggMl@tpYo0*XE+3H`57@a6v+r{4KLeE$o_ z{#TIw^0a6i$ji=z{F(_REpT6n6=4Q$lJmK#>)C$ptfHExCC}f30XVqIPO5tWh!Qt~x z^_m&!j#s$>3yMU{4%9-wVmF|Gxc%PcQIYM zi2ZG9a~h|2wW4}t7eUhSYcF@I?&C9quj@%O5LA@&n-KSy)qocl{ z=%PHFReLn~Rxc=cQ5ifCp#BlK6-ifR+za8Cr=nt%7GMHbX{1ZL1*Lpae+?QFK5_XY zf-FJl?Bdij6e+D7KB1F>IGS)vFo9M#`pKoVX5Y$JePu?-<&GU8WmUg?7mKNLS~N|k z|9E&_vGuF-X-!=+b>`!nE?-PR%J|h+XT&|-!|F7&BmqAE6u{0DR7oLp3f>w#H%1N< zYR~-Eeb5Y$zgqf0{((&vXl>F=maMLe)B_tII~`pUjd!d|Yah=Na3Q>`I{9OTu9lS9 z#ZDm@*LI=R{{O|^TZcv2b#3F=3L+{cQlb))N=SnN43g3vk|Ny=iV9ML(v3KD_s}EK zE!~XLF~dj=F~qkAz3=CKp5Oa^@A3QVJHF%i{z8wru4~6yd+oK(bFIB|n&HhWcuAkP zFJos{WXRdr{w)qD(4e!?nS^T1+wW56vz5U29>?dk2AAxKFq*$iC1H-ug5M^7125Xw0yj%9W?og>+d2`+Yl$HIfD#pMNC^OOD4dAnXOpk*dNY<% zKh}9jZG)^G;SlHO?E>DO3H|rCXWo>;ib(PEDUftO#10H7Q2{@oZ{!HF@#vccI(aDfg%NrEJmU!hz?A1;{B>hK*iGaylzl<-YPPaIypL5 z`~9wEBomtt?D7ZAzFEyvA4r)fDeqeLow-Y7OxGt{E`T2`_bnsIus=KulgC_Sx`dDVptTx7D}slk%>;`CR+H_^zAmu+f!i z%E1R=x4;9@dAp(ipB|nAY9!cl{!2`rTMJ7w1t-wGq%W>if>;N9aB2w_SoAhvk zMg(Y`9Py9VgNee+mrjMhe-O5mu1zBkECkinf4wd}{DpZMWFH7WrrrDyVA5P_(rpM7 zo8`qAM)?M+c;bj~iMOB(CHS8b|tWhrY25poJ=al=kY6*Ax|M|hgUSA)$nf(9!AbPcs+1%`=&Z)0FUOXutRLG?p|IzgB4BJU3_glpk@^wZ= z2?>&aChUn z>j3pb-oKv`EB((wx>>>&T|<9`M@*r|E`97(T*2~EnaOHyAA<(Q=IO9kWm=(~?99UIX#wmL`9`>We)#ncOC z{OebuRLlCMaKp2gTTG13&IT7IZYt3n&T{!YSRA(u{i8Cc2-Y8cB`}%?%4D$E<89YS z+@enK>oT${Q@`c7M{}@Yi2d=n5xHp?WEXoxcmJI&voV#P_ z<|ubw@z1yrZ0x#rogDw_5;u$bbQtlpyF#6NUah0oRxS9$b>mcHf-$$7-vjNF5u&qI zh$brd{_bJ)$=aiCB>ubM{9Oq zHJFF^u3d%Pa|QVnjJY3RS0s;6EZ5qagT$U_&iwf%Zf&Yl*!AQ|3GMf~*X<+w|9biw zgjBr`a#2=ax@(& zz>7FUI`^Y2a8yCkX|@TdOJ$gO`Hgk|;)}pVzkb&J;;m&YI=9;S#3=sNe8={m^bA)C zC4ca7ReynwLh~{FTkRWFpOs6$(95P+O;ztVTsYDAPwEUYWDHOWa@%;{oJaUf1^I4P zp}O(O8?oT@U7!8-hkHAUz6Yu@oQb$&RNW;vGPUs?`HO@%DvBC9JoCU!xLGw{Ll#Wxi&-@Qvx6t@mNO;G7iSPYxkP`E6F)kC1@ zB}#5_p-pq(+&t>rsIf*GDlt@5TbsY+xf=BQ+yFmQha4~}mWN{n zu|}s>)@o`1=E}xqTz<&GB_pHBn!*j??pLoz99YX1&yFe0w$#6b&LvLXrbkWJm3j!8 zW9{7cu!WtKDh(yDFTGr}`>i_$|55V zJ+7KtM}X<1)ZLq)d&+lzrwDdxX2cc3=If@um3lPoS-s=23mwQby33(a5PRRq4e{+D z32XHD6Ncj3wHjKE9gCgW+9ZRX#Etg#8=u(QWId^BI;8;wPYCX$tN4eeOw9ChsIvQ8G? zDe(2RBqR9B=a%q3GO!ui5!Z`zS27V#3nC15Nxm1P-l-+o=@2Y)OthxQ4RK!&tE-uw zd{)QXtCXSWS?7&=7otEGOj^B)UwxZY($hqkC!*zd>HaEd$i0oAPe3^_u=@{4*+{a zdzWn;&qpxTvv*SqFN6hNl3hb!hCh7Rm!%AQQ1R_~5o)QGa^`4n)_}t8DCuA^1?qM9 z^k~st{Af4y>`)DPq%57hK`?i}hS*-yVR!t)XwipWqmC=Gu~6^DzAfkJF>C(G?_xf^ zw_!aH`YOxyFYv*VS+Sj$<4@NFPu7(e9L7MEOzxc(Lz5tGn`WA>-?7@S!%x$NH*iL9 z*t{6p$AM*kliYErkIpG>&8`!GDKwLYg#cSu12aYMSNPY5%VCO zi8wW@E6(eUnb({p5XWBdycPlA7yr6KGb2RFq%hQD#9qzPKlH#5kQ`q=(gBP`5XzO9%65z-yGl}rJ19Dhl0#jFGd@DS?71H{oT8n^6diw$+czpZ zrezfnN+GMm-J!N)qW!Bq z?U-7Rx$pbGe%To-+?hY#{bu}X%!~0=-#7j{cc?qcnAnWBpOQ?QHEs5^bV&Gug@65; zEzB?ZddlX}27Gn!5;CUxNu*d=c7Y@>>KnJM z%KBjBeE(6l%6VpH&M0DjY;T~k(kzmW#q6^VyoY;drEj(+UAE<>&WkoK;r+eBJ^Be) z$X&JY`u%5j+e`26lm_)MhGi6{6W#4j<7YIFX1~ev`53lR5@?Kc-}2^&7)dSMQR{?| z*j^+lyo(!W5n97VU-h@{a>{V zZ$+5TI&RFEw`K&IZqsqGj5@M2|NPZHvl_uH$3IpeH_5i$YL<~GO39oO73Nk5DRbMY zN5|bW-;75IH2_jBNdh!(51tIYS*IL6C+Kdg@5=>!{xe_^|bkI-YM zHLq-yC_d;|Y@FS#?9V?mkhE%Ot!)z(hDrB z!wS~|ZjrtWvw%9ZqWQ(QRTb4h3KJjqwU|rrn942&aHKN>xrU0@wt-z-{9vJhMn+G` zWO`@0<6_8jQ>W-hW?d`F;fw{5;QY+y>u!lQvAPJGSBpL9CRUN%uh!fn?zKC0ARp`a zSEFle=LZu>MKP^vj()m%Dv5rFVj*n6aBL^i&f9Y~Jn{y?7;pLH@g4-!)4G(i^Ke+? zkGu5wxx?nt<;G3+_I&DSFp-;`Ao-N<*cg{e`7BDq@o0M`+gDA2b9rCe?8|j70gN4W zLLKJv7;eAr94wSAuGFlap+bgZ6ko#aG42vWq*&h`nR z)ciL#hl#4pjnt_0+)EE=uhi>YgwWu=vD4#bln2Ab!>G+aokcTp^pEukP7F+kah1Dg zI?kqr#PVwSxXSw5S2;k`H_GzUAF5Gmu(oj(G#Eib_nX(R7(}F_5)0_;-@=B?a%@9o z?yoK8#s~=u-FfI8s#06_P{&gwKEXMdaJK1s@PwD8_d@hm| z^ekEe5V zqF%m^Y=lPN^W=~6#QPW1AG%WbK`!68o};{_=^f{J=jD9rD`tB(3^bS24Q{w>iAQ%+ zoU?(nT6ayZmvWGW>c>>=yeyuIN8GWtcCdIml7=FWNso56XRJBonP@Z43`Qqb7piH5 z-aP?)UJ*a|5PN%;oMMA_DqQiisrQ~)T&=pp3abOFRV5Cdp%w%pzY=ey0#|_`SgBO z*r3#{PbW!8FMojrxE$;T4|`b5Y6qLU9WRm_z+Hmax4akLR9Nd^-4De2tl?7t^_*Wp zEI^|_-}f~~kH2`;Zw#g0=)2w($0tWcV1EZa0S`o$O5cV%q37;Z@j6)t_XwK^={a$;r;E^o5;{X^ob3Iynvi(Dr8>u zVD&zc5j$0?e8R(^ao#CecRi0)|F>HBmE|`?=}&vOWmv{yC%nI~-(@mTY91N`QD6)~ z#unf9L9V3D*t*e)sqMebs;&&5IyxLrCWk*B~xrjGN`T3rAE+eEVhUuT-->aCUH#ntXG2J<&2e z$-30tvwK4{#)02T+8xbz;wo0QL7B+ADWZ&_+Lgen;zC(S*9UbS&RQCmB-H>h<+TRm5{0hsdbzcFOwM2m9 zH{q>OG#qbuKPx2JIhyo0#SG~MAEx-ufz#uj@kDdbGOMHzy{~C*spa>LWPRxHZj+qH z_{v;*ZCSo-jAaZbT`9J)ec^&M7Bw~-DAqIFNh$k-Rd_{%(|vih;cbdd-|haRYhfx# zDhJdm*i&6F(|d>Wwn%SR{yR&Ry>(>)la6E{vbz+0jurW9OZ8XkbVA34t{pABmu16bQS6T%>797k-+LqT;M+p!YpKTA64YlQu z5;>j?itTO|ueD!kT9BvC$gTuQus>pG%f%g9j38`0czZZKxYO8ZDZ4P26gri%ow~Q4 zfUW158y!)8>sAZwuW3)-3#sq+DV`Q2Rf{ZirewE{^l9_EelBPKczXCIYc3_+fY9T4p)jtiRLE!btaJxA@)^x!(GyfId*hwl^?fb zUIkGm(Ihdf9~=m*D*+>?%W^Npjm1J)Rmveyud64Jp9aQRU;_6Q@?6mDg|uErj(@bk z+Om<#Afl^y*xaS8_TPsKo5if{&3}?qMhL77RM~H2&)a)?7sKA4^EBbOph#A;R_oXt zqwnZ2yNH$+R-S1zg*TaHd_aq+Z_sLI8(;%b1qXhWe9t^HTO+5X*+Z4Y^;y@PT87K2 zCfJ&w$$P&V>l*#F{^-Tv<|f8|qW6QoPgqUSOLLdLc$j1^al0(ui!*M0=;W6NZuR}BtN{wNrFjmw>Bj{J6@gv4OX zxmh|%#r8%A6BovaBeU-KHj~x)$BJ9NN7ptzfTK-HeTS%|7})445JUo_FKWB?Aw!-z z9fSJ{OF6UL5gb7UZd&qOTN{S8h61moayi4&GFbXw~tCorF@(*ze&)Y zs1~c~nA@EoI+Rf(h52l!O~w^lkbVh5-YjLfv${wRi8_suFqfMO7ju{On0%A=fSN}! z@!qs^HJYz;M^~uMVZQ#_ofF9Ahc$&VlZ$Xp83(56voSK8=L*1dAH1g-?698)7n;}EBN+z91(zD4myE830-1B8~cF@>BGn(8(qiVHXl+xC# zgWaO`u;&G)`uT=22Ss75t60zOG9{xaSNgBG$r(>4j;I^L`Jd;m=y%o#fp|fXO+77) z?gUo+D_&6YdGT$Wr|Py=5n`1!?l@hKba0lRpB(@mwWY(ZUU5{b>SJjWqV4WieKrgmEe%06eX=Ya zbQN&6_a)EsVAYO>P@R*kU+mAG@8p^G=)8H3z>av@jKOrK8LG}eOfzae_rVA2=Bx&s z&Z!z#Xo%J5+KRfOm_1_s48y4$aE4*d!<^5PjnMPbw_M5y+mTP}5cA%6km(ca2qjMO zZdZsrbt!|fvTDo5u6K*Er_-NyEtRoS7&crKc zX#?6-Vb}m6W!A=9tW&~4es@qTgi%E>azYB04|p8piNUR>z)Qr|x?f;YtiZeWXI#k6 zK|t%Qz?piN3yWg8^Q-Q%;=}mNh zx$pE}_HXpxo@@#Zx>NplJn`X7yCBuukH9T^RY=NUfFC(G`X#nc0%rr6|M-vZDvu#J)Z^Dol?%1=!_JO==+^zB5B^Z!PKYu z^jYk>C55^*zFn0j25N$pLWFAPoGnxXiPFTv*N1g7qq*RJ*P&a>10i8Td>Njh8D~4&h3F)^$p0@4MbxONr!cC${P<``MHC%e2c^ zq9@;K3k~dPUAMS-?E&0IZ}kEz(Z2?vb_K z%b9u`Q*Bqj9c8x|LK+T7L6YE`L5Q+}?;&cu{BNOk0pZ3g!EJD0w$|rHi@kB2At6rf z5ynq1;xcc(r9HP}jNt7!r%eCM9bG<2O#Rm&KS%a*b1z6pTz)mBeOR#2s8^l0Gytik zK;&i{9;<4&m2g(lOr1Q{2mvo)VPNt#=MsTH#ruF7&QdG*ySq(@-w;O^tr|FzB*WLOpWgoea z-xH>CTHy86#uZ$qkWw%P`XuVfK_*bp83VV}TMvg%)Vu2m0nJw%_kcr{q_5B?L#KsT zDD4k%lxw3ePGUea27)Vw@Y=?sYz0F|>Z>=j+rgr}q8eqEF@GdN@K*zGXy50Ya~b4v z5y#~*bN2jq9~VyX*`IxP_C&Mq6P>ansF|n5Dmwcyul{aaIN56>V$S6E@0n+R|4u^i zju4e)V1*f292ivn_us&~xv=NuU!nWW6(E4Gv`zWXp*Ohx>G>#x{*e7nXjiMNe!v@X6rGs=MhOP>@dowgz5(8#ejIfY*6w>bZc zXh9tB>={yWfM1J0&wsS8-QhN(hheZ+9&(PARa??LdK zu_G8z)=M=<56;xU2oM?Ifo)Nb0}PM(zxp93>-;{s9B6dW^CZ+#wT|tf5Z79PfNP_n&Y7GSU0}_3ER2gwH|oe~gXbITOC=+O=w1F8#6-W5mNlaN$13 z5V^Cf>`0R|C|2=C28R^weddGi)aZ}_((CYQoA0ow%C;uh0C?bd;=h@;o+~8Bh~vfMe~moAysOCnv-Tq(nD%M{ z3bQT&D`GV*fu&QgI~$kYJ(g@07(E9>VH^3Q6w zTA#-e-f#Wv|C-i06-eyr#s4Ee6)lc7EA9lL*q-pIpIW#vc$Egt(i-ud5S%OX<~8lP z3%Q0Yi2E{9HHL(&w>sPv4tr?CO^Nm;&Qy9^C=TANc^AMScWOT z7Cg^)77*Sotr6dzST>b^gPwkL%MkL|5(1x3y5WSX@fj1x6WQaPXJr>1;Pk@P?Q|L) zU7ACs<|O5{0Lp_ww&5|^?cCVjpKn*`Fy4jAPY#Sc+a1^*b70=~`XU_GI*Uf^g7 zP6@6In(32G(cQ%}_;%{s3_J$ z9{?vsMMXl+->CV5mZs}AJ4l$uJ5@`d=8#Zo+!w~t_3;Y82i(_}c?)mpD)t@bqQWs_ zP!F@^2g@vA&X$>@ve7!fC*EiIyLkO^&M(AuDKmFDeDzxkDyuUkod zd$l6Ia9<~3)bb;R@svOr$B8a{i7=m0xuT;n7t=G>(mj^^(8zi?t31v&zjrb*T3geQ zgOvRR^^TnDZ8*CFx+}kxzyoCw|VJox+VtZ`n`xaG5LY&AEk0Q_Q8; zh3M@}57dn`x+aQ|-{F~xVq*%qF~1 zQ%Yg;;>nS-i69SQe0XbS;w^t#1{4{oq*X9s4bc-9Ev5a|Tbgv#z&n zx~6eVmtzk01OhQ9bUNQJ%Y31KO_9R})&?SpE&a3<@w6|cFg7>Z6u%Behroqi;7+CS z*)RMMl9vM+k~J>*pt^Uu4{9@hAMH;)-uZ6AxCAD%(XehyqT0!wALH%?02dN5I~{DF z!O=Oft?haMS=b=!yFH1dWO-DdFHDx`mnYU^&fHl`PlmVFImwPF0R93n^cKrA4_MuB z)_UU=)>IS}One{Ay1kw@JG(gPpeB<=J;C*+bN;InvS*sOvrqRY3zh~T#L{eY*%F=| zu8zp_iLF-pM$(|OjB(8JURv?|E?x8hGMw~9=9iGSFvSAHb|0CC>!m*~`MQaj6F+vE zTczI+@5QaR+TP8Y=HO|SU=FjLqQJeuXReG70lMn^Wj9p{lZgagi%S0O!$U3l;arWK zCqHebDm<5i$9Munk%cjXj^2xwE^C@UMCm{pz>0TY!=Yse)AGKm`c_A$qRjYU0WH(i zT^qacnx}(0L`cugf*PAo4z%tGh1XV zwC7$w`U>?cr6nJ;9p;;M5TK?F**7}cD$G#WH|mr7#O_sPSdAyv;tzUq0a4&#dz9r^ zg{UU6`@>^f3H;%);jkLlx{=S1y=TSbN@>9vmrvc?teu^C`AmJ@$bh>qAJ09Lc&u7A z$m87cW3DMZY^!>+N4PqNn@vOR0U{W_`vC)p9djB4}drgcx@S4q8Pm0Z1q9*#$9e(sFx`%?0`4JfGH=VO9%Gm)UdY*YHRb{e(Ff6WI{;~r$`fg9m#2y2N-T629K>u`{nTB1rpYCFUh+EvCGW&!k;`d zIJ_I4rX7xp3lm(pxBjIRoQ$L7$G$rwjilS@$H>%+xDTmu_#Q9m<*F-$K;uFAy>OV% zMBPqv$nw(xY{5q%9k@+F{hRFI#iUUAXWA9!BTtrT+b_VLRf|Q<>)2Fr@)!>_oUvpS z@E%`nfuwbsq|peLi2XVbpY16nwcdg-WoSxZj5QS^dy6`!8oN4d zN#R09aQc)mp|)^N8l9=hrSV>=m@m->?vnz+cVBW@C^hUQaigQD@?#qx*w%gYWKwT# zX86#k+KEkYHl<@HowrbtEu?jqK`q(Z)IHAl0k&G#DQC0^MZpZ7j!6x$(dgt>iuN$6 zF6-ESbeX)AUTf5ACDKmtk8n;KMJC?9=-oXGFX~1yVa+(pbXv4ESq0mwz!<> zlNs&0rg`h3j{KoB4$^J|GD;{pV)pAGOtSx_UPT zo1->*Y)U7TTE*&q?qgf#zk@sr*JX!%Z3=!(gpb~k!FArFq|{C|Ow|k9Fi1phb@8&d zXOvg_i{R5;53(3WeV5X`S|c`+N|moL2kwca(C_ptY{`4781aPfpcM6MYeqlEmor&l zHg*mC)-QxN>ja5M&Z`Ax(fHNkDLlm2nJ*^f#j-Mk)=zH#N3D06BYBbbr@n2 z5`LS3?MS)q=K7xaJ^ygk8vDbV>zK)^BzwjIO5vQ`MbvWs%vC_x)!+ZI+QJpdT?%DreE9i4dJG1vrQ6@hw3rdO3G@ghu)Gzj$1f-rqz}~rp{zfh|2(E*bz*oQd6auKl4diq6Ux-YVKe$GXsTv9f)+P@pj09rP(SV3X4#wNN(-Ngss9*~ zufpurjGL`%ib7!(M>2sE;r;!q7^|3_A^p9KXpO>nlgSUlMsDJ**jUG}o;ITsvwd?p z;okVA8gpnit^Zqi0^@z$roC_=JZQ3v8$jAqRW1>^q%+b;U(cQ{uksQ7cCO$5&^$Lgc(mN`@>5u* zOK{V8ApeJP-n!qcG&hlkJ#I)TZmMH$x2f2`CXd@@*n57S8SLQgu^9b{pE<&qC81yG zt;CGX6uu*>>_M@lzt7A}P zQQf-LN_73!`z$5c#qx*~?%xrmq6I~kW(`GkC%f;zIdw=66v}jR?R}0M(wy#Nj5i%Z z(P_e(e<;_rs`~kIY_92oKj4|=3*t>fp?UBPRENj_|*WY%EJP=NM)5dL^K*;?Kqrx{vLt>V$w|Gm&!zc$5XD`w;0 zDe#YUV)MV==I-s;&jj#E*C?-Tq?r!zGGum@`1<%SdbOl*r znR1-`mZP(u?j#nzj?3Jr zK7@gcO#N}p%XW6c@2qDTTO$v=BR^$5i|{8Ab*d5Q2kdK!IX*oM$Tb&^npiV7_|OT~ zs(1wHEF+#u1bdecs(*ewJ-9XR(5__tvW`0&cgfF>BgyXO{cJMb@q7;oI)oN-b|#D9 zt*~~r(k|lxKqyYyjw)UJ_<2GJxcoejOn!UWk{*qyX)%-+!ZmC1C|Al$!(BbxVSl6TV?>)FZ~nI z7HOZ`e#)gkxBLA-_CcqNBAV&hcm4jnGxYgYxHX$~PYQRLq->DtVG;r>H*PL!oF$PTS#wOX02gUcM5VLT~C$UVczLUT$gvf z!EJr2lPVjIv_|hvxM}xSP(q<`CB^pcGTWE~x-(DrQ`+ z`uWNUy(6H-U30pO@Ns*wgDf`F-q8Ujb*&`P+vKfBr~O3TRHB)0}-jtDxe0|N8*>u6C@UC$D zUyL(tcTb=Ol$V*`XBt{|hfza@h!FdmD|V^gT3L$ZpqfeFB$^q%i%vRnY;;18$D2EtKv#NW}k!27P)C8^z;cR*21#@S`XohK?&Q1jeCjO!I zAYgVAVCFaHC3!MPtf#80yd*lZwCa1H9sY1zq|{}XOa+bJ1mP#IWRV&N&C*k<`f%UK zCTkg8ZRY{`^svVDm*ZTI$M$~y{Mr9xip$RJbj67rb6qh;xfw^I48k0@ztjhgNltZ-Xy5gV~LtTw!(Q+8-{Lc^sP<>Uu|BCYOy?F(Pa34As}z z4}NI@xsD9%iLGYz-iY3+JoC)>p;@wkimsWXc95pJ{aZlW#``i#ns4?)nF2e>pGmOB zob<;Fs~$DJWCPe;%Sg2_-2}b|D;kO&NmL0B8FF^>A5e3b z$oQlU%Kl?x0;tdDzu4P|>J#?XaCvV1!z4i=Wb5TyjQECw0m&SQ{?j}Y({+xZmH=!M9vgwUPG3qZHh~;N&igYZHm)X;nw5*W%bzDg6wdFD< zQ`@b21J0E5l79xKN}BtcRTbbw4ws{{2^=rK0*E5zUqD{oX;wUh^nds#fRcW65BxGV(%|nmJ&?9<1={ru3U>C^=o_#!?$tvEPYL! zu!Ls9IWATAhXr42E;C=cy(zCyP-VmbnaxR%c z=H{J|pcj508T`-Woc(0R^1(q-sEP)p3WQGbNIWy1(|#a#mAlFAQPi#e%jcBVo-xO~ zrIk$m)n$t~Hr$aT#hA9$G`YSSO>v`8EZE{u|%;WzPiPw!Ek~?YO{b--H zJM(#3T!8_WcKS<$fh1!&~T7I}fue;-h@r=4YLpedD8V_bD*XX83Su$}83@3bF625O#kx z#XNH~_`P~m%K7O1DaXNllXKqUHC_U4Ip4JIurxAYP!$oaF!cHg17*h8{|>)bj<3qE znHr|U;6;~sYof1g69-A&-H$^ZCnjxLC_E0r1ITCRIXuOwgmlW>4f>woq4@1DHJCMZ zaCsnrxK~U;LN$l}R&=@?08s%H-j?_SI$>K6Jd2f)!ih@zw|%hvl->zWO#N=_^otiS zoLAoQzas0a^IVLgmn$nw4|f(3elwq>!DSe5bP!Iov<+Kv&amUP5mpIwrjS6Psj z8t>SNL0I2gs1#49e03NfnPwWzT|Y?rc~V49L;>Vl6FwXb)>4JO_@cUJVYkv`CVR7X zzat`=+eK@o@Ey!Xu%A!UVbWFxiW9tAF-=R-6^DCg6rhuXqkKQz6C7mcZoLo^;?t#nfZg^WD$FyHcn0A%6C>l|5~gZ z2%9#0(JWm^M#uHH7lx-j6?gPWkoNnkA(h|%!ufqOGfnD`2315&e}W20;KJqW30o~< z_pr`?`QYCSfb&NWP_O3p>&-+-mp6dSX$1d0G*mz>bSJ;*^zFlX1e0KwAVpvGOI= zwc@oE&Tc(n%B*M>XUy~%%WS0{zFw}m1-fT!^er5RsZp!w8RabnCgy2Ei;!@R>=`}X z{zunmK|#u0bRov8yN8xYbrQLyFi~UFTH)SriBa&Dr%+ajrcQBYFI=_Jn?!nu?3axY z^9OdCRs1do3=Px(@PgD8`bk6!OWs7vD<4}jo-}HkY7ZKCXQy3hTX~~IyAi~@idVb- z#mpt8n=HbtZ@;l*rwLcDtZ1Bo&9nnjb@KvVPRp#BIp)bek2oNZ?vDeZR-ZiLKGko4 z4CNToIICS?@~Dt>PI(5dZhdHeF5C;yk@>-~ETwmI2kR&nm{02q(@+1>A&~vt>*(t? z%3>Ccm|7W`gYZu}ZLhBM<&WI{ZTkKd`rH>m z3rQkIfa{Ljsas0~^ZK_`tG_G6@;T|-B}4$EFnI4R2wXdY)@ot7tfdak?^;5syAp(N|DhOY#&Dt| zvQkOf50B3%YlKz1vibCW0jA&davRE^*);kWD4c7KM#<_ibO`Qs78KbQYTJT(8GJ3= zTgZ9+Fy!Rt84x){WYF!tlm|u>+(M{Tx=#zE3ma%!dV@Sv2cK5Q{ruJ7oE&~>%zqoh zI%W417eHVCG^)>f1f;SXxtS1x>nP0y9k-V3VY9^KV3SUjH%_7*N`J#y&~MnXmCEfL z&G5b2+0prZ6LO1AWY9J8@vmGcRKcJqHZ!PU)+V~XWHO^@wo!W+?$|aU14!xvk-g1OFi24%zegef9)gdY; z!XR@7G*7GNyv;TiRt;dMekp#u*45;`Xk;^?Q;$WIs$}NQzh(6S<=aq^67wcaPAy)u zN5_z%x+m59&HQi(j8yYhHgtup^A86H-kqhv%Xg%kw zm!@8QawnX@r%IXj4N$juQq=ZJfvepfFm-df6RA%VUf3SI__q7T(Tq5vDI8ANgGmIX zVuE2oaWp5DVlwo{ESuZNNsqTQ*#RV;YoHFPy+OMfCw|=alw_(PdR~B~)v}!P-mw+( zg4(24(@4@Kr|qu92W8-hPAQ%)huTwL?L9>qSH1VMk=dqHq1=V#$ZSSypN)0{&QI=% zOj1M(q2b-2f@;7dMUz#l+D#5v$j}Rqmn&q|7`f(6?sfL=teQ@Gzl!$sfBAAilj*&~ zUF*Kv%FnPmdp5&Syl&w9zrhk2ap!Z5$Sw77CP_&5{Si!hZ9xs~?9uQnBbqiKK@Mpe zfwoVtEu8?RqkgkTk>w)=w8J2gfsyJO>zZk$;P7mpl}22nd^_GC9ZmK5CMEr{|Fpep zoHf~aSJmP(DvL(Mz@r%6wu=eT!oqTm59u;t1qFw6^5aYXVVruRjLh+P<;;B2{@{*A zvu4Lpq~jMQxn~-PwO=!RD1Jtq4>-gyA@mqZJ87Vckee0J4~Wt#puH01HGHw%PQcT{ zQd2W_FZs&TBbQu61bt~&qr3*9fR1aGud~|%`>F?_FS=bQBhBMmqhn=0k|69ZbF^a* z*kKVyPu$M{%y+y-@J_ARdOB@q#Xy%+h4~LDX{zu`p3Q51n zF=0EqDbs+sSovG{vf~K>397qbufP-hEFpOLw zWd!i&v50VU5zJt&5KhUXy7uCF4&E672mrUVB&aY^w#Ps;TK7d2AbNT@(fG1HVtT#m zAWpFH_svReb9rrgU}|>M9lNJjM`-m$;tw-SkR%-15;sREBmkp7B-q?p#NAom*x&;# z>Z})3vlPhkK^9vl3uevP!uN&u`iS}U4y!$C^G4YFyJme|sVf}YzdV>BF@1sUdNH%+ zWUmL1c=-MVou24Qy0OO+)Q7HHNmV0E5+5PW-WHD6A-l6-pY>s1Z1Z(jDNe*EWvODq zCPxFaG09U5NCN2|`2-;ermGXii&H*2G1GOJI3CZ+`BDefa31f$jAgoksgT0jgS5-k zcMJ;Ac0)<7PPDuIv;N_vF&;tYD;gjA)J(mKVbvay6*h5)bEG>Xg$Ax*mks@F93*U0 zXj+$j4{^B)@PFZIcu=neoI{P}9DoMl3-Lk8*lp6!;1)(#P+u*j>;J{dZ@$2rYl4w9 z8_#ECPyUhXnwef06S0%MVzS1O8C9ojdx>v&R-W6t-d}qTs7ZX80Ww)t}Fcy`8uJQ%*s%+%eI9I6r(*b})&{e}yAdY_<=Rb9K zdjhI=bAOl9QN{J%Y^Mmo$K(!P8gty2&I3!9$>03M*i#Y`R6y`Q&u z>51e-rfhAR1_9Piq_6eoyQ$Qt)U#U06v8>lUrYj5S02c=m;*Z8X$lL zBGN>p3swk4h_r-`%AsQrLaz!)5d>+WgVfNG5)cAJ=`DmNA=JA9ddeO5-XHhJ8*hyF z_P;%{v&&j@%{ABjzWJ@N&SFq*SL7fr?{+-q@`G2BnybOEDbq4O;oF=gNI#$RFX1MX zbT~_SU7N+z^*d@#PHWL|BLt4M*nbQ$58@~s_*Wyq7+~;V`1S1KoM7{sLz~04#m*Uo z(E{U}rC-e=K#XtD3>38_lDGA$b6DBX;i#xT&$IAR%%lH{A$5RtZu#>g+A&XJk^%cD zVg0}nk5T|cyFLKpqc#wcPY#>gxPmvYS?h9@OiB3g;XJ*VF#E5uWpi1MM5{JYP>`$1 zQYS3I`P)S70zh@z;%n%F+I%eleBZ$nS^AjZq>l(TmE1nxC25VctYe|CgCI_ zJ8&=+qSGNmJ4x`pC2+>@y^pT}Ba?2Wx^gx*e&QMl1dsae$%+%`2BlTymnR>?HX41t zgC#Ru$AA(+K-F$VTFF|*n+asaStA(@R=16O`IRsa_&TVI%a=Qi3uRQrT?~zUcw+3c zyFAJ@kwBHe&V4V^gE>q9P}tuJU{g2DTKoZoi*|nSkxDACcr$V`!vL=_6!COKKa4QD zI^z>3IY9{}mxnle&EaIJXSWRTU2qIWyZt)6`l?E=4oKivxRRGwiz9EqEBn|}u7v(rD zlOJjU!l^uglu)mYzQ(k%o{^Ti7F*Am2^u4w+3j*{ZQDuUQuD(gTeBr2$(fiR-JPD6 z^lsuXzq zvJYJ>df`D+f47k-pGtfl<(g{XKdmrJ^u)!Mmtk{=cm$S17>D&8^15I;R-vy7r4K2qWcfm zB*phDSz$TfOP_)=LiGrzt1aDvx;ZhYCbTt{FIM0RwC!zRR?63VXkmj)3IJCJC2@YT z9QKZmB(ayi`5;732<7%#d>g}5+ zP2{hp!oI$~sx6Kkv$SH1A0wX-alm9G^Z+PBitsArV@PfAn04)$g0ZtPW_ARO{nHdM zi?(xLoG}ejbc_|a_{!EVI!%=zhmw-w;QPj#S~0!o;$xphuq`{> z?;ERR!!P7hhuz+2E!K2f6E9n0!5vL~-j1EYW)II62{<}ZXra>BTEYe6db1mX9?^0j_YxWsP-of{VZiz(J4&B5?Z>(gv6VB{ffieMY3_ zy_e;)4l@!l0|tVB)?PV<^N0>nwYxdd_*gNNJnC7rE=6CQ@{P^*FoqBJS@}syjJf`B z4x@!e?x!+ukPD0YP`9nC z1TwS*E-8y2pSvq#r;x#dm}bJ)hVxf!Ua7J|85!G?ZMTk&Cpg%_KouDeXt>aZ#=?hD ze?0d4`b_jn3iqL;L7c`pCQv&j_78j7l9<^>=-Ef6pHL2G%N>X zdDLz{d{`HV&(UfAMFHyY8*?n_^-p~_e%ig@@Q38;$ND6Eg$Mq1&DrjySF03lU)Dm8 z-5>Y8PJIDk5gN7Xvw4*F+Dz8P_tETXT5`6lF1vU*^ou(@AetC1!f9 zck=g3C1lCsQDpCs5IF;`-3vbMljV6V-_jVx&UPW)(dt`S@)68*UXrH3V$$aP-UmP9 zo-!=$mEgn^AGTA>LXO=4}kma9+0)ShkgZ|BF6Fhm#RC@LE+2piNlvj z(W3y)XB^L1|NldF8)D!8vE~|E2;%NNy7$nqFF>^B;9dX1#J%&MYhR*!v(9jlK%((q^%}hiO$=B2S)Bdd<-afM!+N-#m?sES(Psij$?mR; zBovgtVx`HuYDe9tW2S) zG zuMpZmXE3yGYnf1Wc41SN-k|`YulsIxEQ8t=-h=NjEV80`Bw?)wo$V4zotCgna|0W# zAW#|M1JSYUP6b;pFufZGg&%{q8*MDS$_y!}e#9jnQ=iJ=B7^P*Ab;SDs-D6^NbkP* zecc-7gDZ-s0R;p<*NjOpg98l>?d#XK0HMaP)G6rDFj$rY{Tm-|Zxxm#0}1l^X4km6 z^ejyK+}E5V(CEx|3LB`I)4RD((_$wJoekIb^{xaA3 z!sLWd1h+p)ZGA?#Q=)&TX` zl?4RhM55Lr3dlvIX>iBI6|aPtV1}I$dOBv~0SAF(L)0bAb=`&;U;gs^u0EE zUuN3&+c;#^+z5i3Tlo$WafIW25H!YOSI_Eq+4Wc>RFnd03_u0O7-L!6DAC~`B3y1D zE%6X4A)%YG)UoP*tG*4djYK&9NMZs>6{3+D-7!Bqi?&ODu741)!$4Qd#o8qR*QjI@ zQbd08nd-F>ND@r#yQ}!ORa4L_C0;wnBR1dO;5>V-hf6lbpy2vgB;;R2UJl3EUV1KJ zK3Zy-2(kZS{PsMQ(o4nE&H9)UN^*lb*6YVJ1@q zkyxW=;*ke*KMq4NKtc&12D2X{`7edV758XAS;%%mZHwqMFgV2si`j^Nphe)hweARZ z9vb>Bf{aZKvqpegsq;G{f#Yva{A*BkP}WK1bTiT!Y|kQ3#*jQWS~9~NpU#I~0o-H& z89D%cAf0l6S8J}6beL3L^Oer2TuL#-fA0M07yd%RKPj0F6chb4jpB+ZVW4%@z$>eE z{GJQj+Db+gSj!m1mpuFZ`^@_S>q?nIB8oghiapHQ>zR`y{Oi!|L9iX=W<5^m%-S_5 zB0x8`O6uLIT`Ld1Xoz>X3%-Vibe}roKGwl;Ar{%Rl))Z*$vP^v`7qz-G$7b2(WM8< z#fAaQewYNK44oJ%b*#KqEW0y^1@*!NVCNuX9aOAqaFoVKb{1K5U#(ZVZI2UT!90l- zZV|?A%`_D~CuuDv?X6t>Kcnbhz!DoL9m_28w^JX&HrHTKZR<}~PL@F+&Lgt`QKt+y zX)lRE|F!N68TU1rFDWAdC9!uA-?dTz8Y=2xx9}xR=ltLFJg}wC#WiBuVNx_8J2Du0 z!+_ApC|(;Wb*Zbo$pQpadqL3}Kip=J58f>Tjk(}%4p8B7!Q0BP+Xi|=rIEn3^Mkr0 z<`F#|6;I04o~(@GOh|(RC=zg+{9y1;4>MzCT+3$NB5P~DZswRj5A;NuRLiX1nR9T= z$vfSD-~Z)|{~7UCvPIi_h%<1UA+oTWc za+BZm6n3R$&GDpxNMrClS0TNydr|j`)J{YK&0yZYFC74vW`}s7FDsv72D=)F0-2fD z2Khp(PEtqzvJDbN#Lnw4ukpW`x4|-^nNga-I9_0RDdz1>$H#zl#rN62xD)$=Id_$P z_jkqn#qB>RKkEBr*LR9AlUldKkw5!#H;IUd$Fkk|sCC%pLePoyPTSK_my@&}-D>sI z7;JzGX;nk6s4MbuT|I_j3u;Nff(+s*e4c87Kfu$>$#xy)7vPul>{-|H046I;XGop* zes;KAhjZ{`FQ%&0k#}c3{@u(05R2@$1Mm6lSR*6!y!{8`6>mJ}1Hs7ulRs1+7&lkA z9lo>k3kbZOo|t2Gk^B-FL5%hYh~X3O4FO@iq82W>UWUd;hV-p5lI{aj`a$`wAQqeN zMwN|^tWt`comZ2+^=zD6b_ymtG-Q&VR)l>2$_w@mAIC}*_^W30IGFEcxtznm`5PX_ z1+aJi(zLhGS+rt#hrl+7h(Wgbw6({zZwrr<(EX^=n&HPvn?^{QOg99Eve>Tr%yx*CJI$|5FW0iW(A2P) z0gH{F`r^X*kY@0K)V5whYL&F}R4@=Mj5c$v9R^QNOz4bBZ?7t8WGD0ICar+FAY z^|4}OJXy|ij5c+&r{@0_f>hTF1Ssm?P@=nczSJ|Zb+HR6zM8eO>a+aF{;O)8+%mZcP%zATyu3nv z7k~HV8x)7}!)HqI)`rSq4U65HLxZJ`D&_N0!l(dV5{g_A3TGD{bY0BZQ1(Mp2F7cc z*Bxf@8|v3Q4UA)x+hVLe+K9;dZ?1g3@NAr$N@3+jEAQ@TC{zzkZaP)fEP~2CpLtWF zmcQe0W32w!!RSj`!ELdq#liflz)?7DkY0LavZ4A_HJ1=IY;`JElZ1c2J=_Zl88cerKs4UE>x9lZ#qS7jwFFj4f>Lm-PW17nS#hW zfo||Aqn78Df~@0@w~tD8)Oet4YQIv_`YEbma%zG6E%1pdwga9_7YA!K2T{) zx;*vyjd3Eor_+|>r1G!)o|}`zOV-U^TJPT*MaK0^BKmK&Y7m1I2A^&@lh<|AAeg~( z&P5W@$V6_=3;oWMtbMF&!E+<4Bu@#DeqZ&f#9)h08UVqV7B&RmQBtTyXOeShB(V_2 zFgj;y<%Bz?Amwq(mA(Yy3eeQpT}L(0Cjw;i1RwDvCKDj8=Dw_?RhgxaQS(DI^WG=R z%^AsN8lF?UWOEtd3fNwG9Z2XdLgnCgDEBtTH#%^ zT&X28D7Z8gKdWFWkkD16S2vdL$)kHaPiM8`BV%B0P|EbkxZu`yU??{=l(6%a8ye1i ztu39vkXRv0kvaa-Zi&QQ?`3x{nQ!Zn*?l!Zdj~X8kkK%<>$zJSTdo&Dv|IWuA~4e` zf1z1SLt(`47yj*9RML?nuwtWCme#q$d{5E9J269JqrXj|N!IpsQ@SUsL*p0b(eb<^ z0(>xYk~QhBOVb&IfcsPQ;uo(n2yymfxv)_FFdx`RntaKkE9J?cv9s$;S%>AWS(dzv zPHqx#vA938c?~*xvDUsyDn{)r=X`5C(|j3TCo|L!7?Ni2&XwPNeQATJ#Cg~4ks4fR zkqqz`3&4e%920S)Ux_ur^_B&f;^4p(Z&2)G&QPV|Q+oU&q z(+Godrd^Gr!Bn>J<_!ElG!_eC+r-xf+HS8N6x@=3n5ob0Vo zsng$*` zS0ejUIi0ftD|engq|^^v{Im8E9JL6%gQOLnjwj*$x!{z7M|pv^i@<3$WR6v8^1=p2 zKg`F=;88=RrzVEWcBt$%|M;9SK6SH|75_9fCxwHZm;HRbUAtoP7BQ1jk1ukJ)6}!m zM{`<&1YGq_*?e_z#{qLtg${>o;H_)fGvt6>tDK+IaHmD@&BP4fb3rFRvG|dtsT$w0 z1Swwmy?HcN6w#K-|DR@SAw;0#7)Y_*>KQZ7-5XK=QI~CCsL0w7mPe z5vn4*%1~Rvd1;I`>vk)YBC^K3Fs|uT$IGoiPVt-4bCsHkWJP+$R- zN9(KCvl`x>+#Jd4bOm<*>(jf!E|_ott8 zY4IFMCXYC6Z_?RNY`9TE@Xp$3uwM4vlKa&!=8XJug<~Tf=T(M-x5jXqp>P&A{YwWY z&ZCCQ)r8oYI&oU{O^84-F|lQNOQXsDH5I|LVpiii=61Ix)q8s;%r|>%i#(kfAwN$~ zI^WBNjfL7H`HAYrd5=|6EYR2LkV{R(-^v@yMT)?d} z-l}rbyH$pE)(C%==FQXCIp@d;HQ|HC@O8vUiMd?1`NL3#e#kt#Cf{42x2>g929=kJ z?|5`g4lX?=4*?&)E-8x0r^1C!s;jE1;)3&gdptZGLElIfq9iKm6S0frn1RoMqhs){ z@s>zLhEQT-TX<}U_tx1whJFr|HJFl)xcC+InZV?19t-B zVr4Nlp_W#C+3MN2*pP3OGf-%-YJGowAQxhe7@4ecTls`$yo{7}(%*@QuswQZ8o4e;lx_ddy zwdxd=@qww?u|eBv+h-o0R&fB`*K4%}8+ZmfiOWMc|0iJC22`X0Gp(+6u&VR62z~bT zn}zr$H6h%V*Ew$-Nuxks8n+yJHKDgVH+n%GaQn0Jb zCJ`#>#tFVeeFg<%th?3*LK!GQiXvl0@#<%WTT(p4@I?F0y!!8cSi9FUVuNUX8nFZ} P+oz(WaXtI${TKfMx8ZJ) diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/auditing.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/auditing.md index e9aa5070081a8..ac06e28366166 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/auditing.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/auditing.md @@ -37,9 +37,149 @@ as hinted allow/deny security. * Why "hinted" security? Any custom code running in the JVM could retrieve the AWS credential chain and so bypass this auditing mechanism. +### Integration with S3A Committers + +Work submitted through the S3A committer will have the job (query) ID associated +with S3 operations taking place against all S3A filesystems in that thread. + +For this to be useful, the work performed in a task MUST be in the same thread +which called `jobSetup()` or `taskSetup()` on the committer. + +## Using Auditing + +### Using the Logging Auditor + +The Logging Auditor is enabled by providing its classname in the option +`fs.s3a.audit.service.classname`. + +```xml + + + fs.s3a.audit.service.classname + org.apache.hadoop.fs.s3a.audit.impl.LoggingAuditor + +``` + +This is the default: Requests from the S3A Client are by default sent with a +special HTTP referrer containing auditing information. + +To print auditing events in the local client logs, set the associated Log4J log +to log at debug: + +``` +# Auditing +log4j.logger.org.apache.hadoop.fs.s3a.audit.impl.LoggingAuditor=DEBUG +``` + +### Integration with S3 Logging + +In the logging auditor the HTTP `referer` field of every AWS S3 request is built +up into a URL which provides context and span information. As this field is +saved in the S3 logs, if S3 bucket logging is enabled, the logs will be able to +correlate access by S3 clients to the actual operations taking place. + +### Rejecting out of span operations + +The logging auditor can be configured to raise an exception whenever +a request is made to S3 outside of an audited span -that is: the thread +interacting with S3 through the S3AFileSystem instance which created +the auditor does not have any span activated. + +This is primarily for development, as it can be used to guarantee +spans are being entered through the public API calls. + +```xml + + fs.s3a.audit.reject.out.of.span.operations + true + +``` + +This rejection process is disabled for some AWS S3 Request classes, +specifically `CopyPartRequest` and `CompleteMultipartUploadRequest`. +These are both used in the copy operations during rename, which is done +within the AWS SDK's TransferManager. +The request to initiate a copy/multipart upload is always audited, +therefore the auditing process does have coverage of rename and multipart +IO. However, the AWS S3 logs will not include full trace information +in the referrer header of the associated copy/complete calls. + +## Disabling Auditing with the No-op Auditor + +The No-op auditor does not perform any logging of audit events. + +```xml + + fs.s3a.audit.service.classname + org.apache.hadoop.fs.s3a.audit.impl.NoopAuditor + +``` + +## Debugging + +The `org.apache.hadoop.fs.s3a.audit` log context contains logs for the different +components implementing auditing. + +Logging of requests audited with the `LoggingAuditService` can be enabled by +setting that log to debug. + +``` +# Log before a request is made to S3 +log4j.logger.org.apache.hadoop.fs.s3a.audit.impl.LoggingAuditor=DEBUG +``` + +This adds one log line per request -and does provide some insight into +communications between the S3A client and AWS S3. + +For low-level debugging of the Auditing system, set the log to `TRACE`: + +``` +# log request creation, span lifecycle and other low-level details +log4j.logger.org.apache.hadoop.fs.s3a.audit=TRACE +``` + +This is very noisy and not recommended in normal operation. + +## Limitations + +This is not a means of controlling access to S3 resources. It is a best-effort +attempt at supporting logging of FileSystem operations API calls, and, in +particular, correlating S3 object requests with those FS API calls. + +* Low-level code using public S3A methods intended only for internal use may not + create spans. +* Code which asks for the AWS S3 client may bypass span creation. +* Application code can also create a new S3 client (reusing any existing + credentials) + and so have unaudited access to S3. +* There's (currently) no tie-up with OpenTelemetry. +* Uploads and copy operations through the TransferManager do not pick up an + active span because work is executed in threads which the S3A code cannot + update. +* There's a limit to how long an http referer header can be; operations on long + paths may be incompletely logged. +* There's no guarantee that audit spans will be closed/deactivated. + +## Outstanding TODO items + +* thread ID to go into span from common context (and so travel into helper + spans) +* Maybe: parse AWS S3 log lines for use in queries, with some test data. + LineRecordReader would be the start +* log auditor to log AWS Request ID in responses, especially error reporting. + + verify +* javadocs for RequestFactory +* Opportunities to simplify + +Tests for + +* RequestFactoryImpl +* callback from AWS SDK, including handling of no request handler +* verify that when the auditor is set to reject unaudited calls, it does this. + ## Architecture -The auditing subsystem is implemented in the package `org.apache.hadoop.fs.s3a.audit`. +The auditing subsystem is defined in the package `org.apache.hadoop.fs.s3a.audit`. This package is declared `LimitedPrivate`; some classes inside are explicitly declared `@Public` (e.g `AuditConstants`) while others `@Private`. If declared @@ -51,7 +191,11 @@ declared `@Public` (e.g `AuditConstants`) while others `@Private`. If declared package org.apache.hadoop.fs.s3a.audit; ``` -![image](audit.png) +The auditing implementation classes are all in the package +package `org.apache.hadoop.fs.s3a.audit.impl`. +These MUST NOT be subclassed or invoked directly by external code. + +![image](audit-architecture.png) ### Interface `AuditSpan` @@ -71,10 +215,10 @@ This is the central class as far as actual FS operations are concerned. Garbage Collection should dispose of them. 1. The `AuditSpan` class does extend `Closeable`; calling `close()` simply deactivates the span _for that thread_. -1. FS API calls which return objects which go on to perform FS operations +1. All FS API calls which return objects which go on to perform FS operations (`create()`, `open()`, list calls which return Remote Iterators etc.) pass - the span into those objects. -1. And Any S3 IO performed by the objects (GET, POST, PUT, LIST,...) activate + the span into thee objects which they return. +1. As a result, any S3 IO performed by the objects (GET, POST, PUT, LIST,...) activates the span before that IO. 1. There is also the "Unbonded Span" which is the effective span of an FS when there is no active span. @@ -120,7 +264,7 @@ and attaches the operation details in the HTTP "referer" header. It can be configured to raise an exception whenever an S3 API call is made from within the unbonded span. -This is primarily for development, as it is how we can verify that all +This option primarily for development, as it is how we can verify that all calls are audited/identify where this is not possible. @@ -130,7 +274,8 @@ The class `ActiveAuditManager` provides all the support needed for the S3AFileSystem to support spans, including * Loading and starting the auditor declared in a Hadoop configuration. * Maintaining a per-thread record of the active audit span -* Switching spans on `AuditSpan.activate()` and reverting to the unbonded span in `deactivate()` and `close(). +* Switching spans on `AuditSpan.activate()` and reverting to the + unbonded span in `deactivate()` and `close()`. * Providing binding classes to be passed into the AWS SDK so as to invoke audit operations prior to requests being issued. This is essential to guarantee that all AWS S3 operations will be audited. @@ -168,118 +313,23 @@ public interface AuditManager extends Service, AuditSpanSource, } ``` -### Integration with S3A Committers - -Work submitted through the S3A committer will have the job (query) ID associated -with S3 operations taking place against all S3A filesystems in that thread. - -For this to be useful, the work performed in a task MUST be in the same thread -which called `jobSetup()` or `taskSetup()` on the committer. - -## Using Auditing - -### Using the Logging Auditor - -The Logging Auditor is enabled by providing its classname in the option -`fs.s3a.audit.service.classname`. - -```xml - - fs.s3a.audit.service.classname - org.apache.hadoop.fs.s3a.audit.LoggingAuditor - -``` - -This is the default: Requests from the s3A Client are by default sent -with a special HTTP referrer containing auditing information. - -To print auditing events in the local client logs, set the associated Log4J log -to log at debug: - -``` -# Auditing -log4j.logger.org.apache.hadoop.fs.s3a.audit.LoggingAuditor=DEBUG -``` - -### Integration with S3 Logging - -In the logging auditor the HTTP `referer` field of every AWS S3 request is built up into a URL -which provides context and span information. -As this field is saved in the S3 logs, if S3 bucket logging is enabled, the -logs will be able to correlate access by S3 clients to the actual operations -taking place. - - -## Debugging - -The `org.apache.hadoop.fs.s3a.audit` log context contains logs for the -different components implementing auditing. - -Logging of requests audited with the `LoggingAuditService` can be enabled -by setting that log to debug. - -``` -# Log before a request is made to S3 -log4j.logger.org.apache.hadoop.fs.s3a.audit.LoggingAuditor=DEBUG -``` - -This adds one log line per request -and does provide some insight into communications -between the S3A client and AWS S3. - -For low-level debugging of the Auditing system, set the log to `TRACE`: -``` -# log request creation, span lifecycle and other low-level details -log4j.logger.org.apache.hadoop.fs.s3a.audit=TRACE -``` - -This is very noisy and not recommended in normal operation. - ## Implementing a Custom Auditor A custom auditor is a class which implements the interface -`org.apache.hadoop.fs.s3a.audit.OperationAuditor`. -This SHOULD be done by subclassing +`org.apache.hadoop.fs.s3a.audit.OperationAuditor`. This SHOULD be done by +subclassing `org.apache.hadoop.fs.s3a.audit.AbstractOperationAuditor`. It is a YARN service and follows the lifecycle: -configured in `serviceInit()`; -start any worker threads/perform startup operations -in `serviceInit()` and shutdown in `serviceStop()`. +configured in `serviceInit()`; start any worker threads/perform startup +operations in `serviceInit()` and shutdown in `serviceStop()`. In use, it will be instantiated in `S3AFileSystem.initialize()` and shutdown when the FS instance is closed. -It will be instantiated before - - -## Limitations - -This is not a means of controlling access to S3 resources. It is a best-effort -attempt at supporting logging of FileSystem operations API calls, and, -in particular, correlating S3 object requests with those FS API calls, and ideally the jobs. +It will be instantiated before the AWS S3 Client is built -it may provide a +request handler to be part of the handler chain of the S3 request pipeline. -* Low-level code using public S3A methods intended only for internal use may not create spans. -* Code which asks for the AWS S3 client may bypass span creation -* Application code can also create a new S3 client (reusing any existing credentials) - and so have unaudited access to S3. -* There's (currently) not tie-up with OpenTelemetry. -* Uploads and copy operations through the TransferManager do not pick up an active - span because work is executed in threads which the S3A code cannot update. -* There's a limit to how long an http referer header can be; operations on long paths may be incompletely logged. -* There's no guarantee that audit spans will be closed/deactivated - -## Outstanding TODO items +It will be closed in the `FileSystem.close()` operation, after the S3 Client is +itself closed. -* thread ID to go into span from common context (and so travel into helper spans) -* Maybe: parse AWS S3 log lines for use in queries, with some test data. LineRecordReader would be the start -* log auditor to log AWS Request ID in responses, especially error reporting. + - verify -* javadocs for RequestFactory -* Opportunities to simplify - -Tests for - -* RequestFactoryImpl -* callback from AWS SDK, including handling of no request handler -* verify that when the auditor is set to reject unaudited calls, it - does this. \ No newline at end of file diff --git a/hadoop-tools/hadoop-aws/src/site/resources/tools/hadoop-aws/audit-architecture.png b/hadoop-tools/hadoop-aws/src/site/resources/tools/hadoop-aws/audit-architecture.png new file mode 100644 index 0000000000000000000000000000000000000000..3d0adf61931cda905e601d70ef45f62945617250 GIT binary patch literal 55701 zcmc$`1yodB+dhscf+8RyA`K!Sozfj50-~fKFd`sGhjfe}B`O`#C`yNP4-!h3(%mRs z1I!FF^WTF$&*S^O-|t)BxBkCa%e7?AIeVX-_rCAzy6*Gx-dz=v^EBu2@bE}fZ!2lx z;SpHi;o*NF!UtEbcV3~#!+VaWswA)DVX{7T&gZf&?8nX{Jmo8=$d%~mX2Z`*Z03l0 z(GxfsofXyDzqY{Jel0)ac`0Ytr5)J>g`5%?`|g7 zIBZ&OqKsx5T@(y-Tq}O+xMt_?qWm!FKAhS4`I_l6#;(1Md36*p0?-URJTXW>$;lsh zhxbSccJe>o)hNDSf86?i`bf{h`dFDz|89Xn+1F7@T#H&+cL}T* zEc?>pa?L(BCR>T`cxJryIR);_URfmWmujaf|I7rzJy}hJz!xUlCE($2r^5Z_Re<63 zyzr4HzOaiMi07{&`Q-MPApQ6+eigs=-zeTOjc zcz%!JpijwAy@Hg|@+PbMG{Vd`U^bV1%VDCjXMiQSn=9mXUT3)?p;q_OUFGtsnesmy zn(re&ww3VI>9~qnkBFKbolJ~=#cJNZY; z3R2;#ToEXn@9_MMi5Owi=(@?2vIA*357o(O@&3ED~r_V*3G zlJX}r>`0CqM$=t_Y}kcfL2Ec`Xg(;h1OJw8EVAD zw;8mAe}!1#>-b_~Y0{j_`O|0j(E|})4r&iQ9?gtLB;2L(!G3PBSnD~ob5`<3xdr?& ztaRN}g9vv$B&`h5y;bao&Y96Iau4tEocy9y@hAJ+GQLIq=T+W#?SFwB8}VdsA#)tk zP1-mE;5#E(jr;STHmB+iR)^37Dw_kl1hm5q>}xsgokn?f*-)%Zg7L7jUQ9@2GTUoL z<{;Obb>meI+3_UI5t4qBWf4+7?%8)YQ1Aw4<4#*$nDx?r-DYixT0*#1#|XrGckyL# zpH)5o?%YcUY3yVu;nMx08f2cPOxh(sl;!F-3pAtASf*Lq!QrIu9!y$wd?DBe_Kmyr zcvr2Z(moN|wlMg;#vQ$DS;>P*`CJ8vmgET z{TZ}Atohxpf6lotY<4gq8P2Fk#K@VPp&SQQdkHltf#!x8VFRvVA9Dz6>h&Qd1&d3s=&cNN^D4(! z$;N5Bg$`9A3k+Y7V2q|A`mXB09bK7S2!s8|W;6A7NWl%+t=%)8Mh2d(Ws0&U5~`Ja zW%h7!{KOW`L@7awjRFH9NYwfE(?3TTMb=Gn7iXKQ4mW*0k2dD zhZ-O=Tz!KCveKq-AL3nEU$3J`n6DT7#{fJkWNUM*EV+}4ZnCk``>@FQkYMlYYienu z@q2~owVr9V*fC3nBddWx3>ww0&F>?@BKK=~QsS29vX4J9T+u+x$*quE$EQq7;mRTe zT^Q`sbPC(4jx|!e{mwMYnly2x%6jO|lNv$0&Edi_x4YBU7;cw!!r;kwog>-ZB0dbJ zGilcqVN8#9JkQi5bhl7Y@I`iD_xjN4Su=S?42MN^_Bvf0HkKXq`h_V)jhUwO1~A`Q zS8Gpr9(Ado=U6cRL_#7qLtZpzNi#t+4le4DlwV1)i#WSl+!zT}Y1j_so^e9KyW-WO zgPhFLquhj`Uro1XCkQY6$d9l3$P+8s32Bk|(^m9Sza_Vzi8?Nz1 zxvjFCMYc&lV^ZF{aWi(Qb0JsoK6LpfVNqidB!QRS0rr!oBHRznlf2x4v>+TYu!E#M zSsQ*?+<5SzR(Hs4dU7CmtA7r z*S0H}WNoy=%zqeG7uam8Lo}=T+>eob^SSm$UruJ?-C~R$uw!wuy6<-o0rD^GfAqhb z`s{!Dx)*$vcWA zLa;~9oL-DP+<2PFyXBN?np`#QykS>G!hiYBv(-VEW;$1z$FU602fbj|KMn-NbiN$4 zl*Y%?`{aTUCT6a+>W8jg6F1#^YP-imuVK``bcTeK67$Ve^zd%%J6)EmI=7REF1ACToE&bbl|tLl9DT2zfuPsavnv9fLR>`3wXpUz)8Crs;9=ce!Z5qG&h~MB1 zvB#t-@IWp;K^4~@KWGc5>Y_;4^*>Fx|6@Sc;z~i9gRaFmSbK|Xo>#_&qDPZz<%T^M z=jtpZ^Y!jRF*WJB>62_rTaAZa`Fe#<)S)Hywu17so~phj!LU@7*61RE4my&Y}lbwQ|%a(B;vk4ig`1VKDgY+ z`h^PxC&6$x2T@g0@2uY|^qC7NIk)mKnNG@~`0?suRD#L4!FkDNRtc#U>TGkxuw4(Z z<%W_HAk!Lb+lFyohp>8=m069_-Oj#gy(P6PjjmrP6DsUeM=NZyIbu{QZQS1>bH5NO zED<-`Htu??f>jl}a$~8%%XOv(t*C1c_ep1vQLUkSPXBs{oz-0iQ9B1t|^y}Xg1lc)aO zPe7hguWy9&!yVKitA4t$AE*2(+tRYS|M@?L`)JxnUX2*NLR?SthrN@BeZD9Q-A|*s zC1cBbs1f zE3FzI3L-OV3%M!^dqGj#WL$NlS%Pj&6nDmxFSX17e+w#VVLdJbH{ZRXGLefAdJ)|EdmO?BfgP5gDq7(Df# zT>4Ugt3Ca%+SfqsuMAQz=#$H4x8}0r8a4u~cGY>Pppmjv%BAd+?oEdJoT_i(DKkj) zpnWwT{qUq6Of+xF@Jg2BJ*)ru{aLaa_SwMxq@w3mk7V$+9J`L=PX;0s+D$l1Z=X1$ zb<+Kv2m~Iea_@Ipi~!?v_i772HEEn{-YUC%dYiPW+u>J5M{PcM(@k3}VqHA&GvM|` zl5WE14_vyWCja?8OK$|3<*qszGdW@Y|GG9Nd>7K*DC$_5L4xhrHcdp8)7N`*)rFR~ zm#rzz$X^8i0v|Su)haJqDKUkOYT}wy(k3=ZF8uh+;FxX3&c9yz;Sze;_bhlORQl^& zcm_F(T)N_iA41@Q+`0g%`R)z&fmHtrjStPzv82a*m zS;@Y*Qkcx?KMG~^F>CEpg9}UtS3@KNZoS6yC5?LD2QK{G#eeJ1=Pec{t@Uj1$nVCH zhh96-20#DqUlE_&hyQDX^4CcE__NsFoWJ0av;z+hbb4qt_8NT}um7=*j?UGH^KzL> zi;Gw5F87Nq64ROHlP^irZEvwjQ46j2?EIh&ydr|Sg!g%flatfD(^5v_V(a18pR137 z6Dhb$5dLW>wELVioydyEK8XTxvhO*(S?fjOP&PsY{wOd`Zs0 ze5de?-uRpvSOl)RWIL5uvmT2e`@Z97%EEvs5s&QwE8g?`C_bhnx<%qgpbC{yYo+Bt z2GV2~DdgFAK|QcDc#b!B36x?$T)EB!nI(Gorah&<#ndHSWrnHz(^vwq6}Mo|t(?JD zrT^8L9+zgm#v76RjT7faf_lW$Y8)=|i2SE7LZ@uM>f2k~O2-@p1#+h#EKHp(;*>Ou z;}JcYtx(X!C_es>MiF#tiH?h>i*om>*-3YL%n$)@>7`%dfgUTK^!R=<1!hkQ@~ z#KF&du@8#e)r2BT*Yo@Oj<>LWMaRW`beJ{1PHcT)ZSyh4U2SmiiH@G-+pg3wD~HAo z&>Jd2h~uh$azzf}alR*kWhOX~=1SqqyUStOY4weE6Cx@Rwt_R4U$jp)EcQx53mE5H z3uJyxQ^78Q00X5k&pM-YKIeB*`r85tl2>;x0uME@3w~c`dkh3pxwogJw!FJor<1lP zM@7>G95}zdm0lZydSaMeJk3CG`Fm`0(kar=tDmum2!~Y|sVx=`)j8h5+DYuecgCe& zBH9`~=C`r*`a5;+zkBz1pW?ZT6s1B^skH|nh9d<$eLLQM_(q|dxH?m!`Vm1`){}oz zoRd>%#g->Jlbf6eMq&wQ|Jd`$5(u? z+t3dY)&&}A94eDPjB_6tmKs*h`~gEmcqrTwUUfJbC?0hflSpQOoDjV*B)+G@|bJ`YPGi4448rU=az>q41nNS(d2VpiqEOZ z@Duy<9r->T7$)oJ~T{K?(_72bZ{b$FM1VnmEs-6VNVY@P8 z#z%T#TT?x(tRXLGcSbhSJ`GpZ@E?7=5KL49Q)+s*PUd!5wQzdOs9r|4F)M5CC;QTJ z*FsS*$J^3rQ5!XiVcYQ`*`YQ|F~se~ z*{NABgOBHz`X{P1ixVI;m9AaKlJ<=z;hi{aq!R?zej~=BZa=@3fH2Giy3rBCPE+)L zeWtO`BEdC^yhT?Z9xHkGs;oks1Bj}Q^RIEa<_F866$?W?^1;wdUYg#98PTuffOMHSiJ$WxIV#iAxTok~W8$fbmhXc%w+e1i z>1E<42F|!m;O!k0R%SM*P<`TL1RKhGt3h{_W8p@w<+oLLm^&)5E^b`@0`msiTSPu9&2&_?fecE=eIY}s=r*1lVenF{(Y5}V^AZ=w4&mZYg}YdX2^9g~B- zVU!`XFN%8%led0c;45CH{c4FLC-4=sZSPJwj9Kv%njmCO@eOUcmEp`NK&-@{*ti&; zOPrVCUksN+M`_yA?5y_#aB+2Hs}tD1T&~mFqJ3)W>ah!BDzDhuoz(2LMyRhWLXrF1 z%+8P21~T*M_`DC+vzA)~^WF-2ns<&Hls$o6Ya2nhwB-=qxtqkng6Byj9^YkJ@xkgN zr|dH(i>0uirAd?5hrV-)cn|0AR%@k;Z#6Q%xFmkHtJ^4l<&){5{kWJVKyg|IaEFZlDuSP4@NpTcT=E>;6tMamSQCFLvK=ARZ zlVl(lQNWr%5%&r|P6)AJlQKiTqFh=rPJ48dri8}$!JG3EZnk`F7U+202X-n!;(U*E zi-xq*Mk4$MQO(2?IX+=S2V|R&i3V?d>gyn6ebW77?%Q(SRJo78@LhoD@HGmn7s`5m zYiw^6+#d%95bOvX+@TnU54;qzUtUU!_+ncp2D7_^^+Jq7a|b^Qg}j(?&#^cU&SS!3 zE+MoF2@XOa5qN6+vfG1dgF9N)LoR_YGI6h&UfDXe{)M<;F|YKcBBSLw#uToEF<2}W z-TC;gb;IR32e!D*MX2jtfAz*83B9#7#l~=MzR}@NM)&k&`--sviuoHMxt8rd{z`_b zMaRi0w8yMn1QV~E4+!@NlF#pbE#SBb{VJcd)-d?EqQ3ZAH9f-S0sABGaLab890#H3 zAPUQoPK7U`T&(8un4uZ#tJK1U4&2B`^>ovDntq4mO)88vD_?1gNe3Abo3(4VbZ@kY z8)yXMmvXDjoH~una~8K!tvS}kcAiql(8HTqN__xy%lA-F75gCA?ksoFZJiPa&1lm= zW}yv$THWhP=S}CVvY!^#x7$1LAIaSv%71d(6F`A-=-VH4^YwC*S2~_GJgn*+Hz3X@ zkLcdeeXsP`x6`)JD5yr})GOq}A))@zgL9*sYIu11LMz0N#W{(DX9l!4;b4`%d`Z>* z0f|`gWBdM8*ZwBWK!{&zWF|iR9Pvgv#xO zZ?(~{eZ_%q({?>hPtWqTCa`tKu1&2?$m8I(g9v?#P9HH}haz21fKpZ4s&u-i)uln^ zM>}3?D0{|?B&a`F(`FR-WQcvH5v)F_|AeYO-H_WgIr+dCCif7kiI&2e!`&HA0Yoke z4qpD1jgVG;qM&s?Rk^HEc#&XVz|QQGeKPUhb&)H}U2dfZLDhHqm+AT<`R;X%mq6-v z`2hwBCe!DrNlg61&*E@zfb#&{GU+pdi%r@*vo!Q zShI8JkU`B((tY!Wd+gF&3!#m8cqB_&UZ=3Qt!C5>^;9wF7eFju!$+Za7cW(#dhShrUAV6jP6IyH;{0g6*t(LFifdhxxM}Hw89E;xHf zpW(o<9rG@M!^b1weHetFTU%k>`e%5TYEfa!Cw4tafh@SW&A=#^_?k8`k3%zUMnmhf6Ou;$7|O4U-Qpo3{iG1?vB+Dxjqxs~?JQi)Y&OF3!!JB2k6`{4L?!MK zl+aP0N&5NYY3OOxaA&K>T)=Eji?#6Mo3)>4D2=?HuDCy`H}~Vt-^k%#At>mCM|kI1 zdp~|WFZmsE;ei8o_+b$V!uLFG;$pG;(+?%xY||6=UShHUp9dI?P+JGxCLPs>z2ZIZ$Ej5)yYDYKgyb~7#zG6V z^8!bcCK==VPFhc8i*}q{yqM?P{VT%JNaqVh=VVYvq}|MRv+8Q&6zQ@u->20i;J(FT zdW(u#g`m`DtD(xoyN9eIb=X%U;=o>cMiK^GN>ux)ue6O!{ZEeD7EMr6EK8Oys%rPCRK+G?=8W z%4XzlvPdrnhY3eWE%0V@9ezZy%mCkXv>Jm>DTr`j^hl*EA4L9!(9QRmROH34bv{yv z>nhO*4;6eEPyYNMvlBSrP)7?gPa>`To^xAA#`+8%|@@0rOVppH?j{sYN`e_0oboUfOAIk5G=@cYd|;+KzrvgsI${v zHdr|{``M4tA!LPJea3$9p@xd6o>>&(9k_Wg-{~luV)Q#bdtJtS-0Cq$y>0pBPz`Od zSdkCimv^C;UfSkSjPi6*+1e?L&s_PGjxC$C>B^cNdc~~r*FrKfs#d7n>+jo)ep-LQ zW7>DcBC#Z7Ed0FZElnxE^w+P-w|vVv6WTT=YZiui_K5dn>(+{B`8K>>+Ol^a47t-u z?QT!)`hYM71Ga?r9&`gm`f*&}$Z4!%aZ3z73|#oA5+(-+w%N zfbIb7p-uO=H-#fop{Cjlh&i_g zA6?AlZl?N%fhw$3%a;SaCBgKB8g#3x3`ct8JeSy2sSh33jg~idFJ`U@k!y>dVc#|# zPH3M_7qTg1(wtYX1LFW1#oh!+^FxFQy(Wgy-oe;c?^VjR#s8*p#PQU3D=|H;Xk4l^Q zb}CW#Z)adAo+~m|?*X<(Rkhe;%H1A&&u-5D9?R+6V`=x90_m{U08g_;n4oYJ`3ssK z&wepE>SCg$QKuanVNHB)@$%iFmLb+ldhn9GMqTN`@wSV<04bUnzR>BMjGFK?!ONcT zxVE}y=Sam{mlE7j+b^69;}pVifgrF*t25<=#AiwAm=vivVo6{^3#^H?5A9UGWOrrf zL7?l?4f&X^cRfp5eYAeZnugM{sAq;pUU#~aC2O^_4@zO>+f%YNs>XLNc2iAKcT?S7 zT;WX^hJmGquJ{?lCR3~ze`DC@- zg!4>6y7%orSLb5e+q4%;Sdh=L~(Y*b?zHX<@8uH z>q?lU8syh)X1uHCeZU~7ozV5O~RONF0o5P>{U+*btpL%A^K%DAl`N% ztpmUw#Abic?OCb0DwaOc#Z%JeHJ9&5 zlW1Iif^v>|owc7By3M>%(1>ig__rt2%J0&gR9mpv$$8Fze}+c?9kyQ1?AkZDF%9rf zOp&*Qf-zw-*Pi@3TsFW_$-{H?jj+r=DA$fS`)lEN%gfi7M2`JT|1P>SChCXzAz)6F^f8X&>R1o|0Gho63+5ZLO)>tC$Rv`?qQd*4WlZ%Nu2_I5U zX$rxnqxkKKd6G+@czA}d!@|xl+&^U{{2NQpAosW8z}6ZyKLAWd%HGaE1a@TtD-d5x z2q%0W2o@}$SPX;h={Zfg0k=VlSYm`GipdJBxKyPDWMSm?L6mal4Uv*EVq~c_CVKV~ z;4ADoVMex;`Aqzz^A0-$@qtuyZ*{}_J93Hi(lxEQ4u5;cY4mG)9!G|>Jz z*8r+cL++U7=H!2#R{&6Gz+P0I7fqW7Z_oi90dEk~tgDtINVp1gVch5rJIYN7TuCc6 z0LY&KMc>ZN<5Lb1+Fp#41lKK>L|N}o-GUlHp#z61o%U4isTy}5Yp z$w9_Y|1-M;cQeROu?N-vRW;EbsM=l}SM}A~zpK`2|EutULF%L)IX1gGSN0<9xkcg> zITrlzb1r{1nk_$Of^!~=d-G$K-)|N$<>kHCgzNDrIcBjk%tHoO_gmFpb^oGil;S4H zDnNTKNo3CTT7K$-obw&Zkb(D&5P{x{zJhZh%4@F_p@Mzys3%fn7N zb|&Sr^H=_*=~ax@H88w?@2l27uX0AIp2l;!eOB&_{EwC+HYZ9n#r>AE2pf-pyK(8* z_FjIeGjregVMS@PLo1YK!q@Qd?3?k3AzcG>k9s?*jYKer02$j1s)X+9&pbuSKkofJx}Jx!~gdNnF-;w3E<)5iM8j4P!`J+ zU(NhA51*aZAK(d}THy!H{LgOGkv`b5p4t9u7yuJU*A`A7NAYP5bohYe-)s7B7MQOK z!V?QWb@4gdqmffwCL6RSz9s^vkfrrY3ELiyZi;fbX_C^EIi=jQ%|5xA@Qa}+fm-vhDY-Iw{h@Iy?{&m1u-v@pyAz(YDC{7OIfUBSaCI)bW zYL|c)Xg{_y_z?j@nEgCRaY7`)UZK+D*TmYXY)!xvCUZU3V+6hULo~a)OoD6F>>>H& z7eGlKNlA%Q;##JM2+(dEH^Mk{kzqF}&KX2Pd)AcrM5vbRxVuFY+5m8Pw3z##JA{^+ ztMvR=wgkO4^Dz5x#kSHb_I%&iiGII_6T;*z%VP~2wKid9xaM*gExC1E;&O80z#){XG@&!SE&=w9txzIo4Bqpxxg}(UzKOh6o?(8EZU$k>tyS( zkpu|+^Gj@dw2H4{Fg-z-FPBOm!EUN|-{oOw$lCurz+Gsr>NU_-N8Do*{Bl7zG5OW* zn)MqX@NM518QL(3GUf-o3p~A~#2hF9Y0K>gqi0DQ$J%GijPZU z&c%7!iH54}$iQ6a33dFblPVVNmw2TOO8dC=ieOT`EZ|H2Yawu7orLX7+J|!sDk(+N zMSR@6Mouje*q=Cg(-j=t)Y%oGs~bEqGyxVDm<1%hdr2pi2WneKt6XbywC-?ebp{BN zIvbq>zs+!wg7J}RkK_}>BwruKHIVWj}e_~Gc7MANV~t!OXHQ9Qfx2G&4n(02DpK{w~s z#^LOA5GyDQw>1$VYK7Ul5do3pZ;&y77XKwoCe1S!9&SHn5OWxUQzA8`c)6D>uPRz#7sJA z-%r!SMz_ouyGNGEx&!{Dy9Mxk(c+%pSE);7F2P#HwuTdK(@9{aUZY<_598n0|D;u0 zbfVh0Az?D)sI6rWi!JWuh%D?G*Glo8t?STgA5l&l8J`T^Z}K-Y*u9ax%6@FQ>Q>yt z^|rL>@Zi=AJ9UsD?DIdA2acO}Q-!gX9?GwlhIsEAoq$g%Qn8}BkU$GYV z9Z~T`?Mvv97eoNH!bhqQ*RAOazye3eI7f&uO?=vvHc(O4Th_9ZM57LC)=POylcWMC zfyAANqac552<9tUVN2vWxwDYw3-Dq|SL>PLt++Lj3jAxys`t1KT*;9*AVWmZvNemEbTd{6V|o&z75XTi^li}D$nRg@lcHS3DOqH z-{90IXquK-!dk7#|41H zo73O<5d{UEk*dYr+wNOEQ}-g-UA6b7BPV7cQLL=vDS8T7)oShp`5%U4fAfb254C9l zX3b(?7JWzGVx>zvQoL;PL(#X0@-*f(iUO$XEzCPQ0}cD=)u)sb21#5IT=209cFnB) z-7|eZ#yvfsm_GG9nBQ;ynS7_b?m+|ZxGN(NvVVxyYN)6|5QLO2sXqNPJ_n~xhzx{@ zg&*yjHm683QvRt-Hg@3U>kj-889KqTIigb34-W4PUj0kq&9R-jQft_#%xnIOLw z(mo;}-o+#OCRmVsWCCEd0}&}-{imMHh{?S^tdAPgZcF445@1e{zO*U5;v{BKV^51i zus!m32YdD|QBb7&F8fDZ(EuqQ5mE&3pDzQBHGv5914xIE_Iid}>Ps+?rg9Bvwyjq# z?V?DB?sjf}jTD=OB>qgdA;LjPM2VW=fD|aRO1yH{1Uu#VBr(D7Q|4`+tz4pBwV|iS za0DV|7dl@y+RTo#I|dneysNNxAaqDjWt5LP zg8_TokIa%3+#Div+ z!yd`EELHIg=VXd(U79L`cLPe12&k*$3saMXNCbZ3M848XB~{-q`{^qDBkYgvL-W1P z(hSJpppUi`zI%{stihxnOnb}{$PY84t*M4NUd*6srhv(3tJ z;aPK_nuR`Ur_b{_q2a!OWR2ic226tl>6e`OsPS0YOqeorR?<;pO=uh48mUcV_o|@NQ2TWtmn>*6ti|x*&7VTxy9=xk8#d4V1nPGPjwG1zL?G9Y z9rb9Sx{*4Zd;uipg%}h6b}~gI90wO3?1br?cp9Z888S#69pIS9CA+7@7LvRAXw*UZ z{oSn`Du$>fpp5Vvi&hQZ07o66!4fe(yfNEje=Y9P_!fts1MKq(gl*u6zQ6a|&cr0{JwvR5^$k0HR zWOyFO)`|UM>!h9&eFQ$*k{U=+azs8BH%gTkWXVVXs&2c) z9LZBs!VfLqT)6OiPOmPne5-e0Lky>*Xz*QakkgNR_u-S;vRGLI`ji#XWmk}tbF-pp zv3YntI3)mqBBn{?Bhv1Bu8g{hI|@M3&|qn${vhws1<(8|8qFFl2Wme^CoAAflJhWMSuKr3IHUv~BT$vwn__fVVcm&rA<(kaFd+}Tm z*_}i`6SEm31&sbZ$>9Z~#1K&B{qj*=U~)X(sBK}u>&sS`s<;i8wzjsmF`|Cb7Rz$D zpz@11=}#;>%yz?)j8v-&WEni0usc1tOQ~<<_FY3?6G&1XuP67lyU8FO>uY}kRvl>T zDVA3Pai+X^kr+RVOv%>)meHBciJP)!`XPUx?jA_J!w*qS6B>%=RSflWy`s8wu zR;F*$f$JGxsdbL(Tw+HA=`vpcHZ6fq?9#DEP$!izZo=@L6K2o$`Rcvz0CdDJstgB| z{Z8uWTy6ZOd?oyNBKuNl2CQ4TZpd2h%1?|V$YBC*q^RR8-b<5)+Acf`ho_E*w9ayD zW=sYi*oG03rSV}JaVDWwe_xx4@}~`sO{1R?5fAUf?r$D7H9tat1R{u=0TnfImVFxg zX>MwrT-W`zE{$uhIDE?Nt|QtYZ8x|i%{#>T6yRhhzWpks z5%t6tICl2u(|=KOi=;P>eEn5Ff(=%Ac_sI1f>CXcWlGy-AvWoG>B35n2&>4I#R$k+ zzO=g+#P|SWmn56KlBf%3Kd!J@*I&78wYRM)1UT=wye%4jfVGy=8yKK^m1{<-D`etE z`bQwf>BX?|ZS>&0C!FyCzlsw4LSSCRm1P7XmP95u+E9-I{=Rg!{)Rb>>3FSFxWUvn zjyK8xorqk+*Ly-B%W=_ae}!CO7)X`;?`@dj@V?*qDsQUx$-%gTd=-@G_k$21wY;OQ zP6v{PUWe%5@>C=V@ZqO3-jPk_>fxNJo7=l1SzR|)e$FZCc(U*v=`eEfAFsOhjHIB{Ai6J}dE%Y<-BQ5j|w_ zK5uo#&`xHuWuKSslietW;O2rxkX_fsyC&$~OZ!UJOAU2N>E&ASntNBzS2CF#AiR&i zalFL==&x;H?b;g+H=gaEB&cWX;?&lwfOXG_QZ03H>)01c z9x9)9{=WZ)6^NMDR^1q-l5PU6=e`EXOnx?uYYuqo7Hy^HNiDiA8+WM$PQq->zj;BP zdR7jquuW?*!W&*UA+wKsmc}gF%08rh7ZFx}=DyiMU&bbOSd`a0`Y@_uY4YZA)bRH_ znTm?02w6HYniK?8{8D@AxFHDa9&*S$b@VjZtD@4Fx~HW-V-Eye;LOtq^3>{x$rkL? zH18STojRo99n-xr6hn;}h)FEdYUHZQi_k?E0x#XS8wcI~%J1Pj5>^2-%;?8o1T?kC zozCA$J*E8i>q1j??$SP1#(tZ-z?2n|)SsP^{RCJ#n;6SZVGIzN<;3Zyt4jin(@5`Z zl~$nvI-J|4(TYd0gqY8EuXmryLW8;m?6CxLT_7b{?xAmJtCHX{`}sGA_VO~d2QH)W z&Du;~2l5D*Kvb2Zo%dnkoc1`|IfI zUw6`lG$|=*(fOu2sB_F@dDXL?-VMgiW+2|W&SrA)r|FGupoRim&gY@WB#Y}dwKPFf_BYEW_q2imz*O?iCL!VB&yNjg0JAy*5R4q`E#$d+gX~*{o*1 z)t6XVve6~DJ}g-zxr%RZ^kdz53k{Cg8Z;)8gfZnS({ z?L{3u6U5j=p3zxdK#UpyLJyZF_KVTWOE>gk8kEz(z6NTv=SL^;IG}~5C8atpK!JrU2|6uipXj<)FFXN0?R|UPK$z-lJ>h=B?|5-kgEuW>OHgE>B zJhb{N>xtg>V|EDHfOU~fwdHS9FBgRL4J<{bn+^CeGJj<*TU*d=>Y5^10(AS2KEJgV z2PZ0*l&GoJni+%0r+FA&=kGhK>S!`XD{^O4#ZBr}Gpu3G-hs8n#(5&1fWCYQZHx0T zQJg5~K4X4PZcl%#rPPbO2Y2Lp5b`L|l zJap|hwg+gW5~)l%>;#FDEzX-`f;c%XloIUuw)!tB{V|{d@tCy!mtZEy z7y4f#oj%b29-7IN_?TWc>CQz40G@tQ1Oyp6%u(36g9@()$iwS$eEgrf-)S||N$+NZ z;1wtSD-MhgK(#U!&(?J1}~wmE0Uk#bA&vkT4 zM`HFEv14WiWLvI49acQb0>PuY!H1P#HcdN|4a*F3JG*|=r4MNs$?74apqX^Hm%sCb zl?}6_-zYfE-21wlhC}lCLoNF`K-O7_$Ln#&r9xM}q)NMSPJ^Z$0Kbs{G%atrBn27H zIZpkF#jZ7!uy={*(&~&F4F?1<`{Ju)D}}3*JerirU+n^(=DZSD_n@jr#onP78DL+m zf!gijR(Z_y@A5xisPQ!@B(Q+w*bhGT*>``HK4JQ#IQ+Nxp1+hhSKO?YJ>P(=Pa=>c zZI#^kP<8&UugbZN3!}otu4}|=vccO^w7D`LlsZ?~*l4ycitbZUJ^#TyXJi3Yn^<{vVY?IMnal*_s9?y{7AG z?A~fVXg1;A7Xr@eR=n-EfI93KNDtJXJJ8+%G(t}(J>k55d(1Sa7<_|ioZ7J!0z+C4 zuqI4;B9=H+GQer(x-kLmFUg#=u2=NohZto9fRT#;JRlzZo~j3>y)5O{(8|#3Q70z9 zAgP~)0opU9?^91@AJ=|Z%vhkoZYW;Tfr?EYVB(#k(^e3uQ9Ci)a?8g}8k&_t7x&H*K0` z?s~~2$XoM^H{A0k-!4I@w&`83{Mo@O7As4pJBw&D?l7S_z7s0*xuv5s<((ri5UuMB+fw(aZLjSFJUZxHy!q|~Q;x+c%@koisn($naLr>$*&?{k*Pw|tMR z2t2MjI0AmF^sUSm;Eao!q(nr(1|kQxq?K}d_S5_jZ0;8=_;)$5Z9}TCD1 z`y`u_Q%|S0{!2Txga^beZ_7;VFi%v~+s)N!(d|An`BIhCkvB2^ddYinvpdsMtS@Vd&fF znw(UAd(!)l?A=tFkNBUzEcsq7`?`;R^Qo@>O44HS9EWEe-L?o&A?kStU(xa_qLxyY z)*fB#>GiLU`&?h5JnyPv*6lKPh8B?REmu>Rv)+nUs@HE?T>!EG-jo*#YI3q<5eGXA zCfX|eVv6bl4z`INQv)Ekx8JDVi-$KutW)cn7(=*+t}@z4Dj7&aE9>$AciIR^jnTEK9( zQl7Q^*JiF{^k3@lYsvJj4;dLGztDS8yPNhQn~A$H$X6Dol357*qq+=o@_)JeHP-6f zkCnLrndugrRj>#JzSNgk;6@aDfyTvD{pJ6eqA*2To>OQksqr>%4-NXJ*$aQX7IZO` z>pym)SNq_KdO^wx%Mwh;5Em59_RRf~9 z{M^ajyNm~UWtM$3$>*%fZ`urBt<%XIILxrj=JQp9eWo<$_CV@5fzJwsrbkkJ=^YIn z8ElcCVY)_wyfAn0X)L7S&a$nbx~M3XypMwq!;MOzw#+SZL!cR&iRN=8_!{7JRiy1! zeHH^L@+|q&eoLaNkRF+J-R`kawi9tq1WufTjQ8>9;=l5`<&(I`{7txHzi8;~7cz>~ z(zXYt#gBV$0Vz7r`FP_pTEA5RXUD(f5nXUk33+jD$P1Xf63sq!^M+{|nKz}2$a`EG zrpgQpE~ym83Yk6snqYz7HvEl;hBVzi$+d48eHZP?l6^<>-l^IoJ-Xgv@+IYsl`}5Y^$?fp`q80-sv6&{N$f(h8Oxpx za)}D$zY$(Dq6$AGY2K4hXP#sbL4DE>N-1-yW~ZS5LM&z!!+;>rXr6R4keYaS(1KN8 zPZgJLdF3jgjnZXOO?lm=Kr8o)e^RBdbj--2cZkm!i0ySLIMinec=OOu3*<+<@Kt2Q zEGTE6k4RN7jsqO*Nd4XVpx=qDH4`m6E0iB)Se(A5o50uLI7T+kQR;DJ`pYWZEOqi* zx}fG{rfc2nhuX$)DVynhkC2aukXjyKKtEe^R~OdKQ!PRLmbW3egp!-Z3man3xNYVR zuAwD;l=YfM7r(9kXk4)ie^WlPpI|A7IjEBL*_&X2V!kIpk>3+6AijRfQ0$@0l8J|g z*@0o;1+sk-p58%o-K-H^pM$No=*FT+WKAGge`eGqgp@Ol>}t^ILRyVUa|04%xx@itIGo!Wcw z>Bs)G6J?_}NP|s<9=}?s@Wg%I>CU@J^UWKEDf3;LnB(5?GtVuUVWW$cRf}FMA#>!J5gTuggfv#V_zCK>l?xX#7wA?9ta;<&tjiD+? z>jEF#sBS!{7n(pFT-UfNI}g5B;yVXEwQ*N$1{pgRyg_?y(e2tB@R=hWvJseX9l*j6 zP0c02@2Lb;M#Cm+@u;)KtoJVSX08!R+Q*JS!UXO@F`+}U@!=a=726=#O z)Trw5A5xxcgzBeZTeLC4ztj_N&A!!M@)T;Py@f_SiSFAGJsyZ@s!G4`c3Al+UZ`Ru%Ap< z_s-1qIJDmw0D6Y}U>evPV5;H?Vu)0iCC=FHCq{04XImtGzvDL;GIXFg+$dmbDZ6K4 zz0=LZisCG_kVAg3R-X-)9KtyUNN^hZsV(2Xa5|Z4FdfwN2&iQu6z4w2|IQc>nviDkh zuC?Zx^O?__tJ_y4QP)~-iT8GITk?rg&lETPvj33sUjuZ<8NKc7b>7u9rf_eobYIL! z*6^g>8LQKA@q76EtQvo2ZRHayWVP7wzcPfo7~OwFenPDXefaMZgj=TUa}yT($F-o@ zcS#w2vr;Ph5J#srJn2u?Z!x4(QDk)ahVsOmuW!y)T=*QLcGEmKxXOz8MsOsgjxssJ zK+>v7O(*R+WN7*8TmePkRApg-)?*!|Zp5tbZQoYxryYCnl#**rnJus(VUYocRH98MAe;kT*)NS)>7_!$F7uJ2I%vyVgxN*80R6vT*g+=Q%9k9a~rpX>4g0q4bS@=yqzd-szBSTAj^CmrT2V1TaR>qbh-sc(=($Y)(TDz|q zVnkxRUM#jhU3w}Qc{p3Qiv2eRnc;|5N6$s-<^NFmisDW^X>a7 zC&q06EIorcdaDH5C`~X0yKe!xCXDB>d2Zg*|F;rxRM#9zR&LlzQO%oW#QLX*&v<1M@5Flb3 zslNLW+M1fW733BiX~nMuN>5&hV^3NaP#v4KtRcL4O2~m_{mvdQs0bv!e$2vKmRT|4 zuic*iRkElUTLHhY3+wX8eZ8nZpu(F9YhB@`mVP&aDh6sE-U}LF|3-Yp`uXQfz`0pY zw#EYh|2i5gSmU9rz2V4jTd&klG^2=X`w1j|;@q^ma8EEBoN;ZhyYm2Oe)rN@{Pv=| zw0JWl{R)%y6I?CSK&JODr=^76#_0Y@ihn=tUGPVc2>lTm?YR_C^j%Z+T%_gCJ6HeP zB<5V+f}`Dz+X=#dXavZF;pBsX}hfhaZ=1WH*Jv4b%u6=wo5{#HYY(>$7YeWXTlgVu}BQvvi_>! zIPpV18~)U)(5)IePi3t=@H=^VcP0&%{lr5JNL@~z^*TLe!A`KpwXb;Z#xD8o&0gKT zB~XHMuQt^z@e9yS^Gl?62!0D)2j~+}o9S1Z!reMcIu-9%&j3M&C353*lS1R~Ry7&2 zVOxwNg1q;Fb7NMNGitYcBTw+mBD_eyi91N}z#lxdkW#=qKU zwp&(xW*0g%{yde}+0Q$$1jAb*WclQd^Ayt8RqXnr0&>{@iyi*OD(eVASwpq;&~g?? zc)}bd-OywHGsWMtnMOB&^rUP(=mkzvAn^|x8X3Y3q{;!*QpJMU+(yT`d4>ilF<2L( zeUMoGNyU2j`=gmLD11dek?9#gTT!Zu@lGnuH%2o!=Ds%L)Zp4;`1GsJ`xVna|g z5NHUB7K6K~cniyesDI9E4~o!kzPw+O;(1qY=3<`Q*91_JOhI-}*jmLlccBdOwzC#} zP1n?%G9fsl+Ol?p(Zu7HlBY;~(y+5!LdkuT8*ottQV{h!*vC6&#sjk8uNp*^ekv9+ zS_e?igtH%t+#xEWw*rmmB&{u3kZz=n4fB|qjc~?|fj!G~75$sx%61dQM(>YmqQ~3- z0>%cS{O;l82;@i7h{sPxt|5D#A%AYlnx(uOSp1cMvLgO9Lwkb_YwQw$ zN{z}v|UEH05;FUT?}_sjmr(d`l4SRIUTmP2jr(Nr+&~BGbTTQv{7Jw$k01&HcT2G2E?)IfE+fl#pH&H1pkY-vJ4gd-)6oCSD zA@7cxA2)DYHh=5}bN1r3rd9yDOY@K6cdh^fFBc=d@wg0k?#Tb*y#;UqgfE3HO5C%u zknNF^jPi*Rjq&Zixz(4eLC+1;*oZz#?mUCO>OV z>XNX!-v5;RQliGXLG)8#Zy;F{v3u;;UgEl|T{k9&n$=mxlm!XuY+Qc%&8hFi@S7cy ztHx(?6;Sh6{dP`!T@hPUkLAt-vr3vI6!Kwan;!47s3Of-q&1U{iVJl15*O`~fDjrk zmCaZK35jJrXI5GQtR!;Q1|0rby%q{+?%`JJryc=V>vKmz6-88=%(XOP>(}=s$O!EI zTa41mQfDyE@XGF1u9DdgY=DQBdLTjtP&tM;ft?&AabUow1d1o*7}WBNIk!fGzMBk5 zVJeW_t8?}?HO#nQkx{G#ro!TFZS-jNpF8h+oCgZ;gT&i)M z6jl84X0H9{beP{mZ7Tzyke_9br7n=zLO-(yE|noR<)oQAoQQG#_Ea4_lBwh zm2FLLjXYzJ_reZd zRo6MGU>4@SuvQ^eH2QqN9H@IaIodl=avn1r3@EaX2-<(%>U!m~5`6OBVSPhq(?B1p z2ktF{0lp2JTr~gF)=PX_-1OAl$9pGi8?QmPkE~-EvGpsiXOKv;^`I*|r*2pwG1FNkVTJ3fvy|h;&c>1up3tz%F0xl2 z$-kKnpgVW-{EJBC@@}ai*L`I*wS&dX+i@3NZ@Y>_vj&kHL2%cM$UVV@QA;E=v0sZv z8&ihPl+KSF+uQM9{83a?UXH?-;rv^Sk;v+ou0C<@6(wK2TF1jDZCt~_2Q+Y>Wdjwh z_4Y>gz0bSdQ^|L6;#H%0VTEqsMRool9)6uj}4#i*Yhwd|=$eY#2o2>@SYHp8RkhvGl zvJ*QxvtR7?o|UdJA+&%2Qq+sTopND1H!br)&DVVv72QMm{Er}HBJocDLfPw(Y70vr zn`L3~$QvfBdOiW{VjJNNUmHtpcrrM$6{sPq zW%9M*etV%#uuy_ty33$sya|Q-;4ptM^mR3QyP`n5NnLlxZ-c@6JBM+9%mBNaBzA2x zwYfDXB13xsGkjn$`l8ngm1-~T)k?l%0>C z3+|e!C&i^vWrTcv6DcULyHOs1oY0^vXJ>zZk4)81;;z@g6xfoa$G@AE zxq+6{cCgPr)*BGFe$3!gL@N9YFWhrqstj^O2lbemy^L$Dbf+lm6?>7uZF9{?r z8q8~^G@4siDD@hosf%4#xb|fAO{(0;9n5(Sq?_ye=VD~tBY6jUuqzEvHcpI+m7@1l z;^OxCh3sB_<)YrCn8Hx0c=|U+BD*gd_?+)(9DwxwwYBRqi}-jryDQwyE}48$Ql6hoO0}7TFTE4+MgtJHtPiTZ|8*kMNfuQe~@MIi#4Q<~CX% z2A@SWbf;Ad-8#dJ9lHfDBapD1b^rAOm6)Fsj(JvFd?meVtKSdwaX+}CWfNsrTVuam z^g5uGm@3?#d>Fx4u02$BEcwW)y-Db~2}1JqMHG1?LIK@LnXf{?-f)H>;CLrd>F{VM z7!nX8)`X6=my@>1esJ8uPt9HMd*lm+at4VI-Th;D&tZiXT$Z>fZ4#}S^17nn8F+Ri zMtmiFpCi09Ohn1I5RyA8G1?ig*|gRTM_5k3;X&TWsl-&llHHt`A!*;6{SGjoRd4Gr z`)$T@QBU)=ZjW^gKIV!XQ%u?z$1J^#%9!M_^xzf4NByuG| zsFwRjuJrsTp%$2S0K~t19p)o46K72MsN}TmC1K@grE6J?;CQ22vWU+mPSwx0j@}`{0*T7c%Kn#;?@UH zk6*uBjSDR5*(GUx$W}+3sq*ezWFo!xdGUmCv~zu-&YNV1;!}-AC|?t?}Kn@dEn+1pCyS zsAfD;mkvdigo6x*6LY_RafDP7Cblaga$U1KJKpPQ9XaCl(0C&-@K}OnCSEV@E!6W@`#}eKMUrZ- zOrTHlNtIryP}qDh{OV^n4qw-9Q%X-XbJf@6gYxo7EkWanZ-^2WXJ->ZWBj~SXL#Y36Y#4q=dMSd6y3#T179sA&bws7Zn84d^CsJcBM#H59l zHsl5!2((KS*u}c=&+O_wP#xkK+G1k_RfBD`48jK?aW3^GIsO*Q(yx**+;;A9WxhA} z=C}JdVN$_E zlCO4B$S#-fLj@~;s--Vx`uI6J_FdDImsvwsi?%iRM=Xj8pS_nErS&e%e=2ab&f0(D zu{L88TH{5oP$8q&BJ3*S_-crZM;4y0zCU zKztRLtH%$TvW~w6_h}~brMltJ1qyO?HCA~Hob}sZ_neQq$0A+_JXtID zBxvJouH-5mZ#Ww}E1SYBci(Ah$7lILi2d}laV9Wj_}p7p##P;}cx_%A!{sdSScDMa z0a#}GwfJoPyn)4e=%tQy9tdH~IqTIb__SyLF)=UcK{&0OyN)RY*=aRpMXYPas7%#K zf6QO2MH5&pMKDYw1t1! zao0}8iWFzj#Oc_!XvW?9GaZZHCHW>TWc1{v=04+{QL4+g4?POHUWC^8nkUCK#FbQ2 zRvJZh>B(=Ey83n}s46twW3hq&j`?Lzby_PcRv?38;Bz+mJ`p=)rl0G` zwQKi1roFbUv`3Z#j>n|cD%m4g_Cz|YU<_uAO!BjP7Vxxtb}%3I4(ILUA+VN2XnvTF zEn;weEyZfrf$`{Pau2$UFk|tO;6+Zm_ zb2XL7Y(Bs=`7O=wHRK%NGnge~sjJ>*q>_S9Ja8~f4OmWHrk!q^xF`D6-UIVN8NsV! z*8UM5o&Eb@br`xKUmdX{e?d<)bG71VgFp>e*JV4}_!=B3;z<`sjuJANo4swIOE}E0 zP)_&q0kkrr-~O@#&^wgM z6NtUx#YV5sd`%r*0igP7a&WvHyWrk|8UDoPJ$6?{Q7|m)*3CRmNv{!Qv)M?E0VR*9 z42Zlph?LV9|7F_@Y?dT;ox$73@Cm#3)Rn_l!HlDzYEm~Pw?hhBbHA4{@17r067fpC z;b21EOe5S{ITH;365k}LrqUcn@q6zkb9_k6IZ%70^WK+ihiIe6DQx~~nO64;$%bs$ z^VBlIl07Vfv414_k^g5au!SxppS>SX6>EP-_}f-@)kTUMU{RrE;F)sPudsV3=8e&T zw#g^ovM6UdvPas!c|38mAxP-)app-!e%P>EsQ#aI(}s(lEFm5&<(xAL|4oOBq|?Bj zSA^_*8X0%dsOQ)ZF_M359;2YGpo7Js6KOD7ooV>g`(}O z^6NP9G_y=^jW8#w!=HmV(! z67_7TIIz@rc{(WH+{8mVqm*=yGOh)Itrr87Qy`=IEmE();{M5^oWhcE?8%V!CT9u8 zNAJ#uIl(J`;KXDs8ZVotFMhzBXKN##hPDD7wB)U5A;MzTvh*x{ z_=f8IRwPEb4vofZY44=KU|RW#{1?UzM}bc9`Rw(XK3HKwio9;s*459tEVRyglMxWw z!`kgnVFF^tq~5VdDa0>%Nd{SSt}mucKHdu#E72y~OB9)UZ|3_GiZChj2P+b2za@!j8WCavxD4;f#hHF1miyICq3 z;!X?j9jaz~*|WLoDADl0yFpyZ%ydbG2b`0*_btS*1#i4|?NJcnjAh^s+08D0--T}k zT9d54<7vrfq&j^DnP^2JTvrYsY`og|sVhX|1! z<7-?3;v~QFkUs)Eec*75&3H*-hD7wt@SM@V#_2Z~tF&z7o?vv-3g$j4dkz@br%Q*{ zHK7%=C-S)JvSCQXje&@ZTr#_(bv+_Gp~3_3r5@g`mhDI7As6rzbT?4svfBjgw~%E+ zC-3zh-n_m$z{E3)N)2}O4Y>8%yUL)wz*)b__}q>1kj-(KHxhtAFUQztyK8)W5B%b* z%fEk7@YLvh@r}SkDQw zAf1XVq?7&dZo`1Cc7r^WzmT8?2=R-^4`yIdJ!wDn=(fYy*IzPzxB&r{;hEphz|M2_ z@~sz*G(o@vm{Z!3;x-=9y;5=OB%Wq^F8Jp1`Kg+_z{I+4sE*Hap#A#B{*wfjdhSE$ z>38zu?D*SBlI8zLkMyMcy?zHynYyY1t|oR{tyYCA_fBOZT=v}R*-7fGF>_qS;rUb63`F)PEwsAxA@8Dlj?yX0Xd2K9}jc*}zD*lhktDZt&zy z_)4a4zkyM7FjuD6S6En8N2RRj9g91)em!)G+*80c+g-NsxuR^?uo*v_1|1#fP2@gn zuI)>ZsO3fP2P3I!5@{9>pHW$bUzhMs6pYuXZ6Z504{Uq%kAa_3;;6}$GdCO}$>~Mn z1vSQ@p1xvxL(| z3%=VStN%z+|FSsIzpNMi{4qUyyK1J@#k_dOic_*)cTHqFE_YDQy(KBEwDWCvs(HNNqu%bS@a%Z##*I_wqTYtn?o$mXd1B-j0|*t;J>O+F zDlM+y_jf|aMmx`;$X~rc%Si6I#U0}Zoik&)0<#MY-oA$2QQ38{2|m{I+PJ=2gL%&F zKwowHZr_<$wOu}oJ9!SiK2!oRJIn0se}@o{ zBq#!30j;|usxp{kEwYu$3%-Bt02S<$B?>Aj0t~BJ-DQi)CVt!#_JgtXs}>}t)ndcv zL!Y2~>a9Uue|0Wv=j~yovE~E+qnb0Mn)|-m}z3*?_eaaguVA`V|Tl3SSwHs!Ek|irYyEE47r8A`5 z85jD_5DJvne-;~l=9VrhRY|;CCd54u+Yx7VL=GT|zohDEzO@B8SolI&R5m~?M~Ol7 zW%eJb7KClKQt<~OICj4fbvt_t73Gb+pf>e9DgyTQGh=31j=kJgCy#GDZV_XjuYJ3& z_G#HQ-Z&~>h7Dm__Q-V(r3d7Lr6p2Dq)@&(&(1JI7`>R=ZatyE#JB% zF#$`;^geu#1S{-mKP2*ATT&AWB&4!=7klCNQVTx*!{S_LJ@w{11KIN65l~6MSMd{UuIKHWucda)J9X3W=rGhEF z?_Wr}oH%k@Wg4(|TP!&o~bn)0aoR+Ci ztMGWh84G>T5We#^x~Eq6)J@3_Nt2sNeg|g*zWMjbn~Gw~A)A$XtChganp~-r6EnGc zPp?sM=AN(dBL-JQZM%fr%=7iFu+n%MJ8jt1%?5j84FQtCP#*ZMC-{*SWTm^0FH97% zt1H%Ft;4Z}9Jhnde>9A=biL(sfcrL?xGc`7S}Wg^YJTI#eC_fKJ7$+hPMH5qc7TDc z+JI47Jcr~}g_eeFfpHk-*Ia#K-6*kFK*f)K);1@L150AYuX-afJNa^UhE5gA{m5u} zCORi=0Cb;j;TPz;q~Be6Eu1BN1?b;CHYt>8m#tDMi-5?}_fll%Gu@o4w+m}ljJCLbk?f96Z*WL?#*lbo6$ zm!yep3l?>pfNR{MUWe4!97P_6rY34x&&l>1NKlV#Nu^82DUsy*qg~FHIM~g&7ry8o zJsggaxKfQ*ta5fAaxYLvN>lG~H@&e^&WrCmlLXF_paG^_P*VhSDhEHCFoAV@ zov?pUFup3(%*glx?)^dPf!zw~sf0D)}FRPIaj3`MhH_Y<#&ACTo^~q=# zg=f6v8R6!@U@=QcW5 zaGdC6f&_Na;rro27Ke+!-j6+p#DcKQ)`*bS8)bpiMNcS_zELD0PC^6nVcxD#h5n1H z{N@&LSx?w^l`-us4V{xQfB%4{sMoA@p`DV}vMbOb=weS!-=0{FVw97MciR37JCCP% zEo=ndqoLMvm^MZbZ9Ux}%GP8mo>43vgl~p!86w?~+*)-5ZGMU%2YRIhxcA2?U6{p- zaN4d*V$>IsJ1Mo|9*34l&UhD^^i%Khp=FdB4^oM#wQ|1${63PlJ0DMO#=1x(8LR1_qaFwCu3GHzn~` z6a=_z@oJ5mp^=o+yL`&&WYwsbZ`#qDbm1+ z50)8ET#k#J=BI{`3lz2pM35!7WU|!Nv z7jUNDQx(J+cKig~&v@cHe?hT0IPUEZn0NVpB9(YccOnoE&ezOrKHW<6e&^70vI56D zDc#nxdOiuP{LA=~1KMP=XfVrFrg{s3SXSP~pAA5Hwa&@HxYR`kyEIiQQ;pG-BTbBk zy5UBjmp(OnA-12YiXq|uD4_=Hx3_wY4jX-kBp%i=*|N@7a*8L@yCq>OU%z&Dkln`= z7=G$tB*#);o&-k4R<@7{9a$PBrjX-d@KggC_R`*urHqN+rvc^&{-rmU#0b7XXRIXjJq(6s91_Mj!FWW ze%R}ws?`^({}_eAVJp{v8&($Z4$91lDw9}n;Fs5MGqaC60$z-$Hd@JsVJikTG^t~1 z=zv~eGM2?W%OJ*q{%Dy8>2|}{Vl$p{BYXcrVMn5qxKdr7%w(OHy^+fpZ2t=cAQ1RR zQk~P+@_8@k#87Sed1Eck-?rfFJ8?DaaCd^V`Pw$XJP;FLPoX^CO-)SERv4=gvumU@DW~-e-If{JnTC`4mI%~de9x#diNIh9|eKpZWqEWr`mPkH8 zC~(X3%y+9|EpiOmuureb)>aK?l!M(F>`ZoFWi}~V$0m5suI87Ft7XH$EX$fOxt{Ul zG>T<9e9w}e_yBv?CAjNGH;B2uNr?ODK_!fa9m++4v$qNu zQ6)Hb0{cy5s*2B6P2(pG?4psD=q{3iTV>)qaO`Y>&>R$^wT>beSliM|>if%T-(dh9 zwx>4^cEcXd-8Z?J(jdw`_6VbAtUtn)Xn|*3{Hk9iY|z4G=eHJxIFWDv&Z81T)vd~X z5TgQ0kKr4SX3tdZ?O+KOYAdG*RjcMhMdRdPA!mnjh9ez*JXHV|qYyD2Xm>=ub$Tf6 z_TZajjTo&h-{dhtZdAwQNQB@)(EFqJTK|@DM{9emQ75HF3M0biH#uO# z7Kwf&hOETpDjFgIr-61=uW%6u4h}pM1L(C)qco$jK|W?-ab&k`755&}g}XrdSt7p2 zkI!vLd;QM_k7T!U=kPhGUzwp~|r>lj!+ z3CiIHqSAf%!gt7gCtm4Cim#dXY4XMr*g3veIsiNOqQO}sY*Z)!m0J-u+$Uls5&z*6 z((QQ#;974d-6xt2hOEj3G5(YPOUD=Pyajj~4Ud?@+oFV8I2Y%`{Vwa z^CNEK=?<}LO-4R-w{rk1Jc1_H$p-?#&8T9KNdAq8T|L4=CF;FS9g}`U_mboCiw{c@ z3t>W8j7GDAx52T1aU7^NI@|bTzg=F!z*>tVH|&;Uv#+qsPFeoiC|82*S5G4)>%=562ewXqbB*P-%@MDFO}Ff{O+B!p z(VvkD7hjd~M!!3-*@Pr^Zlrj+<<-@)&G6Yg70;&U5<37}&DIJ0%M3diC5OHKX4 znvMhJp7P`xf!DJg-zt0AL?N^Dqn2^k$7*|BYJr^?#ox_|j@w!-6myD>VfO7z-EUH& zc(AsKI41jG&C(RU?by9|N?~c+ClUTY%`Ge%1{y_cUKpXY3Y2 ztQn zDa^OhnAirHZHPa>#K~FMTJ)1ur5`o`)Xx7Xcu2nd zk;%%?dux~lhWqs|cd^hyaJfrCqt@QpI;lrBni%A><)*!Su%SJif#rdL^QP*60JCPH zQ&#X%{AP@hXzg+UqIr-xtk17&pWNVi@a`wv29%;TK>5Kz4N(m{_{|s0 zAo(SelYz{$HO;|^bf&P+;(yi?xxvPwTmwQu8o#}%U3!rE#NgCxRBMv9rtB}L^18&J z`CTV&Wh(>H)RgLs=Hx7E%&4~az5FyM(~rI#aG;{58|sU66TZ+x%Y@dp&kdXF3>?tD zF2zteU0*n(r!LIstY3K1N_w@31wkWTo-A-;+FFnvwg(naCZPpJxU`E;_JS!?=U2f< zykZ^TXt*kNsi9l%SYW|-@19ULtqm@GvL!8fV=J6WtP@{=BD7=;#H}ZH=d} zM2-E51FBY9G+(KFbK-Rw;i$RQG6meaYjBGXAfnv>^l@mBotA0;)lab?e2EqBbz1N* zpyhEljd@=#|SuZ357CjTC zklh`N-0^>JPShQgU36c)P8)rP+%W@g>dMzum>nKt{#O_2KsPN;u%(Bj>^H#U^i|%> z2@-l1n(=RCYzBW5&IXJOIU$&g^EY(G&_l=V@o!552&f1J{ZBjM!T|cYIz8xJpL&A1 z0d~TAdr&xA6O!tv`oDwk#3N{@slv_RrydMKKGV=HvSk6F8rj6h@a*wj?tcMa_#Ltj zTv^9aO}<%GswXGM`bC z!m#V}G$+T#{&N9<9CgBJEw{oFufC7cSa;efM-!0ch{d zz>nXZDyKU+Mp}0JPapr!|5zgrYf06!!SGKdE-|iA0-Zl4O`AR{^Tyopdg%_?+U=8X zI9qKAXYPo0oc#OE_BquxR@q8Hlav3vlY3=AIu!~KJ%A54CEpI=DAXN*cqj0hzw+xg z7Vwh~=c4ASiKnjv4jJ%4*6W`F4E4wHU$O#_)B9zRQ|%pICkGc_X8HepQ@;@WMZ@0+ zKJZ9pfNr?_4>L2fXPYkrBV$?c$yv+W=TCyOKY#v=F1NM&oan{X>0?NLGIicS0Kxuq z8b!_mz@{fZ1ejcJZkoZr1OloNcr^oZy8KT#t=HXOQvL!E6i&T;vqMH5^lJzX0TQ$9 z<-34J1XKm^G6lqcjtn4|q6WU*#`^PdtNarB8-U$<>Yo5B_*_as+Tv`I|6ceLmWxs& z8Z#s8XvV@iPkV5DcEF@X=PfLH=I@T&jiQwl0nQ>BF>N|L&Q3CYKpa%+oKlmN+Rp-r zRIR$r-g60zKq=b1y0)1rjFQf1CudR}W?9Pe`xq;Fwa=3+=kI>%ZGyZuzs<~&HroSe zH4L)|#+}YYsQ=IR06N{h&~MZ1G1rpsb}Z!YkkkaR{~M&nDwoJRIffZ<>2NY)^<-|v z*Z^3PmZ%stm0p2OR%AJhJZ&MaXVMA$No7<$adgbIG~mpy={t8qVR&f92pSjO-{p;B zC295zm=ZBt{+Q{Irj9I+0}dM&|9RdGQq$+X`lc}6|AO-BxeRnXd8&yEyQ(Ad4S=!S zSR?jX!sLVf2IkM=dpY{Q$d;9=!JF7-GG}D=iVdwCJw5LMB<8XDx}8>L{#{*s)dj#o z28QNOPmUzNvYHSu(BZ64Er)wV)8@egqpgw(q9SK;3ow{heQ9-f_h&EzQngxAiN65f zM84T8rq73BdoD6fzZp~zj>F)<4HMdn%lRQyj zkVLZ3 zfP_lWRuPo|{K?;)Kcnf)tB%k|Ubdf}y6^dTt-2={m;Tm?;9g;W>8};}nUuMpJ4wRl z_sh(A|x(m(MX=4J0^VUM#XOB}22Emd@~bDRW#PTWS~&RczUXFQ{L9;^Qy0>?S=aN6YM zs5-G}okXK*4zS#p;ZRGZWyB?ZZ)ky&i@u@d*fyA|!^I~mzmxJ-=VCoCVdx1a+W0Z^wye^@CG0e&D2@ zqm(XarerDIWnq<~NzsTgVpM`Sw8GP$EPI5sA>9SmDtwJy|JJ%-xY>)9*Z5Nb;*mhf z$xpZ|dt$T~qiD5c%(I4K8%H_v2KX}eh{UekUbU;$o}<)Mqx~jHUe)DKebD$zV*`bX zy(U?7J-xGItP_`=GXaFrmrv+B#6=wD?*;hb7G$e7v<8!mOO-x}PQowy-2$ZRX40pjJ{+mcKsW|HamGbeN}ww~4B;4oaM8qKE=G8#2K zfGz*Mi{QNi{=?6dyI}e0xLc^IM-w@fiNuZ%h)J100YYPdgAH}F#~qO6%7CJ5%9NCS zK9$Mrl{fE?EVa}na1_{Vu4WlzYRPEV7fXBCTkAdiBz@)ME(r>Iejt#E^O4B-IRP4qvot=9ZO`9;Kxh0^j;5AEP@lEk)Ny%ov}H~!LzbJ8Z4zy9s2EX+?Pre**HrMYA|_DbkWTo^Sa-Cq}c z@d}`5`(+sU0VdYXa{wFB?60ZxKQt2nZQ0*8@=Ym3=$EZPG6!shw~>*Njq_5!jf<9F zZ2@}~V49y;6Q@qB`d@!f?D$`AzJ&f-%Yd2F{;Qv-PJIS$^XmoJ?Z4juhWPI{|9hMN znQRiqyjqcOg7pz=_bbb3qABgY8MsDA$l%`$Ogcl{Yu@ab@h73a!3<;tN|(-~cxHFr zo-jUG!ae&%w5|y)tB0Sb2i`miS7(+#wUA+c5r{Nj?e$tQdMv_baj88SAZLiMeLFW2 z@XyT|?hbXpYi<31|I!PRCa;cDlE3-So#%cHcP+2fz4izio)ji%!)sT8wW5KK8 z!K;y*`W+JObc(G2{_YJ^AcE0i1M-Wl-!=IGZggyOfRC^E&1S?Oc8A+Qhr|j4o@404 z@TKDOuS3MbDrX0NF#aYZ9&kjie_91ZI;abt`bm%9qok=DF`O+4a|xk_J~o# zT>uQHZr@`(?2-DAYqgUr)D{TA28!%#NaSGY{bC4mt4U4~X%y%FSjae=t}j3rJ85#C ztiB(m8 zgn0~d!-^@;cuMa9bP|y~RML(`<2w0Q`GdKj#MY%>)N~5~+ZCd{gw~^L4vV=GXWX8V(&g>k)1Tqa*9#ik57c#!gqICjD?FNUg5-EWSMb?u)OH8;Es?OE=D)a#26{ zQmJshmd-TZS%$c328r;lMM@w}66Y#s%9cY+iGw7ud95a74xE3zKK;|P)Hjs0)3;(q z@aQv|{19Zl{|`CoT_TW&cL&n67#p^X^sNG)Kgv~~N*i#2>#KwVaX(@cWM8K-syM_A zqgHeqV7=~8W>xdevLJSmVXq%TS1TL&*7UW#v%t0H-(I&A-QQgCTj*Q8E1(-;o<-=0 z9@HJsn#Tq@9W`*KEqN8tJ?5VCYRv>3H}B)aEIWVVZUy9kS6n2Wj~?8e5^Q>?h=+jI zC7nRmV*eYOiK`2I?q}tP>h%bKO%&kfIp$A-JHaeiBEQi%+e%GqCrN()43*|o3$KeB z+)CK2?$v2TOkQKGVA?F#Y-$$i>%+FdzDb+R(RC8IKzxJ^tecBOV(m9)|!@L`9ozd=(7MYpUG^;M>P(qz(I%KFkcd zC_g=t5=vXTG;jt#c6*_unbb7UG9NqE9ca1m8h@=yU)9!SImJfrfx%QAzXXs2+e|21 zv*?p4iITjB9%b~0IiwOo{u(L04oYZ4!xz($ClR!KjSxFc6g$rmJi->#@xG@gt1lK4 zulS;L$=6A_W&2i?#^zo zE~JSMQ}7c+gp-mg_NB(~t0gV5A*&S~4YKCI2>oWCh>cWJNe>GD)Cmd)Jde#1rwsjs zc+yOhw4f*0{4ru%5{9-?sr;Mta(Cup(xpM@2LI}72xP<2s=FtDep3!G_`mkOMo04@ zX_ERwQtvQZjP#kz4py(Jw@DBArbX^mINBzTL|gUrD%ExrM8cO!(lXavZl_&)KB1TL zqCZH-!phQ@_fs1q^gzBz^5A8Dr^r2ZUT{~6eVk{Rk~BcfNh$51%!S++15SL$OQ=QO z{E009;K;Qq?83kN(cMyCv04Ua^|eG)z44%DLdY^BP42Zuk@>r_s0`PLJn|x*w+OF? zFG>`8is*j1!Epjp2M)?znX}%7{#)}t-Zwd37(5-k|IhNG0dwyCGAD;jlZ5mpfl7fy z>G$GJ?-?628{nmRz*2vZ9#LNO^z@A;NS z0c=_0_-a0&L8iljE=9Ia9S`!b@O7YEYLli#)+n~TI;{pr8+LmT5g{^%t+Ws?RMIdi zm05;R!aM7e%TCxFGK2I*`5!Kwq|Tlsh?yRL<%&ogofO%#WJ$omg2R{KTVUPhorsF0 ziA9b@wV`}}pbbq@@}fI6IA<}s&h~bGPY6Zj*j5OPm^#+9`4$0!rjXSve{kxj?+#%%3W%<5SOSBj!0VPsmYM~h@){|G>W z>SRpKV`s~t@eW-D<}OQCB_}c|U~JF=$io>bMXynW%4nt+?&u zuQfKPRP-`*Lp*)+%L(u4_)+b_{N((G(5&1|2~8il!K65rc)sc6nGr_YnOVKG1~Ff? z&hCeok;LH$+CsuHkk<MG3QbStG9|$bh2rU4s3UJ`R0aO!nt+;qi z!8=i+1GB>q?yF0jATZqk0;5rz=L~}I*oP~eo&oq%ebYK-x5rjLgKhR}p_UD0_kXRa zkunu1Q%Zef&I{LDI6dUHt9v`=Qgzd+J9F49z|+&%HKe}hU>~MnyfE5l?JK<1R@0x}Gc|#Z84i+C_q)|z~zhgKH-z-4nE7&XOC?Mq+?1>X%PW3vI z%Fh0K9D{xJ>rf@_V7WR!Tb2Iwou-~p->s&VPG|s>ZMg>OS)I#PZ${)_=H@v-;+>(= zv&2!%uC8skqLOOOs#}GG1&$<*3J-zq8j6AvIJd@{ar7d_bxknLFu!RX6LQU9q0i3z zo21G1N;y60N1dXJpII1K^~S9BHKbQSIo;313Ug2g3^S(9YV1A~LF+J!`+_dWs-@I2S2pU`KMDq_zVyjhWW@ z8K0~eqzm6K%lGrnTdjO{H_a7R=LfrT+8ZOZ4fHmrkqu^BbH%XM_rFDcGVN0E*P+MS0d| z%Sn(C2!(K#+-E<>0L1YiDTb6NH-wDhFhmuf$v;^UxmhyYd!sw*%Z)7IVrK5lwE$|u zIw>eA6*ED-4Dea7xm!E6`@PHuzy=d7J@ttn?eHPX56NK|bl_SlRdpQWJnU+us#s+V zWgD#0&JghCV=l|q%*s1i4Nd=7d*2z>)YkoZuZpOsC|r$FL_|P9h|)U>A|({1_b63* z?+`AcAiWdmMM4WjdQU{ER4D;zK{|vI>5!0Q4(fg1|IB}$dFI2E4>O`^2Ck0@~*v#o&%vF~#Q`I1R8>$4zVMse=^a1LXYlGXD$8gv>P=BpauOrgY;1f)% zT#9-5lhB3fHcE4ASFcSgTG~~;puV4WKkO+}c;x(ko>6$_)t$me?b0R>Y_=fLjfu6@ zo|nh|n)W~lwolT5LGv0J8;G}e?S-u2ia|?55U?;d_G@)j?w;#(HH+pjk{d6Ohg`?% zTUuKqjPNmTOyI*649l}g~7B!dFKt&efs7-YT zGJzd#RORXU*8h3bq-~V9lzYz@Z|9Iy+d4!b{0|0D>Z=_crHzi58qb6C6d(By^Ni`+_9fdbzQM37r&O78AM^DsA9 zgcVSP9n=5O?b^@wGD7}rDS*ZV&gILP(zm4W73Tcq2Bz@~`ji%x&ZmSoPxIT$TSvz- zH!W@+*nUH!Z;t^Sf`D!VrMZn(9&6Zidh(*@!DaA_vlv&O2eS`9DQ~l1CEph2j&ycy zWCx&2grku2{j)Y93J1ddlOx>J{zp-DTI};BzF%TDy9i$KyS4e+cQGY*#I!@QRrZVk zCY#{W+=AZm36t#t_Jqk+`)?+jFtB^x#;#iuqoCL*LQ(Mh1a^HZed?VdfMvDW?ud^i z0G#n9+OCxv9t0OQ=~;yK(zNGJI|wTJ6;A$!1Dc5Rze*Rasq$$MhE%Qt{a|WUM3Ry$ zAU*GZHP~uk90SL3GO&E}Fp8>5lHSesgq(iPe^leB_Q3x`s+(88_?n8fvSp>8wTn{p zO6pAArlZV{(lInycJ=6erp4;3jY&;bb)#yRx=zz49`iS?R1!+-2xSwD;n%S#L+H}p zonVLwXAfmJzclW8uD5m6B+^iQdfUCn8Td_-hN@a4crG#kWzCwtAE7ia*RrFKfDiIL zld_tE55Vtm+cZ7Wy=at_S#fPX!IFdtU`9rlHWY5XmT;hyNB+F z<0MT)0H5Y&vhVQBu__@;9sRS&aTf@QyG^@j5Ptd|V?TY5`E5-CE@nvUA5}V+{fnZP zwEQ}OymO9ARRN%lO~tNArI@JQ9nKPQ_>_blOt)2sbrXOzOWLK9p?ZGP~t*o`OD?WY`1)# z)U8uQ4+Q_g8(qP6063+Z8?v+g>&!75uYTZ~UIp0krbGsW(HE#l)CdP#MF^sm>5E^0 zl^W({mW=V$qV(WQrf|TIwy8mp6*o#(PWK6?B1AAXC|+| zZaw8~K&UG$q=SZm8U(430o6r-3Zk#=P7pGqhWnV$yU$k<)c8LHjbP%*bamR7T3HM- zpwmU0Q#GB=0PhcG9Ma zy5xa;D*Zlvarr9K!7>ntW`YC2h%XUf&w7?Y0Jii5{;UXKB>O~l=#3xVV0t&DGwQKC zvCxpT6qLSN+lCmJssbB3P^-u!b75?h0kSYhjIDbNY@01#hsUhxlgY7Bl39O)-VWd! zP!tC%K#bG-d4la_oUSslTgh)Rk>z$BG;Zzqa>Dzxgb0n2xeDWkIs41)dM<}w38D0y zSf|$I-0VD;ele>VDPl;WO3xBuO}f<5u+cKZ=J}=lu{7t7HpeNal9Y_a^fVm#WoKq( zKhXBVSuB7U=&3|MEZP3fETq%`|3Rw z)O0a!n)qsQROw=Uttz3zdpUi0vRR%qhHBT!rE^D`hL zQ9+-WrnmdreN`<3YS?vvFKmijBf`4!eexuY{)4*d-dfLR1OdebyqpLzm3h4r7mcg$ z|Q^66gu%|2P4X_4o1MCQLKHb>`f?ru^OfbZGSZ^ zp$3m?`iJu|rX9B6{@AkG8)XaxP8^hjgoy(LczW`IPz z17u@_+fuDcRl5z61LRCZr1BR>Hq>TtcAHh;nOL&$pWYu)2qnu+X$G-Fb5b*-D)dS1 z2aBiLcVJnwPM*`v$(d!k!$jgR%kZd~ zQ+(Il)enJknrWa7kcs$Mgqe^t*I8Wi0YPez+^g_>p*$f#j1R~U129FQfo($p$5H~J zsC4wJln+#kca@_62kt^^V1EOVAN@HiEWY$`^&h4s6@H~O9w$~}dRTcD5as!b=BRe{ z41S>FELnUz?IVt)LcZRa#RAa1?10p7zk17gk(HnhG@^(HU$^kiD zZY2C8j0bB<&Z@Ip1wcqL8#7~sZe*yi& zM>8JY?yHmn)0#q`&|b>hfdJW}FF^Dn;6Q>>9E^19fzR<`?15744nhc`&ctIJ(ZMaY zDKZV~s`fFM8^=BCMn2{4tju&UdL&|8o0hQjAku8F*IH&}bjISS0hfi;X;{=d$$B7S z=BeU-RYO|K&~yP6kj;;B`DGGZUVJR}cdEX6RHjljV6EiSbbD6PB&OSM?LDFTwRNJ> zd}Ak5K14xQ{Jdv`N`wX0LN$HjaU5SakUs~s zH)J`*xtN2m{l)6&uE^E(zGGazn?FK*at0KuR-Ev#P~WC@2H?(1IqkI>A%p(P7fa5R zkCFM3jmF%wOqpA*V@kfqygtb@OeLoS^wZPXGtPZ_zrDl4*S)E5DNh4GXIh0SUO}yUD;U~?7 zgVS6OqH~=`G5Y=_cNVz+J$i&tlmFUjzA}k#P&U<1fcF6MmRcamOGX? zNHN394EqbpY}#+4Q<7TZd0{0ikhs0^uk(-8=@(wo zwVZLE$4*~c&I~BMuOsD2tg)yVju?o9WGW;+Nu-!g^759O&EiytoN=O3UcpI*|dD#R?X; zHGG9%C+@pR2^0KZaHzm;b)#McN7nKwL`pVS z*7o1vx*=5GUk1re8pfn*CGV2tG;&{@LtyTTZNqT1`?;ga{n)1fT!?j9>^P6K>~I^h|zNx$-Mk* zLC=wvc%*v^EPgO$MSp&+_|dax>nCJYyjjx=Ng-`rNZIPIPD3q;`RO&guW7s*uLV9S zrv`5E3*36WCAezu*yxa3dG~tqt(oI~PvIGt@V)bYQ=b1E1^NOQrB9r++oPJ!?%KKP z2v}7sU89%&r@jaLnh*$>kopdXil~IE%zln0n|P3T`T^J`Ej z&^BjBe(T)bFe%`!?=%shh)OH^P9jgu;#O z^lCXIZiLeJ)lO}HOVs5{c>ufyq_e3#?9;CsZgqMEhA4J^5kG%U{MX2;RU=YwqUdGc znbHCSc+vB^(`L`Vxsvm1yeE)pxfM~?H~02|5C0Y!)Ht{zu6B;~|9|!Wv1J;seG%wR zcg5|OETqbcm^g`av^B6iEG-V9#=ekJJ2@ViZ=IWN3+X{zE-LpX1P&T zTpX>+=eWjvlV5a1yEZ2ecnuD#eWnwDp%8}iR5xE`u`LL}?sveH1_m1d9w1_Jj&`&S zC=8#l`uf%e;OqZQK{rbT#^s^JTr;p8A$>uP$)@V(SCO)(4P--Cfilm5zPu(^8Ri@H2ClyYtSS!s{9f z$UnzS3w*jKKXBj*3(++0K^G-MGd0mrU2UuPv6J8!uMNyZx+BOIjJGi~sN#$?& z`6|5a^$&=M2<$=i;w9*SkE?c;yh8C;MqgQ3j$TN7$GojSJ@lNa536A9gO9zlh4!lY z8*8mID_!o5#2gY>qGA2oS{fa{7ADwchhXz2g{k88UouPb3XFfr>bRN?AKq-#)qt4d z!oJVBWL64W(7|l3ZdQQ^Q-fVs1*+#q?zk*mP@4bZQsyVgGb%B5#u%fpGkW;`WJ_Bk ze!w29&g>fxIAV3Ty7y~6o7mmqHLv6xe`JjEiU|9Of}6&TZYZ{yE|z|qpDw36pBw{U za_=oK7^$u6og8sE5E8wfaF|zeZvSZ%iAdoZ-=%q4K%d&7J}yZ!dT2Pg(ekvjf;&1l zd7;v~T3x^|Qn`k)G;ejK=F4MCiRnt6vlcnkW8?!LPk)={y5W?WX0z zK8^6+`2IS~*V!*~2Txp^`Gz4HjBQRkeo=8R)On@!p5}DutKwoQoh-EwQ+3adu)ekT zWw!QZ%$mcA=Iuc8;KrIO# zqYPgiDtr&Ew7n+E2t%O42=#ljqgLx%}r$(}tm~Bd_ zO8XwvPObzR4PmCxc9=&5wsC;vamop7h#l?y<9mg0|x{*x7ZS z8X^H5g?5$b=qu5b^y@vpT{FUBm=i2e^A+SPYnd`8_+}Iw!%r?$F4MY=JLcr^+tx@Q zg@V3fEnLTRbvYcQ3alSun_g^!iuW7M^G04|_;gOmqJ{e#mWq_dvzM72^r2xq`nlNY zuOkjnlf_uNHLP}EK2LL{F4YPs?JqSuhBYG9rCDwz0WSP};c&5Cx~q%FgU*Dc!Xasl zXgdrZ!6qP5C8~U?e|eoWNnj|V@#7me;tqf|f2bzC^TAGmV@TcI1K*k}hB>*LTrUu0 zg4Gj_EK9TqmxoophDqvmOjxsU7~Ca(A%u; zBk8lvo;4fVwnc`FiPY`Z>^eoVNa32PTd7v(lC4dKuY@c6Yr4n^=$z%W3d#KluTV(O zEOOFdo3dsxML!BlW?v-DpP|!zg`8J>zUx#DO$SRP9!-=YHsj!8YKZXik0bflVZmAd zIu9BG@Uv+evIG4zQ;pmO(xBVgDZC{=q)|2K{n31jqAGPEwG6|0#g}FkWe4%t=&Ywh zu$Iq4q|SM3#r0ZbK&##y>`bc}&C>NV*Do=ld7Sm16^MNyf&5f}-xTxmsSotybY_$YdHzblp{7nLU{^}TCi+o9(bCc$6#8~Ms- zqDslpsrAs?t{hg>J5!|2GT05CIH~V{N1q3N0)OwjO^{uf))^K){bwWWtn}!DN&C`@ zA&^0OxoK!N$bhg6(B&sNEJ7~$DSMyngxNdNsX}jap$*XY-TA3>M!&4Dl(cE&+kLqR z_CJ@`o5*YBnpD3gKDw!(X8>$BW;W#VUDe80PYOSU$5)qYApD1%M=(BYHQjstoudR` zF}iPg?LR*DB2e{plm4En z^coF!VRal#%9ih9l`K-c=8@^j+awh9w)W^q+6$d(jUB~5D|2(uf1k(kebWKv{TkH< z%I-lDI>(@gB*pRom=c8w8=M7o;ukNsF5aV=kgy-tJp-2@X!2>ND3K_;kNh3#SXiwm z@);CZj&2=PU29Qv3<&y^`K@VUvjOoZFChmr;;LP-GoDsA?&w4kWD~ZU^>cQohYnq(Ed_kBCu8m|@ zRj4hoHp2wThp?Ojh2D#nEVkEHW^k0NR#^*pNp3M+l=!xJ%Yw(fKYkIb@BJs2wSmNX zg%)EzAkjlK*+P$Xt>E_l5beQF@aE>LhS>wWjY+aSKPLAGzomk(%$#)AIL;vZ2}8cJ zfYRPgo}f+f5OcDe{GYtMnh5_Q=L+LnlRo)({U9O!vkWu4*IM-E#NN>vhi)2Lf8t_0 z!@vx@^vGhT=5neXyD$9wd3W;QK8?%tW}|D+{-yrlnj$I$%2$^v=QI&R{m-Uh!7-s5 z0!wH~l9Fq-&_8DPiu&;dW3&5>tVxKsO$o_MLYK-5wCWcPP4|mFReD6{qbRr5lRO#? zz^Vh6mb?=fYcQXcirC!!dHo_o949?6+DI8T&w)>;mvf_13=2lgx)W(GM+>V#wFasRkG6#Mlmh(QX= zKrHu$W+dC{w1gAmYu4NeTF;62D39Yo#`0ttTf%p<6OZenfD4got1W0QcXYxO%F0R~ zYvQZsoBza7@aQ0fP59oM;-v*uGgtI&v)BJt18wxs0hE#MeOCIVQ-41TlI^ll!wQyK zm!u@;OM+XQR?KOCY4Qhb&Co{|*?~cV1#4dXws+aUrCWgeBgbI);P+4d3ch`o7FcZ7 zNZuR2_2mlaN)R=m9AY}IU;I7TF3r1^lV>TK(fpS18w9Gh?~hr{g>uWOf__g=aUJx= zFg*0bh5vfIr>5vXIzJs!-qCj-E(Z$Tpqx1ApT5|&?fohr_>{HsyP$xL@Vk!S9qa^B z|MACz9O%!#e?6!FfG>k{x0KO^assn=(5s~@=*XQ{29wv{BDlw2oA!v1_H z`7&L=YluH6NRVw#M;QnU{N4K${r8QGkNr0#t&>|Gr(AXBL7ezumLGs~sjjztF{7&Y z`|ZAWb%{g37`H12!MB4si(SvE8(2EUSU#CjwgxW8j0e|4KOV71zd72C>N@<-gE7{? zS|H$tg&BK1>Mkq2e`=H1Mmk;dAotFA2Gs|!!T7M5z`(%Bjr{}L|1>{gNz!Xo|C;H> znF&Je(#E8)<77?!ECxix&II@lml`s9l1iUYE^Avjytz8%IjT@TNslx3EHDJjxESG=Rp z0E?*}=g*a#{IN@O*TlElV>zm1kauec?pf7DbbdUOqs$H& z-4)C8o^4e$Q#N%5ijlhPZNv*jn+H=d`z(m2YTo=D|v)w8(5F|4!BL zQ>f>L1>^V70AGQ8O+3mONFD;q)XA{uuSqUw09@u#;DTIGa?*8 zdYyvWXv~;tDw|8(aKw8-)UvAO8GV8dsvPReXIlZ2y(AzLs#iR9&9_NH=MF3V_E3}B z(5sD)5Nkmxk-DJ`VX!Lw(&ziuhYX3Bi=a22Zj%O+8{=vR#fguTW6T~niY7RpjeMmw z4liq&wJi_^Gb=Mq!lHSBqb9>YT#<))XfOYuXV%)LR-{V1WV^0Nn8|D+H!PPZ6)c+J ztSG=M8+gBPc|e-jD|0;1w}g<3paD01XPi02rNqfsGwMiIn=iSKnT&pm0flQ#!H8(- zgZ1_P{Wy#~O~1HpQ&U1>W#z3HDV_18SE*2c3}TFY$iMS3=q%~PD#U*V&cXix!g+Mu ze<{5a|E{BZ!i);%c5mO!O67>p_TFnum{Fv3)06>BOP%|9wv=Ahzba8*lX&7|-@}wK?j0B!8V*`` z1=r@HeI8%SmAc$8wU)|E1M$~jN(xTnpV7p7*_Ajhb^m?Lxlljd+)Scu6z-?uNc-*F zi-*e#)C*qM%l6(%5J3nZXfPnu$6YEt@(41O?Pa`L_yHak#JFS8 zE<^}c`YT56z2K8DiLvp<*z%zJoHtEP{VoAA2&tsq1? zc&)5UTOA&s@en}gr^*rJZ*)Rw8(KRvw9<=^qF6{wA8gbEhVWH6hr(87?ZgWQDxY{30PY#3c&K~ zx;n6q&BS!+=dhVAq;$MODQ<_2Ev#oPECgRyQ2N9*{vL)d90Fqw3pi>n28zz)88kZE zZevbUO{{&x@9EN%+kA8>ir~BTB$14vklG0oQ=eaZe|Q>+!`(|3+L2M0u;0xDsY5j@ zR#4-fQ+~tIn8)NbnE;N&Ba$m@W+=fCD-W&8G2i>_Jl`Q*^D0byIhayM_)G(SmY?ZO z$arX`MpMhJU}z*7u)8vdM(uG{&=l^Y$$MpOcdqfz%)|C|YmEY0j-cevAdcf>D;Y_k zDx?|~$h2>dCZpA5+y_KZTbN;Wmf_Kysqk2w=%ttGf`IYrvTFqb2r=|?60N>?yGdjQ z6HtH-Y`=uAntO*4qoG4|pTL`5!7}S}Tm=B)iYqe2W$Jr}pbJ#A-z$|@|@?9>~qaK9kH#>=2qhHK2WQy}*Q2a1mO0ZIA>>b?f92xuu@igw*0yH_Jdi#g_ zdc8w!laM`q;Ro|-rhJ$Bmd_ec_sA^?J+a`<(R~&<$#xj#0eS`q_Q;=}Y`_&+SQqGv zWeDOSEGGsn(7F)&C+lJ@Oh_nde;{SqW1yGDfvN7|W0oZngKQ?Oumiy8DI{EwpymgZ zu;)m@a}5MUDPXgHxnS&^Vql4htk0ll4wAjZy?&ELd1%Mw9AQpw360Nz!5h87m+o??rap1!Op=`vT7WLSfYK zm4EC^FX{E;R==hE^oP`D5cXhLhtYMHe!ZI!?5KF?(vMfwoe16J z*6WrAcQ-(KrQbGGnKlnaEAF6nPsHQu z`=pI@;pF*J5iGaM#Q+_BtI_W73F}`odzv(s*AxWsA{pW&jmS)!G(iBoS?6`M(voJ= zT>GQ;xoIWp7L(RS45U2UfgTW1s~aW~4pZOo#)Oh&0T|ojL3HU6>i)z>KB6nVw1j1^ z^)~<75nGgzsqq8bi4ka~@1_`_2TgpvUf$OzZ+O)*J?scSW#aop*3)YBrd33Ivx0A; z!{^Ub@t7khVTeN5^hAsKPuSCGa?BKAD1f?q6xIEMNCU(px^AgUxsOCN2GFyti&lSB zV*pOSxiHNhjksAmot%uBNtI6hPQc?r^*NN^V$_P$+|@snXaowi&2#L&o35Ji^;X@; zg?=>>dIgl0Is6EerMP;OU9U*2a0!yLX*#k^D6`2gIp^a}D{=q4sMQ*u$g}y-3wSB- z$Aw}RJBVvovo+9@4~ZIe*N+G^x<{GM$&GZh-_(DvFW~HG*p;1UC=(H@k+F4b)2YXD zt#VFU(n`9rW|*ERpT0w-&d_{pnY5yS5bQ@#_<*?Wu?5`t-WE_dV1nD(%Ldtq6h8y= z#3-9|J92de5FB-}?qjMrj`R)olj9BN0pAp|83Ep=VYT$o!j)?T#Ji5(dlarj@0k2x zR$vgzzlXT!i=hScb_KWr8fL=yVnyf90A;98grpZf_xT8yL$sXPY6tuEKgdHtzfS9O z%Sk<0KEU-njuvo0{8A;?aUiYE>xvh*BI=Qy!dwzny|Y}~jU(?<=nd%C!EchZc0t@N zfiz7-I%pN2x}@j>=>-}zDHPM7@>ccu9+66^(>wQ@40_>5^=Shy9J9vX+QqIq7}4td z&cc_QAU6*#=zUvIvz4~*F6nhHQM>gRwz$z{_$(h2rbaRj{zo$;Ti4f39XyLIhyNvm zlCec~N&(Ijf6SXDQl1i8a6Yy3;hUFe!tK7h2B|c;pHRT*wCq2OWSH;Z8o2$A6X#Kg z0i)M*ikI>5Q)J++D0;_wl!Idfm5g1aK>v*~OgTq}*PCo%bP2*~s;-=#vI}V?09IWX zIJ?Ho-T3<*)VB|g3mJR-OdLS1H#+ekw;?7%zaY$x>gUJTAscS-4Q zrYC}8;tH!<$uG8Mk7D1s40zikHHN$xi!2O?7G7M|c}r&?r`7Z|ZN&C~kTP4f`?m_? z3?Od9&zdduh)+8(!7Z)AyU+*^T$U{%EGt39_O+UVYBkw#MmEzB zn-Y=7oPAM|b=HU{zUYyl^K<`{ z0|f0_T1#|qm7R6%K-Z;iXc4OsX6PmOQbR^HkNrEqhf+439F=JFwd3#lEF`TYzlQ}} zTGjXy-7T(0IMLnlUdQEyfZ7;zV2Y{R?}YnPh%-s&k36U9hVh3R19E}In(EB#Z~f(u zGA+o=p2x(Tx0(|M7fzEG;ZTpkWUHv?{Ui{mKNC3OJIks@px&Yi-uhXY(xiswf)}Wn z#`R3pQMJWD|K-~27FJSDSnp+<0j_tPC<@%TDI@J+%lHhwyUTbkCc^z3K3icF?%tJBv*+P0 zj{gZl?L~=Hbf}CyO+)O1Or9PUeRJuJa88(7@x!F)D?LN9CfzDKDTEjs^YYCW7+vC8 z;_@d5)t$HT2GPGFS<9yisNRpTy3!=Fa3vV9_skA1K@#~NZuPu1P~6J9)Hs1psn)13 zGI|=Xtg@u&ZmuUo_Z-*CzKGSL(in0ka2pELq(4Novi)T(-N6TP{8>`j4g+IC}vu$nB zqN7#&0r}}Mr>ArKZ%`kOE0xu*9~sBFY-^qDeIYiOMF*4qWKYXx$>8LZG+)1+MRy z5Kz=DDZ*Shdh2)_u(%H?GM>}Bv=pa(uJh32x%y}sVQsM@>^1;mrHFPGq;j)PBjS9| z=aTMd(5tQZwXz#hgprP5Nqe=b+VxTi$Am-+`bAt2)C9Ep9NS*zP;bI)B*ks-?@a@u z?8s!eMpdj;_A^kXDl6whIXQrRbo~bxKRBQn(Hq8WH|A~N>MK;bAUoojZdIP+Bbl6n zPZSPj@zxVUq%evr5UmpDA5PJHcP?YvhkP?=$9|QzS~dw!eNvD;@;J8EjvAU33)q5T7xcXBI>LCy-?Y){Zs1>?=8S3D4t9NyM z-H*6YYax5VF8OFF)v{R#^I#?bTI7&iZ$DY1 zcmH~HCoOxJRWcjc!XV6^tKCAjdldxrdtNgQti*(~Xu`eEz@nPJNtf`j;UltKmGRHnIc+m&ptFEEkrQ1=)ng{{Vw1IwV>9osTn-@;2qpofy`?2f zZ?0iuV`B~3p4~jvRPr_)t+>Ivh9+({N_cK2h{Y;CnSM6vy_l#wvYNZLK3S`7Sv37d z&Kj=}<6@kiWZ&4>Xkj{#AyrYLMdqI#9kr};y1lC59nG(0_&QrPOYN>yR}eL=u&{84 zX?e6XK@t5ca;#<|hXb|EJ9EWh%BI@a%FuikJHW<`z-@xQx6;ZaBm?`LG~@$7cXxMp z+%6H1rkrf z=IYs?SX#b<6@6l4$*$&9<3`OwCmbgrR5KkzpbF+T=r*es%0k&WI_l!3W9SSxCg?dQ=DarXY{nQrO8N_Jx{ z6nyvBpCh-lNUvWBa#OPzez`p{xCv}blxocc%AXi(co%8|%MXS<9tzT;(3=keOTu~D zM)&2vkHjaxWKxa&Bp}I%Z%G&Q?XNH?OHM!IL1V0#3S{_hd#ryO939Ooof#i5>>oLx z(J1*Kt^T}ieoJr+w7a`IGB(y@Gd(>`Fr>|`akRI`l=z-$9Rl38kT~WAK<*aNvL?WW z125P>68e71v8d7|ELd^=L$tKG*N-e`2b&6VCrf}`n(y5m@$I4G-9M%DxkH@4fBJKS z`mzwvVi7065U-3DytMPUyEGbyK50q`^emDng$1@ZQ^-U>n-n@+QAGcBa{2Gr`|(67 zU`dn+3#YURh?8r7M3^T}{ccv&8T`K_Be6m`O=;b8C sourceUserMD = source.getUserMetadata(); Assertions.assertThat(sourceUserMD.get(owner)) @@ -258,9 +258,11 @@ protected void assertAttributeHasValue(final String key, /** * Context accessor with XAttrs returned for the {@link #MAGIC_PATH} * path. + * It also implements the Header Processing Callbacks, + * so those calls are mapped to the same data. */ private static final class XAttrContextAccessor - implements ContextAccessors { + implements ContextAccessors, HeaderProcessing.HeaderProcessingCallbacks { private final Map userHeaders = new HashMap<>(); @@ -297,7 +299,7 @@ public Path makeQualified(final Path path) { @Override public AuditSpan getActiveAuditSpan() { - return NoopSpan.INSTANCE; + return NOOP_SPAN; } @Override diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java index 5cc94fae7d159..c1744832e7fae 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java @@ -29,7 +29,6 @@ import com.amazonaws.services.s3.model.DeleteObjectsRequest; import com.amazonaws.services.s3.model.MultiObjectDeleteException; import org.apache.hadoop.thirdparty.com.google.common.collect.Lists; -import com.amazonaws.services.s3.model.ObjectMetadata; import org.assertj.core.api.Assertions; import org.junit.Before; import org.junit.Test; @@ -41,9 +40,9 @@ import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.api.RequestFactory; import org.apache.hadoop.fs.s3a.audit.AuditSpan; -import org.apache.hadoop.fs.s3a.audit.NoopSpan; import org.apache.hadoop.fs.s3a.test.OperationTrackingStore; +import static org.apache.hadoop.fs.s3a.audit.AuditIntegration.NOOP_SPAN; import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.ACCESS_DENIED; import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.removeUndeletedPaths; import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.toPathList; @@ -260,15 +259,9 @@ public Path makeQualified(final Path path) { return path; } - @Override - public ObjectMetadata getObjectMetadata(final String key) - throws IOException { - return new ObjectMetadata(); - } - @Override public AuditSpan getActiveAuditSpan() { - return NoopSpan.INSTANCE; + return NOOP_SPAN; } @Override diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AMkdirCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AMkdirCost.java index 1cb6dcf518d30..639e1dddbae74 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AMkdirCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AMkdirCost.java @@ -79,7 +79,6 @@ public void testMkdirOverDir() throws Throwable { // full probe on dest plus list only on parent. with(OBJECT_METADATA_REQUESTS, 0), - with(OBJECT_LIST_REQUEST, FILESTATUS_DIR_PROBE_L)); } From b2a256f93805b4040ce3ab454b11749efaed2ce9 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 22 Mar 2021 13:13:18 +0000 Subject: [PATCH 7/8] HADOOP-17511 restore xfer manager integration Change-Id: Ifd1b217861c646a9c17b56614bd1f6da570e3d15 --- .../org/apache/hadoop/fs/s3a/S3AFileSystem.java | 6 +++--- .../apache/hadoop/fs/s3a/audit/AuditManager.java | 11 +++++++++++ .../fs/s3a/audit/impl/ActiveAuditManager.java | 14 ++++++++++++++ .../hadoop/fs/s3a/audit/impl/NoopAuditManager.java | 10 ++++++++++ 4 files changed, 38 insertions(+), 3 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index cf56eb4c6a4ae..deac0c42bd699 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -79,8 +79,6 @@ import com.amazonaws.services.s3.transfer.model.CopyResult; import com.amazonaws.services.s3.transfer.model.UploadResult; import com.amazonaws.event.ProgressListener; - -import org.apache.hadoop.fs.statistics.IOStatisticsLogging; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.slf4j.Logger; @@ -133,6 +131,7 @@ 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.IOStatisticsLogging; import org.apache.hadoop.fs.statistics.IOStatisticsSource; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.Text; @@ -4105,7 +4104,8 @@ private CopyResult copyFile(String srcKey, String dstKey, long size, getRequestFactory().newCopyObjectRequest(srcKey, dstKey, srcom); changeTracker.maybeApplyConstraint(copyObjectRequest); incrementStatistic(OBJECT_COPY_REQUESTS); - Copy copy = transfers.copy(copyObjectRequest); + Copy copy = transfers.copy(copyObjectRequest, + getAuditManager().createStateChangeListener()); copy.addProgressListener(progressListener); CopyOutcome copyOutcome = CopyOutcome.waitForCopy(copy); InterruptedException interruptedException = diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditManager.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditManager.java index 21e28ca8ca0a3..b0e60f57de178 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditManager.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditManager.java @@ -21,6 +21,7 @@ import java.util.List; import com.amazonaws.handlers.RequestHandler2; +import com.amazonaws.services.s3.transfer.internal.TransferStateChangeListener; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.service.Service; @@ -52,4 +53,14 @@ public interface AuditManager extends Service, AuditSpanSource, */ List createRequestHandlers(); + /** + * Return a transfer state change callback which + * fixes the active span context to be that in which + * the state change listener was created. + * This can be used to audit the creation of the multipart + * upload initiation request which the transfer manager + * makes when a file to be copied is split up. + * @return a state change listener. + */ + TransferStateChangeListener createStateChangeListener(); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/ActiveAuditManager.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/ActiveAuditManager.java index 5e64212b65bb9..5c81041192bcd 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/ActiveAuditManager.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/ActiveAuditManager.java @@ -28,6 +28,8 @@ import com.amazonaws.Response; import com.amazonaws.SdkBaseException; import com.amazonaws.handlers.RequestHandler2; +import com.amazonaws.services.s3.transfer.Transfer; +import com.amazonaws.services.s3.transfer.internal.TransferStateChangeListener; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -209,6 +211,18 @@ public List createRequestHandlers() { return requestHandlers; } + @Override + public TransferStateChangeListener createStateChangeListener() { + final AuditSpan span = getActiveThreadSpan(); + return new TransferStateChangeListener() { + @Override + public void transferStateChanged(final Transfer transfer, + final Transfer.TransferState state) { + setActiveThreadSpan(span); + } + }; + } + /** * Attach a reference to the active thread span, then * invoke the same callback on that active thread. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopAuditManager.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopAuditManager.java index d146626d89b36..7a49b200f9490 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopAuditManager.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopAuditManager.java @@ -24,6 +24,8 @@ import java.util.List; import com.amazonaws.handlers.RequestHandler2; +import com.amazonaws.services.s3.transfer.Transfer; +import com.amazonaws.services.s3.transfer.internal.TransferStateChangeListener; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.s3a.audit.AuditManager; @@ -61,4 +63,12 @@ public List createRequestHandlers() { return new ArrayList<>(); } + @Override + public TransferStateChangeListener createStateChangeListener() { + return new TransferStateChangeListener() { + public void transferStateChanged(final Transfer transfer, + final Transfer.TransferState state) { + } + }; + } } From 4c41298bb77cbb3a32f78c8be68abafd27002690 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 23 Mar 2021 14:20:54 +0000 Subject: [PATCH 8/8] HADOOP-17511. Audit Spans to have unique IDs. + an interface ActiveThreadSpanSource to give current thread span. This is to allow integrations with the AWS SDK &C to query the active span for an FS and immediately be able to identify span by ID, for logging etc. Adding a unique ID to all audit spans and supporting active thread span (with activate/deactivate) causes major changes in the no-op code, as suddenly there's a lot more state there across manager, auditor and span. will be testable though. Change-Id: Id4dddcab7b735bd01f3d3b8a8236ff6da8f97671 --- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 64 +++++++++---------- .../fs/s3a/audit/ActiveThreadSpanSource.java | 31 +++++++++ .../hadoop/fs/s3a/audit/AuditEntryPoint.java | 35 ++++++++++ .../hadoop/fs/s3a/audit/AuditIntegration.java | 26 -------- .../hadoop/fs/s3a/audit/AuditManager.java | 12 ++-- .../apache/hadoop/fs/s3a/audit/AuditSpan.java | 20 +++--- .../s3a/audit/impl/AbstractAuditSpanImpl.java | 20 +++++- .../audit/impl/AbstractOperationAuditor.java | 15 +++++ .../fs/s3a/audit/impl/ActiveAuditManager.java | 19 +++--- .../fs/s3a/audit/impl/LoggingAuditor.java | 51 +++++---------- .../fs/s3a/audit/impl/NoopAuditManager.java | 60 ++++++++++++++++- .../hadoop/fs/s3a/audit/impl/NoopAuditor.java | 40 ++++++++---- .../hadoop/fs/s3a/audit/impl/NoopSpan.java | 52 +++++++++++++-- .../hadoop/fs/s3a/AbstractS3ATestBase.java | 14 +--- .../hadoop/fs/s3a/MockS3AFileSystem.java | 8 +-- .../fs/s3a/TestS3ABlockOutputStream.java | 6 +- .../fs/s3a/audit/AuditTestIntegration.java} | 34 +++++----- .../fs/s3a/impl/TestHeaderProcessing.java | 4 +- .../s3a/impl/TestPartialDeleteFailures.java | 4 +- 19 files changed, 335 insertions(+), 180 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/ActiveThreadSpanSource.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditEntryPoint.java rename hadoop-tools/hadoop-aws/src/{main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopSpanSource.java => test/java/org/apache/hadoop/fs/s3a/audit/AuditTestIntegration.java} (55%) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index deac0c42bd699..7479e61b7737f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -97,6 +97,7 @@ import org.apache.hadoop.fs.Globber; import org.apache.hadoop.fs.impl.OpenFileParameters; import org.apache.hadoop.fs.s3a.api.RequestFactory; +import org.apache.hadoop.fs.s3a.audit.AuditEntryPoint; import org.apache.hadoop.fs.s3a.audit.AuditIntegration; import org.apache.hadoop.fs.s3a.audit.AuditManager; import org.apache.hadoop.fs.s3a.audit.AuditSpan; @@ -211,7 +212,6 @@ import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.fixBucketRegion; import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.logDnsLookup; import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.dirMetaToStatuses; -import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.logIOStatisticsAtDebug; 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; @@ -1095,7 +1095,7 @@ public String getBucketLocation() throws IOException { * @throws IOException on any failure. */ @VisibleForTesting - @AuditSpan.AuditEntryPoint + @AuditEntryPoint @Retries.RetryTranslated public String getBucketLocation(String bucketName) throws IOException { final String region = trackDurationAndSpan( @@ -1333,7 +1333,7 @@ public FSDataInputStream open(Path f, int bufferSize) * @throws IOException IO failure. */ @Retries.RetryTranslated - @AuditSpan.AuditEntryPoint + @AuditEntryPoint private FSDataInputStream open( final Path file, final Optional options, @@ -1513,7 +1513,7 @@ private S3ObjectAttributes createObjectAttributes( * @see #setPermission(Path, FsPermission) */ @Override - @AuditSpan.AuditEntryPoint + @AuditEntryPoint @SuppressWarnings("IOResourceOpenedButNotSafelyClosed") public FSDataOutputStream create(Path f, FsPermission permission, boolean overwrite, int bufferSize, short replication, long blockSize, @@ -1637,7 +1637,7 @@ public WriteOperationHelper createWriteOperationHelper(AuditSpan auditSpan) { * is not a directory. */ @Override - @AuditSpan.AuditEntryPoint + @AuditEntryPoint public FSDataOutputStream createNonRecursive(Path p, FsPermission permission, EnumSet flags, @@ -1707,7 +1707,7 @@ public FSDataOutputStream append(Path f, int bufferSize, * @throws IOException on IO failure * @return true if rename is successful */ - @AuditSpan.AuditEntryPoint + @AuditEntryPoint @Retries.RetryTranslated public boolean rename(Path src, Path dst) throws IOException { try { @@ -2066,7 +2066,7 @@ public boolean allowAuthoritative(final Path p) { * @throws IOException IO and object access problems. */ @VisibleForTesting - @AuditSpan.AuditEntryPoint + @AuditEntryPoint @InterfaceAudience.LimitedPrivate("utilities") @Retries.RetryTranslated @InterfaceStability.Evolving @@ -3005,7 +3005,7 @@ private DeleteObjectsResult removeKeys( */ @Override @Retries.RetryTranslated - @AuditSpan.AuditEntryPoint + @AuditEntryPoint public boolean delete(Path f, boolean recursive) throws IOException { checkNotClosed(); final Path path = qualify(f); @@ -3082,7 +3082,7 @@ protected void maybeCreateFakeParentDirectory(Path path) * */ @Override - @AuditSpan.AuditEntryPoint + @AuditEntryPoint public RemoteIterator listStatusIterator(Path p) throws FileNotFoundException, IOException { Path path = qualify(p); @@ -3102,7 +3102,7 @@ public RemoteIterator listStatusIterator(Path p) * IOException see specific implementation */ @Override - @AuditSpan.AuditEntryPoint + @AuditEntryPoint public FileStatus[] listStatus(Path f) throws FileNotFoundException, IOException { Path path = qualify(f); @@ -3264,7 +3264,7 @@ public UserGroupInformation getOwner() { * @throws IOException other IO problems */ @Override - @AuditSpan.AuditEntryPoint + @AuditEntryPoint public boolean mkdirs(Path p, FsPermission permission) throws IOException, FileAlreadyExistsException { Path path = qualify(p); @@ -3313,7 +3313,7 @@ public void createFakeDirectory(final String key) */ @Override @Retries.RetryTranslated - @AuditSpan.AuditEntryPoint + @AuditEntryPoint public ContentSummary getContentSummary(final Path f) throws IOException { final Path path = qualify(f); return trackDurationAndSpan( @@ -3360,7 +3360,7 @@ public RemoteIterator listStatusIterator(final Path path) * @throws IOException on other problems. */ @Override - @AuditSpan.AuditEntryPoint + @AuditEntryPoint @Retries.RetryTranslated public FileStatus getFileStatus(final Path f) throws IOException { Path path = qualify(f); @@ -3711,7 +3711,7 @@ private boolean s3Exists(final Path path, final Set probes) * @throws AmazonClientException failure in the AWS SDK */ @Override - @AuditSpan.AuditEntryPoint + @AuditEntryPoint public void copyFromLocalFile(boolean delSrc, boolean overwrite, Path src, Path dst) throws IOException { checkNotClosed(); @@ -3959,7 +3959,7 @@ public String getCanonicalServiceName() { * @throws IOException IO failure */ @Override - @AuditSpan.AuditEntryPoint + @AuditEntryPoint public Token getDelegationToken(String renewer) throws IOException { checkNotClosed(); @@ -4459,7 +4459,7 @@ public FileStatus[] globStatus(Path pathPattern) throws IOException { * {@inheritDoc} */ @Override - @AuditSpan.AuditEntryPoint + @AuditEntryPoint public FileStatus[] globStatus( final Path pathPattern, final PathFilter filter) @@ -4479,7 +4479,7 @@ public FileStatus[] globStatus( * {@inheritDoc} */ @Override - @AuditSpan.AuditEntryPoint + @AuditEntryPoint public boolean exists(Path f) throws IOException { final Path path = qualify(f); try { @@ -4499,7 +4499,7 @@ public boolean exists(Path f) throws IOException { * {@inheritDoc} */ @Override - @AuditSpan.AuditEntryPoint + @AuditEntryPoint @SuppressWarnings("deprecation") public boolean isDirectory(Path f) throws IOException { final Path path = qualify(f); @@ -4521,7 +4521,7 @@ public boolean isDirectory(Path f) throws IOException { * {@inheritDoc} */ @Override - @AuditSpan.AuditEntryPoint + @AuditEntryPoint @SuppressWarnings("deprecation") public boolean isFile(Path f) throws IOException { final Path path = qualify(f); @@ -4556,7 +4556,7 @@ public boolean isFile(Path f) throws IOException { */ @Override @Retries.RetryTranslated - @AuditSpan.AuditEntryPoint + @AuditEntryPoint public EtagChecksum getFileChecksum(Path f, final long length) throws IOException { Preconditions.checkArgument(length >= 0); @@ -4587,7 +4587,7 @@ private HeaderProcessing getHeaderProcessing() { } @Override - @AuditSpan.AuditEntryPoint + @AuditEntryPoint public byte[] getXAttr(final Path path, final String name) throws IOException { checkNotClosed(); @@ -4599,7 +4599,7 @@ public byte[] getXAttr(final Path path, final String name) } @Override - @AuditSpan.AuditEntryPoint + @AuditEntryPoint public Map getXAttrs(final Path path) throws IOException { checkNotClosed(); try (AuditSpan span = createSpan( @@ -4610,7 +4610,7 @@ public Map getXAttrs(final Path path) throws IOException { } @Override - @AuditSpan.AuditEntryPoint + @AuditEntryPoint public Map getXAttrs(final Path path, final List names) throws IOException { @@ -4623,7 +4623,7 @@ public Map getXAttrs(final Path path, } @Override - @AuditSpan.AuditEntryPoint + @AuditEntryPoint public List listXAttrs(final Path path) throws IOException { checkNotClosed(); try (AuditSpan span = createSpan( @@ -4683,7 +4683,7 @@ public ObjectMetadata getObjectMetadata(final String key) */ @Override @Retries.RetryTranslated - @AuditSpan.AuditEntryPoint + @AuditEntryPoint public RemoteIterator listFiles(Path f, boolean recursive) throws FileNotFoundException, IOException { final Path path = qualify(f); @@ -4702,7 +4702,7 @@ public RemoteIterator listFiles(Path f, */ @InterfaceAudience.Private @Retries.RetryTranslated - @AuditSpan.AuditEntryPoint + @AuditEntryPoint public RemoteIterator listFilesAndEmptyDirectories( Path f, boolean recursive) throws IOException { final Path path = qualify(f); @@ -4722,7 +4722,7 @@ public RemoteIterator listFilesAndEmptyDirectories( */ @InterfaceAudience.Private @Retries.RetryTranslated - @AuditSpan.AuditEntryPoint + @AuditEntryPoint public RemoteIterator listFilesAndEmptyDirectoriesForceNonAuth( Path f, boolean recursive) throws IOException { final Path path = qualify(f); @@ -4843,7 +4843,7 @@ public RemoteIterator listLocatedStatus(Path f) */ @Override @Retries.OnceTranslated("s3guard not retrying") - @AuditSpan.AuditEntryPoint + @AuditEntryPoint public RemoteIterator listLocatedStatus(final Path f, final PathFilter filter) throws FileNotFoundException, IOException { @@ -4905,7 +4905,7 @@ S3ALocatedFileStatus toLocatedFileStatus(S3AFileStatus status) */ @InterfaceAudience.Private @Retries.RetryTranslated - @AuditSpan.AuditEntryPoint + @AuditEntryPoint public MultipartUtils.UploadIterator listUploads(@Nullable String prefix) throws IOException { // span is picked up retained in the listing. @@ -5087,7 +5087,7 @@ protected void setTtlTimeProvider(ITtlTimeProvider ttlTimeProvider) { * @throws IOException IO failure */ @Retries.RetryTranslated - @AuditSpan.AuditEntryPoint + @AuditEntryPoint private FSDataInputStream select(final Path source, final String expression, final Configuration options, @@ -5200,7 +5200,7 @@ private S3AFileStatus extractOrFetchSimpleFileStatus( */ @Override @Retries.RetryTranslated - @AuditSpan.AuditEntryPoint + @AuditEntryPoint public CompletableFuture openFileWithOptions( final Path rawPath, final OpenFileParameters parameters) throws IOException { @@ -5266,7 +5266,7 @@ public CompletableFuture openFileWithOptions( } @Override - @AuditSpan.AuditEntryPoint + @AuditEntryPoint public S3AMultipartUploaderBuilder createMultipartUploader( final Path basePath) throws IOException { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/ActiveThreadSpanSource.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/ActiveThreadSpanSource.java new file mode 100644 index 0000000000000..c24c49cbdbc53 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/ActiveThreadSpanSource.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.audit; + +/** + * Interface to get the active thread span. + */ +public interface ActiveThreadSpanSource { + + /** + * Get the wrapped active span. + * @return the currently active span. + */ + AuditSpan getActiveThreadSpan(); +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditEntryPoint.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditEntryPoint.java new file mode 100644 index 0000000000000..fe937ceff7177 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditEntryPoint.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.audit; + +import java.lang.annotation.Documented; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; + +/** + * A marker attribute simply to highlight which of the methods + * in the FS are audit entry points. + * - S3AFileSystem and classes/methods it then invokes + * SHOULD NOT invoke audit entry points internally + * - All external methods MUST be audit entry points. + */ +@Documented +@Retention(RetentionPolicy.SOURCE) +public @interface AuditEntryPoint { +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditIntegration.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditIntegration.java index 2c3c136c989d2..c6667253ccd8b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditIntegration.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditIntegration.java @@ -30,7 +30,6 @@ import org.apache.hadoop.fs.s3a.audit.impl.LoggingAuditor; import org.apache.hadoop.fs.s3a.audit.impl.NoopAuditManager; import org.apache.hadoop.fs.s3a.audit.impl.NoopAuditor; -import org.apache.hadoop.fs.s3a.audit.impl.NoopSpan; import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; import org.apache.hadoop.util.functional.CallableRaisingIOE; import org.apache.hadoop.util.functional.InvocationRaisingIOE; @@ -148,30 +147,5 @@ public static OperationAuditor createAuditor( } } - /** - * Create, init and start a no-op auditor instance. - * @param conf configuration. - * @return a started instance. - */ - public static OperationAuditor noopAuditor(Configuration conf) { - return NoopAuditor.newInstance(conf); - } - /** - * Create a no-op span. - * @param name name - * @param path1 path - * @param path2 path 2 - * @return a span. - */ - public static final AuditSpan noopSpan(final String name, - final String path1, - final String path2) { - return new NoopSpan(name, path1, path2); - } - - /** - * Reusable no-op span instance. - */ - public static AuditSpan NOOP_SPAN = NoopSpan.INSTANCE; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditManager.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditManager.java index b0e60f57de178..af64e39ba99e7 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditManager.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditManager.java @@ -37,14 +37,10 @@ * */ @InterfaceAudience.Private -public interface AuditManager extends Service, AuditSpanSource, - AuditSpanCallbacks { - - /** - * Get the wrapped active span. - * @return the currently active span. - */ - AuditSpan getActiveThreadSpan(); +public interface AuditManager extends Service, + AuditSpanSource, + AuditSpanCallbacks, + ActiveThreadSpanSource { /** * Create the request handler(s) for this audit service. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditSpan.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditSpan.java index 368fe6f361763..c2f6215b4f0c5 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditSpan.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditSpan.java @@ -28,6 +28,15 @@ */ public interface AuditSpan extends Closeable, AuditSpanCallbacks { + /** + * Return a span ID which must be unique for all spans within + * this instance of the filesystem. + * Callers MUST NOT make any assumptions about the contents + * of this string. + * @return a non-empty string + */ + String getSpanId(); + /** * Make this span active in the current thread. * @return the activated span. @@ -60,14 +69,5 @@ default boolean isValidSpan() { return true; } - /** - * A marker simply to highlight which of the methods in the - * FS are audit entry points. - * - S3A Code SHOULD NOT invoke audit entry points internally - * - All external methods MUST be audit entry points. - */ - @Documented - @Retention(RetentionPolicy.SOURCE) - @interface AuditEntryPoint { - } + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/AbstractAuditSpanImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/AbstractAuditSpanImpl.java index d5211dc986aa3..939341deac49a 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/AbstractAuditSpanImpl.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/AbstractAuditSpanImpl.java @@ -20,12 +20,30 @@ import org.apache.hadoop.fs.s3a.audit.AuditSpan; +import static java.util.Objects.requireNonNull; + /** * Base class for the audit spans implementations.. */ public abstract class AbstractAuditSpanImpl implements AuditSpan { - protected AbstractAuditSpanImpl() { + /** + * Span ID. + */ + private final String spanId; + + + /** + * Constructor. + * @param spanId span ID. + */ + protected AbstractAuditSpanImpl(final String spanId) { + this.spanId = requireNonNull(spanId); + } + + @Override + public String getSpanId() { + return spanId; } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/AbstractOperationAuditor.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/AbstractOperationAuditor.java index e35dce13a1fda..378585c838c93 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/AbstractOperationAuditor.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/AbstractOperationAuditor.java @@ -18,6 +18,8 @@ package org.apache.hadoop.fs.s3a.audit.impl; +import java.util.concurrent.atomic.AtomicLong; + import org.apache.hadoop.fs.s3a.audit.OperationAuditor; import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; import org.apache.hadoop.service.AbstractService; @@ -32,6 +34,12 @@ public abstract class AbstractOperationAuditor extends AbstractService implements OperationAuditor { + + /** + * Counter to create unique auditor IDs. + */ + private final AtomicLong SPAN_ID_COUNTER = new AtomicLong(1); + /** * Destination for recording statistics, especially duration/count of * operations. @@ -57,4 +65,11 @@ public IOStatisticsStore getIOStatistics() { return iostatistics; } + /** + * Create a span ID. + * @return a unique span ID. + */ + protected final String createSpanID() { + return Long.toString(SPAN_ID_COUNTER.incrementAndGet()); + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/ActiveAuditManager.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/ActiveAuditManager.java index 5c81041192bcd..c55793a707d0c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/ActiveAuditManager.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/ActiveAuditManager.java @@ -87,14 +87,14 @@ public final class ActiveAuditManager * This is the span returned to after a wrapper is closed or * the span is reset to the unbonded span.. */ - private WrappingAuditSpan unboundedSpan; + private WrappingAuditSpan unbondedSpan; /** * Thread local span. This defaults to being * the unbonded span. */ private final ThreadLocal activeSpan = - ThreadLocal.withInitial(() -> getUnboundedSpan()); + ThreadLocal.withInitial(() -> getUnbondedSpan()); /** * Destination for recording statistics, especially duration/count of @@ -126,7 +126,7 @@ protected void serviceInit(final Configuration conf) throws Exception { @Override protected void serviceStart() throws Exception { super.serviceStart(); - setUnboundedSpan(new WrappingAuditSpan( + setUnbondedSpan(new WrappingAuditSpan( auditService.getUnbondedSpan(), false)); LOG.debug("Started audit service {}", auditService); } @@ -137,12 +137,12 @@ protected void serviceStart() throws Exception { * span. * @return the unbounded span. */ - private WrappingAuditSpan getUnboundedSpan() { - return unboundedSpan; + private WrappingAuditSpan getUnbondedSpan() { + return unbondedSpan; } - public void setUnboundedSpan(final WrappingAuditSpan unboundedSpan) { - this.unboundedSpan = unboundedSpan; + public void setUnbondedSpan(final WrappingAuditSpan unbondedSpan) { + this.unbondedSpan = unbondedSpan; } @Override @@ -169,7 +169,7 @@ private WrappingAuditSpan switchToActiveSpan(WrappingAuditSpan span) { if (span != null && span.isValidSpan()) { activeSpan.set(span); } else { - activeSpan.set(unboundedSpan); + activeSpan.set(unbondedSpan); } return activeSpan.get(); } @@ -331,6 +331,7 @@ final class WrappingAuditSpan extends AbstractAuditSpanImpl { */ private WrappingAuditSpan( final AuditSpan span, final boolean isValid) { + super(span.getSpanId()); this.span = requireNonNull(span); this.isValid = isValid; } @@ -373,7 +374,7 @@ public void deactivate() { // deactivate the span span.deactivate(); // and go to the unbounded one. - switchToActiveSpan(getUnboundedSpan()); + switchToActiveSpan(getUnbondedSpan()); } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java index bb733e26d3f7a..448ffc22671f8 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java @@ -75,13 +75,6 @@ public final class LoggingAuditor private final String auditorID = Long.toHexString( AUDITOR_ID_COUNTER.getAndIncrement()); - /** - * Counter for next operation in this service. - * Initial value is what will be used for the span ID when there - * is no active request span. - */ - private final AtomicLong nextOperationId = new AtomicLong(1); - /** * Some basic analysis for the logs. */ @@ -114,14 +107,6 @@ public final class LoggingAuditor */ private final String principal; - /** - * Create an operation ID. The nature of it should be opaque. - * @return an ID for the constructor. - */ - private long newOperationId() { - return nextOperationId.getAndIncrement(); - } - /** * Create the auditor. * The UGI current user is used to provide the principal; @@ -137,7 +122,7 @@ public LoggingAuditor(final String name, attributes.put(FILESYSTEM_ID, auditorID); final CommonAuditContext currentContext = currentContext(); warningSpan = new WarningSpan(AuditConstants.OUTSIDE_SPAN, - currentContext, newOperationId(), null, null); + currentContext, createSpanID(), null, null); // add the principal String p; try { @@ -186,7 +171,7 @@ public AuditSpan createSpan(final String name, getIOStatistics().incrementCounter( Statistic.AUDIT_SPAN_START.getSymbol()); final LoggingAuditSpan span = new LoggingAuditSpan(name, - prepareActiveContext(), newOperationId(), path1, path2); + prepareActiveContext(), createSpanID(), path1, path2); span.start(); return span; } @@ -227,18 +212,16 @@ private class LoggingAuditSpan extends AbstractAuditSpanImpl { private final String description; - private final String id; - private LoggingAuditSpan( final String name, final CommonAuditContext context, - final long operationId, + final String operationId, final String path1, final String path2) { + super(String.format("%s-%s", getContextId(), operationId)); this.operationName = name; - this.id = String.format("%s-%08d", getContextId(), operationId); referrer = HttpReferrerAuditEntry.builder() .withContextId(contextId) - .withOperationId(String.format("%08x", operationId)) + .withOperationId(operationId) .withOperationName(name) .withPath1(path1) .withPath2(path2) @@ -253,7 +236,7 @@ private LoggingAuditSpan( } public void start() { - LOG.trace("{} Start {}", getId(), getDescription()); + LOG.trace("{} Start {}", getSpanId(), getDescription()); } protected String getOperationName() { @@ -264,19 +247,15 @@ protected String getDescription() { return description; } - protected String getId() { - return id; - } - @Override public AuditSpan activate() { - LOG.trace("{} Activate {}", id, description); + LOG.trace("{} Activate {}", getSpanId(), description); return this; } @Override public void deactivate() { - LOG.trace("{} Deactivate {}", id, description); + LOG.trace("{} Deactivate {}", getSpanId(), description); } @Override @@ -288,7 +267,7 @@ public T beforeExecution( header); if (LOG.isDebugEnabled()) { LOG.debug("{} Executing {} with {}; {}", - id, + getSpanId(), getOperationName(), analyzer.analyze(request), header); @@ -300,7 +279,7 @@ public T beforeExecution( public String toString() { final StringBuilder sb = new StringBuilder( "LoggingAuditSpan{"); - sb.append(", id='").append(id).append('\''); + sb.append(", id='").append(getSpanId()).append('\''); sb.append("description='").append(description).append('\''); sb.append('}'); return sb.toString(); @@ -324,19 +303,19 @@ private final class WarningSpan extends LoggingAuditSpan { private WarningSpan( final String name, final CommonAuditContext context, - final long operationId, + final String operationId, final String path1, final String path2) { super(name, context, operationId, path1, path2); } @Override public void start() { - LOG.warn("{} Start {}", getId(), getDescription()); + LOG.warn("{} Start {}", getSpanId(), getDescription()); } @Override public AuditSpan activate() { - LOG.warn("{} Activate {}", getId(), getDescription()); + LOG.warn("{} Activate {}", getSpanId(), getDescription()); return this; } @@ -376,8 +355,8 @@ public T beforeExecution( String error = "Executing a request outside an audit span " + analyzer.analyze(request); LOG.warn("{} {}", - getId(), error); - final String unaudited = getId() + " " + getSpanId(), error); + final String unaudited = getSpanId() + " " + AuditConstants.UNAUDITED_OPERATION + " " + error; if (isRequestNotAlwaysInSpan(request)) { // can get by auditing during a copy, so don't overreact diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopAuditManager.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopAuditManager.java index 7a49b200f9490..64b70d019a1dd 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopAuditManager.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopAuditManager.java @@ -28,22 +28,51 @@ import com.amazonaws.services.s3.transfer.internal.TransferStateChangeListener; import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.s3a.audit.AuditManager; import org.apache.hadoop.fs.s3a.audit.AuditSpan; import org.apache.hadoop.service.AbstractService; /** * Simple No-op audit manager for use before a real - * audit chain is set up. + * audit chain is set up, and for testing. + * Audit spans always have a unique ID and the activation/deactivation + * operations on them will update this audit manager's active span. * It does have the service lifecycle, so do * create a unique instance whenever used. */ @InterfaceAudience.Private public class NoopAuditManager extends AbstractService - implements AuditManager { + implements AuditManager, NoopSpan.SpanActivationCallbacks { + private static final NoopAuditor NOOP_AUDITOR = + NoopAuditor.newInstance(new Configuration(), null); + + /** + * The inner auditor. + */ + private final NoopAuditor auditor; + + /** + * Thread local span. This defaults to being + * the unbonded span. + */ + private final ThreadLocal activeSpan = + ThreadLocal.withInitial(this::getUnbondedSpan); + + /** + * Constructor. + */ public NoopAuditManager() { super("NoopAuditManager"); + auditor = NoopAuditor.newInstance(new Configuration(), this); + } + + /** + * Unbonded span to use after deactivation. + */ + private AuditSpan getUnbondedSpan() { + return auditor.getUnbondedSpan(); } @Override @@ -55,7 +84,7 @@ public AuditSpan getActiveThreadSpan() { public AuditSpan createSpan(final String name, @Nullable final String path1, @Nullable final String path2) throws IOException { - return new NoopSpan(name, path1, path2); + return createNewSpan(name, path1, path2); } @Override @@ -71,4 +100,29 @@ public void transferStateChanged(final Transfer transfer, } }; } + + @Override + public void activate(final AuditSpan span) { + activeSpan.set(span); + } + + @Override + public void deactivate(final AuditSpan span) { + activate(getUnbondedSpan()); + } + + /** + * A static source of no-op spans, using the same span ID + * source as managed spans. + * @param name operation name. + * @param path1 first path of operation + * @param path2 second path of operation + * @return a span for the audit + */ + public static AuditSpan createNewSpan( + final String name, + final String path1, + final String path2) { + return NOOP_AUDITOR.createSpan(name, path1, path2); + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopAuditor.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopAuditor.java index b3d09da42245e..2178e1c1b17fa 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopAuditor.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopAuditor.java @@ -19,46 +19,64 @@ package org.apache.hadoop.fs.s3a.audit.impl; import javax.annotation.Nullable; -import java.io.IOException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.s3a.audit.AuditSpan; import org.apache.hadoop.fs.s3a.audit.OperationAuditor; -import org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding; -import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; + +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore; /** * An audit service which returns the {@link NoopSpan}. */ -public class NoopAuditor extends AbstractOperationAuditor { +public final class NoopAuditor extends AbstractOperationAuditor { + + /** + * unbonded span created in constructor. + */ + private final AuditSpan unbondedSpan; + + private final NoopSpan.SpanActivationCallbacks + activationCallbacks; + /** + * Constructor. + * @param name name + * @param activationCallbacks Activation callbacks. + */ public NoopAuditor(final String name, - final IOStatisticsStore iostatistics) { - super(name, iostatistics); + NoopSpan.SpanActivationCallbacks activationCallbacks) { + super(name, iostatisticsStore().build()); + unbondedSpan = createSpan("unbonded", null, null); + this.activationCallbacks = activationCallbacks; } @Override public AuditSpan createSpan(final String name, @Nullable final String path1, - @Nullable final String path2) throws IOException { - return new NoopSpan(name, path1, path2); + @Nullable final String path2) { + return new NoopSpan(createSpanID(), name, path1, path2, + activationCallbacks); } @Override public AuditSpan getUnbondedSpan() { - return NoopSpan.INSTANCE; + return unbondedSpan; } /** * Create, init and start an instance. * @param conf configuration. + * @param activationCallbacks Activation callbacks. * @return a started instance. */ - public static OperationAuditor newInstance(Configuration conf) { + public static NoopAuditor newInstance(Configuration conf, + NoopSpan.SpanActivationCallbacks activationCallbacks) { NoopAuditor noop = new NoopAuditor("noop", - IOStatisticsBinding.emptyStatisticsStore()); + activationCallbacks); noop.init(conf); noop.start(); return noop; } + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopSpan.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopSpan.java index 6b12bb7bb5b19..70aa052343394 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopSpan.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopSpan.java @@ -18,8 +18,13 @@ package org.apache.hadoop.fs.s3a.audit.impl; +import org.apache.hadoop.fs.s3a.audit.AuditSpan; + /** - * Simple no-op span. + * A minimal span with no direct side effects. + * It does have an ID and, if given callbacks, + * will notify the callback implementation + * of activation and deactivation. */ public final class NoopSpan extends AbstractAuditSpanImpl { @@ -29,6 +34,9 @@ public final class NoopSpan extends AbstractAuditSpanImpl { private final String path2; + /** Activation callbacks. */ + private final SpanActivationCallbacks activationCallbacks; + /** * Static public instance. */ @@ -36,32 +44,68 @@ public final class NoopSpan extends AbstractAuditSpanImpl { /** * Create a no-op span. + * @param spanId span ID * @param name name * @param path1 path * @param path2 path 2 + * @param activationCallbacks Activation callbacks. */ - public NoopSpan(final String name, final String path1, final String path2) { + NoopSpan(String spanId, + final String name, + final String path1, + final String path2, + final SpanActivationCallbacks activationCallbacks) { + super(spanId); this.name = name; this.path1 = path1; this.path2 = path2; + this.activationCallbacks = activationCallbacks; } NoopSpan() { - this("no-op", null, null); + this("", "no-op", null, null, null); + } + + + @Override + public AuditSpan activate() { + if (activationCallbacks != null) { + activationCallbacks.activate(this); + } + return this; } @Override public void deactivate() { - // no-op + if (activationCallbacks != null) { + activationCallbacks.deactivate(this); + } } @Override public String toString() { final StringBuilder sb = new StringBuilder("NoopSpan{"); + sb.append("id='").append(getSpanId()).append('\''); sb.append("name='").append(name).append('\''); sb.append(", path1='").append(path1).append('\''); sb.append(", path2='").append(path2).append('\''); sb.append('}'); return sb.toString(); } + + /** Activation callbacks. */ + public interface SpanActivationCallbacks { + + /** + * Span was activated. + * @param span span reference. + */ + void activate(AuditSpan span); + + /** + * Span was deactivated. + * @param span span reference. + */ + void deactivate(AuditSpan span); + } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java index 14fed4984857d..f90c18c718833 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java @@ -43,7 +43,6 @@ import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestDynamoTablePrefix; import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestPropertyBool; import static org.apache.hadoop.fs.s3a.S3AUtils.E_FS_CLOSED; -import static org.apache.hadoop.fs.s3a.audit.AuditIntegration.noopSpan; 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; @@ -245,7 +244,7 @@ protected String getTestTableName(String suffix) { * Uses the test method name for the span. * @return a span. */ - protected AuditSpan span() { + protected AuditSpan span() throws IOException { return span(getSpanSource()); } @@ -256,15 +255,8 @@ protected AuditSpan span() { * @param source source of spans; can be an S3A FS * @return a span. */ - protected AuditSpan span(AuditSpanSource source) { + protected AuditSpan span(AuditSpanSource source) throws IOException { - if (source != null) { - try { - return source.createSpan(getMethodName(), null, null); - } catch (Exception e) { - LOG.info("Span collection failure", e); - } - } - return noopSpan(getMethodName(), null, null); + return source.createSpan(getMethodName(), null, null); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java index 0c941fc7da186..6fd738b7b7bb4 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java @@ -37,8 +37,7 @@ import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.s3a.api.RequestFactory; -import org.apache.hadoop.fs.s3a.audit.impl.NoopAuditor; -import org.apache.hadoop.fs.s3a.audit.impl.NoopSpan; +import org.apache.hadoop.fs.s3a.audit.AuditTestIntegration; import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets; import org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase; import org.apache.hadoop.fs.s3a.impl.RequestFactoryImpl; @@ -48,6 +47,7 @@ import org.apache.hadoop.fs.statistics.DurationTrackerFactory; import org.apache.hadoop.util.Progressable; +import static org.apache.hadoop.fs.s3a.audit.AuditTestIntegration.noopAuditor; import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.stubDurationTrackerFactory; import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull; @@ -172,8 +172,8 @@ public void initialize(URI name, Configuration originalConf) writeHelper = new WriteOperationHelper(this, conf, new EmptyS3AStatisticsContext(), - NoopAuditor.newInstance(conf), - NoopSpan.INSTANCE); + noopAuditor(conf), + AuditTestIntegration.NOOP_SPAN); } @Override diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java index 324b5ff430432..a663da75d0faa 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java @@ -20,6 +20,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.s3a.audit.AuditTestIntegration; import org.apache.hadoop.fs.s3a.commit.PutTracker; import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext; import org.apache.hadoop.util.Progressable; @@ -30,8 +31,7 @@ import java.io.IOException; import java.util.concurrent.ExecutorService; -import static org.apache.hadoop.fs.s3a.audit.AuditIntegration.NOOP_SPAN; -import static org.apache.hadoop.fs.s3a.audit.AuditIntegration.noopAuditor; +import static org.apache.hadoop.fs.s3a.audit.AuditTestIntegration.noopAuditor; import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; @@ -77,7 +77,7 @@ public void testWriteOperationHelperPartLimits() throws Throwable { conf, new EmptyS3AStatisticsContext(), noopAuditor(conf), - NOOP_SPAN); + AuditTestIntegration.NOOP_SPAN); ByteArrayInputStream inputStream = new ByteArrayInputStream( "a".getBytes()); // first one works diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopSpanSource.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AuditTestIntegration.java similarity index 55% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopSpanSource.java rename to hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AuditTestIntegration.java index 51bb4b0419aff..32c3e191e5fb8 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopSpanSource.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AuditTestIntegration.java @@ -16,32 +16,30 @@ * limitations under the License. */ -package org.apache.hadoop.fs.s3a.audit.impl; +package org.apache.hadoop.fs.s3a.audit; -import javax.annotation.Nullable; -import java.io.IOException; - -import org.apache.hadoop.fs.s3a.audit.AuditSpan; -import org.apache.hadoop.fs.s3a.audit.AuditSpanSource; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.s3a.audit.impl.NoopAuditManager; +import org.apache.hadoop.fs.s3a.audit.impl.NoopAuditor; /** - * Simple no-op source, which always returns - * a new {@link NoopSpan}. + * Support for auditing in testing. */ -public final class NoopSpanSource implements AuditSpanSource { +public class AuditTestIntegration { /** - * Sole public instance. + * Reusable no-op span instance. */ - public static final NoopSpanSource INSTANCE = new NoopSpanSource(); + public static AuditSpan NOOP_SPAN = NoopAuditManager + .createNewSpan("noop", null, null); - private NoopSpanSource() { - } - @Override - public AuditSpan createSpan(final String name, - @Nullable final String path1, - @Nullable final String path2) throws IOException { - return new NoopSpan(name, path1, path2); + /** + * Create, init and start a no-op auditor instance. + * @param conf configuration. + * @return a started instance. + */ + public static OperationAuditor noopAuditor(Configuration conf) { + return NoopAuditor.newInstance(conf, null); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestHeaderProcessing.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestHeaderProcessing.java index b8072218c6802..1c6b639191908 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestHeaderProcessing.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestHeaderProcessing.java @@ -37,11 +37,11 @@ import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.api.RequestFactory; import org.apache.hadoop.fs.s3a.audit.AuditSpan; +import org.apache.hadoop.fs.s3a.audit.AuditTestIntegration; import org.apache.hadoop.fs.s3a.test.OperationTrackingStore; import org.apache.hadoop.test.HadoopTestBase; import static java.lang.System.currentTimeMillis; -import static org.apache.hadoop.fs.s3a.audit.AuditIntegration.NOOP_SPAN; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.XA_MAGIC_MARKER; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.X_HEADER_MAGIC_MARKER; import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.XA_LAST_MODIFIED; @@ -299,7 +299,7 @@ public Path makeQualified(final Path path) { @Override public AuditSpan getActiveAuditSpan() { - return NOOP_SPAN; + return AuditTestIntegration.NOOP_SPAN; } @Override diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java index c1744832e7fae..47b6ad853e685 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java @@ -40,9 +40,9 @@ import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.api.RequestFactory; import org.apache.hadoop.fs.s3a.audit.AuditSpan; +import org.apache.hadoop.fs.s3a.audit.AuditTestIntegration; import org.apache.hadoop.fs.s3a.test.OperationTrackingStore; -import static org.apache.hadoop.fs.s3a.audit.AuditIntegration.NOOP_SPAN; import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.ACCESS_DENIED; import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.removeUndeletedPaths; import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.toPathList; @@ -261,7 +261,7 @@ public Path makeQualified(final Path path) { @Override public AuditSpan getActiveAuditSpan() { - return NOOP_SPAN; + return AuditTestIntegration.NOOP_SPAN; } @Override