From 5aff4259624111506c0d3a77d048d74ef746c0f5 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 21 Jan 2020 18:46:54 +0000 Subject: [PATCH 01/17] HADOOP-16823. Manage S3 Throttling exclusively in S3A client. Currently AWS S3 throttling is initially handled in the AWS SDK, only reaching the S3 client code after it has given up. This means we don't always directly observe when throttling is taking place. Proposed: * disable throttling retries in the AWS client Library * add a quantile for the S3 throttle events, as DDB has * isolate counters of s3 and DDB throttle events to classify issues better Because we are taking over the AWS retries, we will need to expand the initial delay en retries and the number of retries we should support before giving up. Also: should we log throttling events? It could be useful but there is a risk of logs overloading especially if many threads in the same process were triggering the problem. Change-Id: I386928cd478a6a9fbb91f15b9185a1ea91878680 Proposed: log at debug. --- .../src/main/resources/core-default.xml | 10 +++++-- .../org/apache/hadoop/fs/s3a/Constants.java | 5 ++-- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 30 +++++++++++++++---- .../hadoop/fs/s3a/S3AInstrumentation.java | 6 +++- .../org/apache/hadoop/fs/s3a/S3AUtils.java | 1 + .../org/apache/hadoop/fs/s3a/Statistic.java | 1 + 6 files changed, 40 insertions(+), 13 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml index 07f3e8ab1e9d4..20e3639783bd6 100644 --- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml +++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml @@ -1698,7 +1698,7 @@ fs.s3a.retry.throttle.limit - ${fs.s3a.attempts.maximum} + 20 Number of times to retry any throttled request. @@ -1706,9 +1706,13 @@ fs.s3a.retry.throttle.interval - 1000ms + 5000s - Interval between retry attempts on throttled requests. + Initial between retry attempts on throttled requests, +/- 50%. chosen at random. + i.e. for an intial value of 3000ms, the initial delay would be in the range 1500ms to 4500ms. + Backoffs are exponential; again randomness is used to avoid the thundering heard problem. + Given that throttling in S3 is per-second, very short delays will not initial spread + out work and so continue to create the problem. 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 e107d4987f0da..6dfe73e486fc2 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 @@ -733,8 +733,7 @@ private Constants() { /** * Default throttled retry limit: {@value}. */ - public static final int RETRY_THROTTLE_LIMIT_DEFAULT = - DEFAULT_MAX_ERROR_RETRIES; + public static final int RETRY_THROTTLE_LIMIT_DEFAULT = 20; /** * Interval between retry attempts on throttled requests: {@value}. @@ -745,7 +744,7 @@ private Constants() { /** * Default throttled retry interval: {@value}. */ - public static final String RETRY_THROTTLE_INTERVAL_DEFAULT = "500ms"; + public static final String RETRY_THROTTLE_INTERVAL_DEFAULT = "5000ms"; /** * Should etags be exposed as checksums? 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 cc12848df9ea9..d48814e00ac36 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 @@ -1648,10 +1648,11 @@ protected void incrementGauge(Statistic statistic, long count) { * @param ex exception. */ public void operationRetried(Exception ex) { - Statistic stat = isThrottleException(ex) - ? STORE_IO_THROTTLED - : IGNORED_ERRORS; - incrementStatistic(stat); + if (isThrottleException(ex)) { + operationThrottled(false); + } else { + incrementStatistic(IGNORED_ERRORS); + } } /** @@ -1684,11 +1685,28 @@ public void operationRetried( public void metastoreOperationRetried(Exception ex, int retries, boolean idempotent) { - operationRetried(ex); incrementStatistic(S3GUARD_METADATASTORE_RETRY); if (isThrottleException(ex)) { + operationThrottled(true); + } else { + incrementStatistic(IGNORED_ERRORS); + } + } + + /** + * Note that an operation was throttled -this will update + * specific counters/metrics. + * @param metastore was the throttling observed in the S3Guard metastore? + */ + private void operationThrottled(boolean metastore) { + LOG.debug("Request throttled on {}", metastore ? "S3": "DynamoDB"); + if (metastore) { incrementStatistic(S3GUARD_METADATASTORE_THROTTLED); - instrumentation.addValueToQuantiles(S3GUARD_METADATASTORE_THROTTLE_RATE, 1); + instrumentation.addValueToQuantiles(S3GUARD_METADATASTORE_THROTTLE_RATE, + 1); + } else { + incrementStatistic(STORE_IO_THROTTLED); + instrumentation.addValueToQuantiles(STORE_IO_THROTTLE_RATE, 1); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java index 4e1de370a6cc8..b9918b5098946 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java @@ -123,6 +123,7 @@ public class S3AInstrumentation implements Closeable, MetricsSource { private final MutableCounterLong ignoredErrors; private final MutableQuantiles putLatencyQuantile; private final MutableQuantiles throttleRateQuantile; + private final MutableQuantiles s3GuardThrottleRateQuantile; private final MutableCounterLong numberOfFilesCreated; private final MutableCounterLong numberOfFilesCopied; private final MutableCounterLong bytesOfFilesCopied; @@ -248,7 +249,9 @@ public S3AInstrumentation(URI name) { int interval = 1; putLatencyQuantile = quantiles(S3GUARD_METADATASTORE_PUT_PATH_LATENCY, "ops", "latency", interval); - throttleRateQuantile = quantiles(S3GUARD_METADATASTORE_THROTTLE_RATE, + s3GuardThrottleRateQuantile = quantiles(S3GUARD_METADATASTORE_THROTTLE_RATE, + "events", "frequency (Hz)", interval); + throttleRateQuantile = quantiles(STORE_IO_THROTTLE_RATE, "events", "frequency (Hz)", interval); registerAsMetricsSource(name); @@ -617,6 +620,7 @@ public void close() { // task in a shared thread pool. putLatencyQuantile.stop(); throttleRateQuantile.stop(); + s3GuardThrottleRateQuantile.stop(); metricsSystem.unregisterSource(metricsSourceName); int activeSources = --metricsSourceActiveCounter; if (activeSources == 0) { 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 e2a488e8fed9c..25da2207afc7f 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 @@ -1234,6 +1234,7 @@ public static ClientConfiguration createAwsConf(Configuration conf, initConnectionSettings(conf, awsConf); initProxySupport(conf, bucket, awsConf); initUserAgent(conf, awsConf); + awsConf.setUseThrottleRetries(false); if (StringUtils.isNotEmpty(awsServiceIdentifier)) { String configKey = null; switch (awsServiceIdentifier) { 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 06c60a46f5e79..018f16b8ed95f 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 @@ -230,6 +230,7 @@ public enum Statistic { "S3Guard metadata store authoritative directories updated from S3"), STORE_IO_THROTTLED("store_io_throttled", "Requests throttled and retried"), + STORE_IO_THROTTLE_RATE("store_io_throttle_rate", "Rate of S3 request throttling"), DELEGATION_TOKENS_ISSUED("delegation_tokens_issued", "Number of delegation tokens issued"); From daba733bb9359b0b770da4409b957f2cd5572280 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 29 Jan 2020 13:54:17 +0000 Subject: [PATCH 02/17] HADOOP-16823. Throttling fix checkstyle Change-Id: I19f3848b298a8656ee5f986a2ba1cde50a106814 --- .../src/main/java/org/apache/hadoop/fs/s3a/Statistic.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 018f16b8ed95f..1d3d4758028c6 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 @@ -230,7 +230,8 @@ public enum Statistic { "S3Guard metadata store authoritative directories updated from S3"), STORE_IO_THROTTLED("store_io_throttled", "Requests throttled and retried"), - STORE_IO_THROTTLE_RATE("store_io_throttle_rate", "Rate of S3 request throttling"), + STORE_IO_THROTTLE_RATE("store_io_throttle_rate", + "Rate of S3 request throttling"), DELEGATION_TOKENS_ISSUED("delegation_tokens_issued", "Number of delegation tokens issued"); From 756313c676ad104cf39129017e2146ddd825f6a7 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 29 Jan 2020 18:59:10 +0000 Subject: [PATCH 03/17] HADOOP-16823. Throttling: DynamoDB Turning off throttling in the AWS client causes problems for the DDBMetastore; including showing where tests were making non-retrying operations against the table. Mostly addressed though ITestDynamoDBMetadataStoreScale is still petulant. Either it takes too long to finish or it doesn't throttle. Oh, and lag means that while a test may fail because throttling wasn't raised, the next IO may fail. Change-Id: I37bbcb67023f4cb3ebdcba978602be58099ad306 --- .../hadoop/fs/s3a/DefaultS3ClientFactory.java | 5 + .../org/apache/hadoop/fs/s3a/S3AUtils.java | 1 - .../s3a/s3guard/DumpS3GuardDynamoTable.java | 4 +- .../fs/s3a/s3guard/DynamoDBMetadataStore.java | 128 ++++++++++++++++++ .../s3a/s3guard/PurgeS3GuardDynamoTable.java | 14 +- .../fs/s3a/s3guard/S3GuardTableAccess.java | 7 + .../ITestDynamoDBMetadataStoreScale.java | 54 ++++++-- .../fs/s3a/s3guard/ThrottleTracker.java | 33 ++++- 8 files changed, 222 insertions(+), 24 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 ff8ba1d6d5dac..1b833c5bde47e 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 @@ -57,6 +57,11 @@ public AmazonS3 createS3Client(URI name, Configuration conf = getConf(); final ClientConfiguration awsConf = S3AUtils .createAwsConf(getConf(), bucket, Constants.AWS_SERVICE_IDENTIFIER_S3); + + // throttling is explicitly disabled on the S3 client so that + // all failures are collected + awsConf.setUseThrottleRetries(false); + if (!StringUtils.isEmpty(userAgentSuffix)) { awsConf.setUserAgentSuffix(userAgentSuffix); } 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 25da2207afc7f..e2a488e8fed9c 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 @@ -1234,7 +1234,6 @@ public static ClientConfiguration createAwsConf(Configuration conf, initConnectionSettings(conf, awsConf); initProxySupport(conf, bucket, awsConf); initUserAgent(conf, awsConf); - awsConf.setUseThrottleRetries(false); if (StringUtils.isNotEmpty(awsServiceIdentifier)) { String configKey = null; switch (awsServiceIdentifier) { 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 7a273a66c2938..536481ac23b7e 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 @@ -439,8 +439,8 @@ private void dumpEntry(CsvFile csv, DDBPathMetadata md) { private Pair scanMetastore(CsvFile csv) { S3GuardTableAccess tableAccess = new S3GuardTableAccess(getStore()); ExpressionSpecBuilder builder = new ExpressionSpecBuilder(); - Iterable results = tableAccess.scanMetadata( - builder); + Iterable results = + getStore().wrapWithRetries(tableAccess.scanMetadata(builder)); long live = 0; long tombstone = 0; for (DDBPathMetadata md : results) { 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 143e276d1fef3..67522daa34f98 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 @@ -29,6 +29,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Objects; @@ -80,6 +81,7 @@ import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.impl.FunctionsRaisingIOE; +import org.apache.hadoop.fs.impl.WrappedIOException; import org.apache.hadoop.fs.s3a.AWSCredentialProviderList; import org.apache.hadoop.fs.s3a.AWSServiceThrottledException; import org.apache.hadoop.fs.s3a.Constants; @@ -324,8 +326,12 @@ public class DynamoDBMetadataStore implements MetadataStore, /** Invoker for write operations. */ private Invoker writeOp; + /** Invoker for scan operations. */ + private Invoker scanOp; + private final AtomicLong readThrottleEvents = new AtomicLong(0); private final AtomicLong writeThrottleEvents = new AtomicLong(0); + private final AtomicLong scanThrottleEvents = new AtomicLong(0); private final AtomicLong batchWriteCapacityExceededEvents = new AtomicLong(0); /** @@ -543,6 +549,7 @@ private void initDataAccessRetries(Configuration config) { = new S3GuardDataAccessRetryPolicy(config); readOp = new Invoker(throttledRetryRetryPolicy, this::readRetryEvent); writeOp = new Invoker(throttledRetryRetryPolicy, this::writeRetryEvent); + scanOp = new Invoker(throttledRetryRetryPolicy, this::scanRetryEvent); } @Override @@ -1992,6 +1999,22 @@ void writeRetryEvent( retryEvent(text, ex, attempts, idempotent); } + /** + * Callback on a scan operation retried. + * @param text text of the operation + * @param ex exception + * @param attempts number of attempts + * @param idempotent is the method idempotent (this is assumed to be true) + */ + void scanRetryEvent( + String text, + IOException ex, + int attempts, + boolean idempotent) { + scanThrottleEvents.incrementAndGet(); + retryEvent(text, ex, attempts, idempotent); + } + /** * Callback from {@link Invoker} when an operation is retried. * @param text text of the operation @@ -2048,6 +2071,15 @@ public long getWriteThrottleEventCount() { return writeThrottleEvents.get(); } + /** + * Get the count of scan throttle events. + * @return the current count of scan throttle events. + */ + @VisibleForTesting + public long getScanThrottleEventCount() { + return scanThrottleEvents.get(); + } + @VisibleForTesting public long getBatchWriteCapacityExceededCount() { return batchWriteCapacityExceededEvents.get(); @@ -2475,4 +2507,100 @@ protected DynamoDBMetadataStoreTableManager getTableHandler() { Preconditions.checkNotNull(tableHandler, "Not initialized"); return tableHandler; } + + /** + * Get the operation invoker for write operations. + * @return an invoker for retrying mutating operations on a store. + */ + Invoker getWriteOperationInvoker() { + return writeOp; + } + + /** + * Wrap an iterator returned from any scan with a retrying one. + * This includes throttle handling. + * @param source source iterator + * @return a retrying iterator. + */ + public Iterator wrapWithRetries( + final Iterator source) { + return new RetryingDDBPathMetadataIterator(source); + } + + /** + * Wrap an iterator returned from any scan with a retrying one. + * This includes throttle handling. + * @param source source iterator + * @return a retrying iterator. + */ + public Iterable wrapWithRetries( + final Iterable source) { + return new RetryingDDBPathMetadataCollection(source); + } + + /** + * A collection which wraps the result of a query or scan. + * Important: iterate through this only once; the outcome + * of repeating an iteration is "undefined" + * @param type of outcome. + */ + private final class RetryingDDBPathMetadataCollection + implements Iterable { + + private final Iterable source; + + private RetryingDDBPathMetadataCollection( + final Iterable source) { + this.source = source; + } + + + @Override + public Iterator iterator() { + return wrapWithRetries(source.iterator()); + } + } + + /** + * An iterator which wraps a non-retrying iterator of scan results + * (i.e {@code S3GuardTableAccess.DDBPathMetadataIterator}. + */ + private final class RetryingDDBPathMetadataIterator implements + Iterator { + + private final Iterator source; + + private RetryingDDBPathMetadataIterator( + final Iterator source) { + this.source = source; + } + + @Override + @Retries.RetryTranslated + public boolean hasNext() { + try { + return scanOp.retry( + "Scan Dynamo", + null, + true, + source::hasNext); + } catch (IOException e) { + throw new WrappedIOException(e); + } + } + + @Override + public DDBPathMetadata next() { + try { + return scanOp.retry( + "Scan Dynamo", + null, + true, + source::next); + } catch (IOException e) { + throw new WrappedIOException(e); + } + } + } + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PurgeS3GuardDynamoTable.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PurgeS3GuardDynamoTable.java index 244779abb939b..d577a91d57c91 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PurgeS3GuardDynamoTable.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PurgeS3GuardDynamoTable.java @@ -19,6 +19,7 @@ package org.apache.hadoop.fs.s3a.s3guard; import javax.annotation.Nullable; +import java.io.IOException; import java.net.URI; import java.util.ArrayList; import java.util.Arrays; @@ -128,9 +129,10 @@ protected void serviceStart() throws Exception { * delete all entries from that bucket. * @return the exit code. * @throws ServiceLaunchException on failure. + * @throws IOException IO failure. */ @Override - public int execute() throws ServiceLaunchException { + public int execute() throws ServiceLaunchException, IOException { URI uri = getUri(); String host = uri.getHost(); @@ -144,7 +146,8 @@ public int execute() throws ServiceLaunchException { LOG.info("Scanning for entries with prefix {} to delete from {}", prefix, ddbms); - Iterable entries = tableAccess.scanMetadata(builder); + Iterable entries = + ddbms.wrapWithRetries(tableAccess.scanMetadata(builder)); List list = new ArrayList<>(); entries.iterator().forEachRemaining(e -> { if (!(e instanceof S3GuardTableAccess.VersionMarker)) { @@ -169,7 +172,12 @@ public int execute() throws ServiceLaunchException { new DurationInfo(LOG, "deleting %s entries from %s", count, ddbms.toString()); - tableAccess.delete(list); + ddbms.getWriteOperationInvoker() + .retry("delete", + prefix, + true, + () -> tableAccess.delete(list)); + duration.close(); long durationMillis = duration.value(); long timePerEntry = durationMillis / count; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTableAccess.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTableAccess.java index 19ef90e455741..0f33f4eb51c4d 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTableAccess.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTableAccess.java @@ -36,6 +36,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.Retries; import org.apache.hadoop.fs.s3a.S3AFileStatus; import static com.google.common.base.Preconditions.checkNotNull; @@ -70,6 +71,7 @@ */ @InterfaceAudience.Private @InterfaceStability.Unstable +@Retries.OnceRaw class S3GuardTableAccess { private static final Logger LOG = @@ -107,6 +109,7 @@ private String getUsername() { * @param spec query spec. * @return the outcome. */ + @Retries.OnceRaw ItemCollection query(QuerySpec spec) { return table.query(spec); } @@ -118,18 +121,22 @@ ItemCollection query(QuerySpec spec) { * @param spec query spec. * @return an iterator over path entries. */ + @Retries.OnceRaw Iterable queryMetadata(QuerySpec spec) { return new DDBPathMetadataCollection<>(query(spec)); } + @Retries.OnceRaw ItemCollection scan(ExpressionSpecBuilder spec) { return table.scan(spec.buildForScan()); } + @Retries.OnceRaw Iterable scanMetadata(ExpressionSpecBuilder spec) { return new DDBPathMetadataCollection<>(scan(spec)); } + @Retries.OnceRaw void delete(Collection paths) { paths.stream() .map(PathMetadataDynamoDBTranslation::pathToKey) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java index f2f37f21ea5a7..b1d8f18d788b5 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java @@ -47,6 +47,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.StorageStatistics; import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.impl.WrappedIOException; import org.apache.hadoop.fs.s3a.AWSServiceThrottledException; import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.S3AFileSystem; @@ -54,7 +55,6 @@ import org.apache.hadoop.fs.s3a.Statistic; import org.apache.hadoop.fs.s3a.scale.AbstractITestS3AMetadataStoreScale; import org.apache.hadoop.io.IOUtils; -import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.LambdaTestUtils; import org.apache.hadoop.util.DurationInfo; @@ -180,17 +180,29 @@ public void teardown() throws Exception { if (ddbms != null) { S3GuardTableAccess tableAccess = new S3GuardTableAccess(ddbms); ExpressionSpecBuilder builder = new ExpressionSpecBuilder(); + final String path = "/test/"; builder.withCondition( - ExpressionSpecBuilder.S(PARENT).beginsWith("/test/")); - - Iterable entries = tableAccess.scanMetadata(builder); + ExpressionSpecBuilder.S(PARENT).beginsWith(path)); + Iterable entries = + ddbms.wrapWithRetries(tableAccess.scanMetadata(builder)); List list = new ArrayList<>(); - entries.iterator().forEachRemaining(e -> { - Path p = e.getFileStatus().getPath(); - LOG.info("Deleting {}", p); - list.add(p); - }); - tableAccess.delete(list); + try { + entries.iterator().forEachRemaining(e -> { + Path p = e.getFileStatus().getPath(); + LOG.info("Deleting {}", p); + list.add(p); + }); + } catch (WrappedIOException e) { + // the iterator may have overloaded; swallow if so. + if (!(e.getCause() instanceof AWSServiceThrottledException)) { + throw e; + } + } + ddbms.getWriteOperationInvoker() + .retry("delete", + path, + true, + () -> tableAccess.delete(list)); } IOUtils.cleanupWithLogger(LOG, ddbms); super.teardown(); @@ -200,6 +212,25 @@ private boolean expectThrottling() { return !isOverProvisionedForTest && !isOnDemandTable; } + /** + * The subclass expects the superclass to be throttled; sometimes it is. + */ + @Test + @Override + public void test_010_Put() throws Throwable { + ThrottleTracker tracker = new ThrottleTracker(ddbms); + try { + // if this doesn't throttle, all is well. + super.test_010_Put(); + } catch (AWSServiceThrottledException ex) { + // if the service was throttled, all is good. + // log and continue + LOG.warn("DDB connection was throttled", ex); + } finally { + LOG.info("Statistics {}", tracker); + } + } + /** * The subclass expects the superclass to be throttled; sometimes it is. */ @@ -284,8 +315,7 @@ public void test_030_BatchedWrite() throws Exception { } }); if (expectThrottling()) { - assertNotEquals("No batch retries in " + result, - 0, result.getBatchThrottles()); + result.assertThrottlingDetected(); } } finally { describe("Cleaning up table %s", tableName); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ThrottleTracker.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ThrottleTracker.java index 5e33be8367de4..942d7fecaff30 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ThrottleTracker.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ThrottleTracker.java @@ -42,12 +42,16 @@ class ThrottleTracker { private long batchWriteThrottleCountOrig = 0; + private long scanThrottleCountOrig; + private long readThrottles; private long writeThrottles; private long batchThrottles; + private long scanThrottles; + ThrottleTracker(final DynamoDBMetadataStore ddbms) { this.ddbms = ddbms; reset(); @@ -65,6 +69,9 @@ public synchronized void reset() { batchWriteThrottleCountOrig = ddbms.getBatchWriteCapacityExceededCount(); + + scanThrottleCountOrig + = ddbms.getScanThrottleEventCount(); } /** @@ -78,6 +85,8 @@ public synchronized boolean probe() { - writeThrottleEventOrig); setBatchThrottles(ddbms.getBatchWriteCapacityExceededCount() - batchWriteThrottleCountOrig); + setScanThrottles(ddbms.getScanThrottleEventCount() + - scanThrottleCountOrig); return isThrottlingDetected(); } @@ -85,9 +94,11 @@ public synchronized boolean probe() { public String toString() { return String.format( "Tracker with read throttle events = %d;" - + " write events = %d;" - + " batch throttles = %d", - getReadThrottles(), getWriteThrottles(), getBatchThrottles()); + + " write throttles = %d;" + + " batch throttles = %d;" + + " scan throttles = %d", + getReadThrottles(), getWriteThrottles(), getBatchThrottles(), + getScanThrottles()); } /** @@ -101,11 +112,13 @@ public void assertThrottlingDetected() { /** * Has there been any throttling on an operation? - * @return true iff read, write or batch operations were throttled. + * @return true if any operations were throttled. */ public boolean isThrottlingDetected() { - return getReadThrottles() > 0 || getWriteThrottles() - > 0 || getBatchThrottles() > 0; + return getReadThrottles() > 0 + || getWriteThrottles() > 0 + || getBatchThrottles() > 0 + || getScanThrottles() > 0; } public long getReadThrottles() { @@ -131,4 +144,12 @@ public long getBatchThrottles() { public void setBatchThrottles(long batchThrottles) { this.batchThrottles = batchThrottles; } + + public long getScanThrottles() { + return scanThrottles; + } + + public void setScanThrottles(final long scanThrottles) { + this.scanThrottles = scanThrottles; + } } From f073f92ae9f69f3e517c230f1f4f1e89df464871 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 29 Jan 2020 20:44:07 +0000 Subject: [PATCH 04/17] HADOOP-16823 improve retry logic of listChildren * Split out where/how we retry listchildren * trying to speed up the ddb scale tests (though the latest change there triggers an NPE...) For anyone curious why tests take so long -it's probably set up of the per-test-case FS instance, because that has full retry, and once one test has throttled, that spin/wait goes on until DDB is letting the client at it. Which is a PITA but it does at least mean that "usually" each test case is in a recovered state. Do we care? Should we just run them back to back and be happy overloading things? I think so Change-Id: Ib35d450449fffaa2379d62ca12180eaa70c38584 --- .../fs/s3a/s3guard/DynamoDBMetadataStore.java | 98 ++++++------- .../ITestDynamoDBMetadataStoreScale.java | 136 ++++++++++-------- 2 files changed, 123 insertions(+), 111 deletions(-) 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 67522daa34f98..9e01673b4cc02 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 @@ -430,11 +430,6 @@ public void initialize(FileSystem fs, ITtlTimeProvider ttlTp) tableName = conf.getTrimmed(S3GUARD_DDB_TABLE_NAME_KEY, bucket); initDataAccessRetries(conf); - // set up a full retry policy - invoker = new Invoker(new S3GuardDataAccessRetryPolicy(conf), - this::retryEvent - ); - this.ttlTimeProvider = ttlTp; tableHandler = new DynamoDBMetadataStoreTableManager( @@ -816,34 +811,25 @@ private S3AFileStatus makeDirStatus(String dirOwner, Path path) { public DirListingMetadata listChildren(final Path path) throws IOException { checkPath(path); LOG.debug("Listing table {} in region {}: {}", tableName, region, path); - + final QuerySpec spec = new QuerySpec() + .withHashKey(pathToParentKeyAttribute(path)) + .withConsistentRead(true); // strictly consistent read + final List metas = new ArrayList<>(); // find the children in the table - return readOp.retry( + final ItemCollection items = scanOp.retry( "listChildren", path.toString(), true, - () -> { - final QuerySpec spec = new QuerySpec() - .withHashKey(pathToParentKeyAttribute(path)) - .withConsistentRead(true); // strictly consistent read - final ItemCollection items = table.query(spec); - - final List metas = new ArrayList<>(); - for (Item item : items) { - DDBPathMetadata meta = itemToPathMetadata(item, username); - metas.add(meta); - } - - // Minor race condition here - if the path is deleted between - // getting the list of items and the directory metadata we might - // get a null in DDBPathMetadata. - DDBPathMetadata dirPathMeta = get(path); - - final DirListingMetadata dirListing = - getDirListingMetadataFromDirMetaAndList(path, metas, - dirPathMeta); - return dirListing; - }); + () -> table.query(spec)); + // now wrap the result with retry logic + for (Item item : wrapWithRetries(items)) { + metas.add(itemToPathMetadata(item, username)); + } + // Minor race condition here - if the path is deleted between + // getting the list of items and the directory metadata we might + // get a null in DDBPathMetadata. + return getDirListingMetadataFromDirMetaAndList(path, metas, + get(path)); } DirListingMetadata getDirListingMetadataFromDirMetaAndList(Path path, @@ -2085,11 +2071,6 @@ public long getBatchWriteCapacityExceededCount() { return batchWriteCapacityExceededEvents.get(); } - @VisibleForTesting - public Invoker getInvoker() { - return invoker; - } - /** * Record the number of records written. * @param count count of records. @@ -2522,9 +2503,9 @@ Invoker getWriteOperationInvoker() { * @param source source iterator * @return a retrying iterator. */ - public Iterator wrapWithRetries( - final Iterator source) { - return new RetryingDDBPathMetadataIterator(source); + Iterator wrapWithRetries( + final Iterator source) { + return new RetryingIterator<>(source); } /** @@ -2533,30 +2514,32 @@ public Iterator wrapWithRetries( * @param source source iterator * @return a retrying iterator. */ - public Iterable wrapWithRetries( - final Iterable source) { - return new RetryingDDBPathMetadataCollection(source); + Iterable wrapWithRetries( + final Iterable source) { + return new RetryingCollection<>(source); } /** - * A collection which wraps the result of a query or scan. + * A collection which wraps the result of a query or scan + * with retries; the {@link #scanThrottleEvents} count is + * then updated. * Important: iterate through this only once; the outcome * of repeating an iteration is "undefined" * @param type of outcome. */ - private final class RetryingDDBPathMetadataCollection - implements Iterable { + private final class RetryingCollection + implements Iterable { - private final Iterable source; + private final Iterable source; - private RetryingDDBPathMetadataCollection( - final Iterable source) { + private RetryingCollection( + final Iterable source) { this.source = source; } @Override - public Iterator iterator() { + public Iterator iterator() { return wrapWithRetries(source.iterator()); } } @@ -2565,16 +2548,20 @@ public Iterator iterator() { * An iterator which wraps a non-retrying iterator of scan results * (i.e {@code S3GuardTableAccess.DDBPathMetadataIterator}. */ - private final class RetryingDDBPathMetadataIterator implements - Iterator { + private final class RetryingIterator implements + Iterator { - private final Iterator source; + private final Iterator source; - private RetryingDDBPathMetadataIterator( - final Iterator source) { + private RetryingIterator( + final Iterator source) { this.source = source; } + /** + * {@inheritDoc}. + * @throws WrappedIOException for IO failure, including throttling. + */ @Override @Retries.RetryTranslated public boolean hasNext() { @@ -2589,8 +2576,13 @@ public boolean hasNext() { } } + /** + * {@inheritDoc}. + * @throws WrappedIOException for IO failure, including throttling. + */ @Override - public DDBPathMetadata next() { + @Retries.RetryTranslated + public T next() { try { return scanOp.retry( "Scan Dynamo", diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java index b1d8f18d788b5..d2b6fc00a8f6d 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java @@ -34,6 +34,8 @@ import com.amazonaws.services.dynamodbv2.document.Table; import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughputDescription; import com.amazonaws.services.dynamodbv2.xspec.ExpressionSpecBuilder; +import org.assertj.core.api.Assertions; +import org.junit.AfterClass; import org.junit.Assume; import org.junit.FixMethodOrder; import org.junit.Test; @@ -49,6 +51,7 @@ import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.impl.WrappedIOException; import org.apache.hadoop.fs.s3a.AWSServiceThrottledException; +import org.apache.hadoop.fs.s3a.Invoker; import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3AStorageStatistics; @@ -62,7 +65,7 @@ import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.s3guard.MetadataStoreTestBase.basicFileStatus; import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.PARENT; -import static org.junit.Assume.*; +import static org.junit.Assume.assumeTrue; /** * Scale test for DynamoDBMetadataStore. @@ -91,6 +94,7 @@ public class ITestDynamoDBMetadataStoreScale private static final long MAXIMUM_READ_CAPACITY = 10; private static final long MAXIMUM_WRITE_CAPACITY = 15; + private static DynamoDBMetadataStore classMetastore; private DynamoDBMetadataStore ddbms; private DynamoDBMetadataStoreTableManager tableHandler; @@ -114,7 +118,7 @@ public class ITestDynamoDBMetadataStoreScale /** * Create the metadata store. The table and region are determined from * the attributes of the FS used in the tests. - * @return a new metadata store instance + * @return a new metadata store instanceFive * @throws IOException failure to instantiate * @throws AssumptionViolatedException if the FS isn't running S3Guard + DDB/ */ @@ -123,6 +127,9 @@ public MetadataStore createMetadataStore() throws IOException { S3AFileSystem fs = getFileSystem(); assumeTrue("S3Guard is disabled for " + fs.getUri(), fs.hasMetadataStore()); + if (classMetastore != null) { + return classMetastore; + } MetadataStore store = fs.getMetadataStore(); assumeTrue("Metadata store for " + fs.getUri() + " is " + store + " -not DynamoDBMetadataStore", @@ -145,15 +152,21 @@ public MetadataStore createMetadataStore() throws IOException { conf.set(S3GUARD_DDB_TABLE_NAME_KEY, tableName); conf.set(S3GUARD_DDB_REGION_KEY, region); conf.set(S3GUARD_DDB_THROTTLE_RETRY_INTERVAL, "50ms"); - conf.set(S3GUARD_DDB_MAX_RETRIES, "2"); + conf.set(S3GUARD_DDB_MAX_RETRIES, "1"); conf.set(MAX_ERROR_RETRIES, "1"); conf.set(S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY, "5ms"); DynamoDBMetadataStore ms = new DynamoDBMetadataStore(); - ms.initialize(conf, new S3Guard.TtlTimeProvider(conf)); + // init the metastore in a bigger retry loop than the test setup + // in case the previous test case overloaded things + final Invoker fsInvoker = fs.createStoreContext().getInvoker(); + fsInvoker.retry("init metastore", null, true, + () -> ms.initialize(conf, new S3Guard.TtlTimeProvider(conf))); // wire up the owner FS so that we can make assertions about throttle // events ms.bindToOwnerFilesystem(fs); + // and update the class value + classMetastore = ms; return ms; } @@ -168,44 +181,15 @@ public void setup() throws Exception { table = ddb.getTable(tableName); originalCapacity = table.describe().getProvisionedThroughput(); - // If you set the same provisioned I/O as already set it throws an - // exception, avoid that. + // is this table too big for throttling to surface? isOverProvisionedForTest = ( originalCapacity.getReadCapacityUnits() > MAXIMUM_READ_CAPACITY || originalCapacity.getWriteCapacityUnits() > MAXIMUM_WRITE_CAPACITY); } - @Override - public void teardown() throws Exception { - if (ddbms != null) { - S3GuardTableAccess tableAccess = new S3GuardTableAccess(ddbms); - ExpressionSpecBuilder builder = new ExpressionSpecBuilder(); - final String path = "/test/"; - builder.withCondition( - ExpressionSpecBuilder.S(PARENT).beginsWith(path)); - Iterable entries = - ddbms.wrapWithRetries(tableAccess.scanMetadata(builder)); - List list = new ArrayList<>(); - try { - entries.iterator().forEachRemaining(e -> { - Path p = e.getFileStatus().getPath(); - LOG.info("Deleting {}", p); - list.add(p); - }); - } catch (WrappedIOException e) { - // the iterator may have overloaded; swallow if so. - if (!(e.getCause() instanceof AWSServiceThrottledException)) { - throw e; - } - } - ddbms.getWriteOperationInvoker() - .retry("delete", - path, - true, - () -> tableAccess.delete(list)); - } - IOUtils.cleanupWithLogger(LOG, ddbms); - super.teardown(); + @AfterClass + public static void closeClassMetastore() { + IOUtils.cleanupWithLogger(LOG, classMetastore); } private boolean expectThrottling() { @@ -367,8 +351,9 @@ public void test_050_getVersionMarkerItem() throws Throwable { */ private void retryingDelete(final Path path) { try { - ddbms.getInvoker().retry("Delete ", path.toString(), true, - () -> ddbms.delete(path, null)); + ddbms.getWriteOperationInvoker() + .retry("Delete ", path.toString(), true, + () -> ddbms.delete(path, null)); } catch (IOException e) { LOG.warn("Failed to delete {}: ", path, e); } @@ -426,10 +411,11 @@ public void test_080_fullPathsToPut() throws Throwable { BulkOperationState.OperationType.Put, child)) { ddbms.put(new PathMetadata(makeDirStatus(base)), bulkUpdate); ddbms.put(new PathMetadata(makeDirStatus(child)), bulkUpdate); - ddbms.getInvoker().retry("set up directory tree", - base.toString(), - true, - () -> ddbms.put(pms, bulkUpdate)); + ddbms.getWriteOperationInvoker() + .retry("set up directory tree", + base.toString(), + true, + () -> ddbms.put(pms, bulkUpdate)); } try (BulkOperationState bulkUpdate = ddbms.initiateBulkWrite( @@ -483,6 +469,36 @@ public void test_100_forgetMetadata() throws Throwable { } } + @Test + public void test_200_delete_all_entries() throws Throwable { + describe("Delete all entries from the table"); + S3GuardTableAccess tableAccess = new S3GuardTableAccess(ddbms); + ExpressionSpecBuilder builder = new ExpressionSpecBuilder(); + final String path = "/test/"; + builder.withCondition( + ExpressionSpecBuilder.S(PARENT).beginsWith(path)); + Iterable entries = + ddbms.wrapWithRetries(tableAccess.scanMetadata(builder)); + List list = new ArrayList<>(); + try { + entries.iterator().forEachRemaining(e -> { + Path p = e.getFileStatus().getPath(); + LOG.info("Deleting {}", p); + list.add(p); + }); + } catch (WrappedIOException e) { + // the iterator may have overloaded; swallow if so. + if (!(e.getCause() instanceof AWSServiceThrottledException)) { + throw e; + } + } + ddbms.getWriteOperationInvoker() + .retry("delete", + path, + true, + () -> tableAccess.delete(list)); + } + @Test public void test_900_instrumentation() throws Throwable { describe("verify the owner FS gets updated after throttling events"); @@ -534,25 +550,29 @@ public ThrottleTracker execute(String operation, final ContractTestUtils.NanoTimer t = new ContractTestUtils.NanoTimer(); for (int j = 0; j < operationsPerThread; j++) { - if (tracker.isThrottlingDetected()) { + if (tracker.isThrottlingDetected() + || throttleExceptions.get() > 0) { outcome.skipped = true; return outcome; } try { action.call(); outcome.completed++; - } catch (AWSServiceThrottledException e) { - // this is possibly OK - LOG.info("Operation [{}] raised a throttled exception " + e, j, e); - LOG.debug(e.toString(), e); - throttleExceptions.incrementAndGet(); - // consider it completed - outcome.throttleExceptions.add(e); - outcome.throttled++; } catch (Exception e) { - LOG.error("Failed to execute {}", operation, e); - outcome.exceptions.add(e); - break; + if (e instanceof AWSServiceThrottledException + || (e.getCause() instanceof AWSServiceThrottledException)) { + // this is possibly OK + LOG.info("Operation [{}] raised a throttled exception " + e, j, e); + LOG.debug(e.toString(), e); + throttleExceptions.incrementAndGet(); + // consider it completed + outcome.throttleExceptions.add(e); + outcome.throttled++; + } else { + LOG.error("Failed to execute {}", operation, e); + outcome.exceptions.add(e); + break; + } } tracker.probe(); } @@ -569,9 +589,9 @@ public ThrottleTracker execute(String operation, LOG.info("Completed {} with {}", operation, tracker); LOG.info("time to execute: {} millis", elapsedMs); - for (Future future : futures) { - assertTrue("Future timed out", future.isDone()); - } + Assertions.assertThat(futures) + .describedAs("Futures of all tasks") + .allMatch(Future::isDone); tracker.probe(); if (expectThrottling) { From 8c4c3005f652733da04f08a4647b9a163102077c Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 30 Jan 2020 19:07:05 +0000 Subject: [PATCH 05/17] HADOOP-16283: throttling and retrying -DDB side tuning - Moving RetryingCollection to toplevel; - DDBMS.listChildren() unwraps IOEs raised in iterator. - throttling scale test happier if throttling doesn't surface in a test run as it may mean that the problem will surface later Change-Id: Ibf55e6ab257269b55230eedacae6a17586d91211 --- .../src/main/resources/core-default.xml | 5 +- .../org/apache/hadoop/fs/s3a/Constants.java | 4 +- .../fs/s3a/s3guard/DynamoDBMetadataStore.java | 139 ++++------------ .../s3a/s3guard/PurgeS3GuardDynamoTable.java | 13 +- .../fs/s3a/s3guard/RetryingCollection.java | 115 +++++++++++++ .../fs/s3a/s3guard/S3GuardTableAccess.java | 8 + .../ITestDynamoDBMetadataStoreScale.java | 155 ++++++++++-------- .../fs/s3a/s3guard/ThrottleTracker.java | 24 ++- 8 files changed, 269 insertions(+), 194 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RetryingCollection.java diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml index 20e3639783bd6..c236b968c6f9f 100644 --- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml +++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml @@ -1706,13 +1706,12 @@ fs.s3a.retry.throttle.interval - 5000s + 500ms Initial between retry attempts on throttled requests, +/- 50%. chosen at random. i.e. for an intial value of 3000ms, the initial delay would be in the range 1500ms to 4500ms. Backoffs are exponential; again randomness is used to avoid the thundering heard problem. - Given that throttling in S3 is per-second, very short delays will not initial spread - out work and so continue to create the problem. + 500ms is the default value used by the AWS S3 Retry policy. 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 6dfe73e486fc2..eda8a38c8003b 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 @@ -739,12 +739,12 @@ private Constants() { * Interval between retry attempts on throttled requests: {@value}. */ public static final String RETRY_THROTTLE_INTERVAL = - "fs.s3a.retry.throttle.interval"; + "fs.s3a.retry.throttle.interval"; /** * Default throttled retry interval: {@value}. */ - public static final String RETRY_THROTTLE_INTERVAL_DEFAULT = "5000ms"; + public static final String RETRY_THROTTLE_INTERVAL_DEFAULT = "500ms"; /** * Should etags be exposed as checksums? 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 9e01673b4cc02..42538cb7ba928 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 @@ -29,7 +29,6 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Objects; @@ -811,6 +810,7 @@ private S3AFileStatus makeDirStatus(String dirOwner, Path path) { public DirListingMetadata listChildren(final Path path) throws IOException { checkPath(path); LOG.debug("Listing table {} in region {}: {}", tableName, region, path); + final QuerySpec spec = new QuerySpec() .withHashKey(pathToParentKeyAttribute(path)) .withConsistentRead(true); // strictly consistent read @@ -822,9 +822,15 @@ public DirListingMetadata listChildren(final Path path) throws IOException { true, () -> table.query(spec)); // now wrap the result with retry logic - for (Item item : wrapWithRetries(items)) { - metas.add(itemToPathMetadata(item, username)); + try { + for (Item item : wrapWithRetries(items)) { + metas.add(itemToPathMetadata(item, username)); + } + } catch (WrappedIOException e) { + // failure in the iterators; unwrap. + throw e.getCause(); } + // Minor race condition here - if the path is deleted between // getting the list of items and the directory metadata we might // get a null in DDBPathMetadata. @@ -2071,6 +2077,26 @@ public long getBatchWriteCapacityExceededCount() { return batchWriteCapacityExceededEvents.get(); } + /** + * Get the operation invoker for write operations. + * @return an invoker for retrying mutating operations on a store. + */ + public Invoker getWriteOperationInvoker() { + return writeOp; + } + + /** + * Wrap an iterator returned from any scan with a retrying one. + * This includes throttle handling. + * Retries will update the relevant counters/metrics for scan operations. + * @param source source iterator + * @return a retrying iterator. + */ + public Iterable wrapWithRetries( + final Iterable source) { + return new RetryingCollection<>(scanOp, source); + } + /** * Record the number of records written. * @param count count of records. @@ -2488,111 +2514,4 @@ protected DynamoDBMetadataStoreTableManager getTableHandler() { Preconditions.checkNotNull(tableHandler, "Not initialized"); return tableHandler; } - - /** - * Get the operation invoker for write operations. - * @return an invoker for retrying mutating operations on a store. - */ - Invoker getWriteOperationInvoker() { - return writeOp; - } - - /** - * Wrap an iterator returned from any scan with a retrying one. - * This includes throttle handling. - * @param source source iterator - * @return a retrying iterator. - */ - Iterator wrapWithRetries( - final Iterator source) { - return new RetryingIterator<>(source); - } - - /** - * Wrap an iterator returned from any scan with a retrying one. - * This includes throttle handling. - * @param source source iterator - * @return a retrying iterator. - */ - Iterable wrapWithRetries( - final Iterable source) { - return new RetryingCollection<>(source); - } - - /** - * A collection which wraps the result of a query or scan - * with retries; the {@link #scanThrottleEvents} count is - * then updated. - * Important: iterate through this only once; the outcome - * of repeating an iteration is "undefined" - * @param type of outcome. - */ - private final class RetryingCollection - implements Iterable { - - private final Iterable source; - - private RetryingCollection( - final Iterable source) { - this.source = source; - } - - - @Override - public Iterator iterator() { - return wrapWithRetries(source.iterator()); - } - } - - /** - * An iterator which wraps a non-retrying iterator of scan results - * (i.e {@code S3GuardTableAccess.DDBPathMetadataIterator}. - */ - private final class RetryingIterator implements - Iterator { - - private final Iterator source; - - private RetryingIterator( - final Iterator source) { - this.source = source; - } - - /** - * {@inheritDoc}. - * @throws WrappedIOException for IO failure, including throttling. - */ - @Override - @Retries.RetryTranslated - public boolean hasNext() { - try { - return scanOp.retry( - "Scan Dynamo", - null, - true, - source::hasNext); - } catch (IOException e) { - throw new WrappedIOException(e); - } - } - - /** - * {@inheritDoc}. - * @throws WrappedIOException for IO failure, including throttling. - */ - @Override - @Retries.RetryTranslated - public T next() { - try { - return scanOp.retry( - "Scan Dynamo", - null, - true, - source::next); - } catch (IOException e) { - throw new WrappedIOException(e); - } - } - } - } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PurgeS3GuardDynamoTable.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PurgeS3GuardDynamoTable.java index d577a91d57c91..7caac5c140678 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PurgeS3GuardDynamoTable.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PurgeS3GuardDynamoTable.java @@ -172,11 +172,14 @@ public int execute() throws ServiceLaunchException, IOException { new DurationInfo(LOG, "deleting %s entries from %s", count, ddbms.toString()); - ddbms.getWriteOperationInvoker() - .retry("delete", - prefix, - true, - () -> tableAccess.delete(list)); + // sending this in one by one for more efficient retries + for (Path path: list) { + ddbms.getWriteOperationInvoker() + .retry("delete", + prefix, + true, + () -> tableAccess.delete(path)); + } duration.close(); long durationMillis = duration.value(); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RetryingCollection.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RetryingCollection.java new file mode 100644 index 0000000000000..36667db7b92bb --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RetryingCollection.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.s3guard; + +import java.io.IOException; +import java.util.Iterator; + +import org.apache.hadoop.fs.impl.WrappedIOException; +import org.apache.hadoop.fs.s3a.Invoker; +import org.apache.hadoop.fs.s3a.Retries; + +/** + * A collection which wraps the result of a query or scan + * with retries; the {@link #scanThrottleEvents} count is + * then updated. + * Important: iterate through this only once; the outcome + * of repeating an iteration is "undefined" + * @param type of outcome. + */ +class RetryingCollection + implements Iterable { + + /** + * Source iterable. + */ + private final Iterable source; + + /** + * Invoker for retries. + */ + private Invoker invoker; + + RetryingCollection( + final Invoker invoker, + final Iterable source) { + this.source = source; + this.invoker = invoker; + } + + + @Override + public Iterator iterator() { + return new RetryingIterator<>(invoker, source.iterator()); + } + + /** + * An iterator which wraps a non-retrying iterator of scan results + * (i.e {@code S3GuardTableAccess.DDBPathMetadataIterator}. + */ + private static final class RetryingIterator implements Iterator { + + private final Iterator source; + + private Invoker scanOp; + + private RetryingIterator(final Invoker scanOp, + final Iterator source) { + this.source = source; + this.scanOp = scanOp; + } + + /** + * {@inheritDoc}. + * @throws WrappedIOException for IO failure, including throttling. + */ + @Override + @Retries.RetryTranslated + public boolean hasNext() { + try { + return scanOp.retry( + "Scan Dynamo", + null, + true, + source::hasNext); + } catch (IOException e) { + throw new WrappedIOException(e); + } + } + + /** + * {@inheritDoc}. + * @throws WrappedIOException for IO failure, including throttling. + */ + @Override + @Retries.RetryTranslated + public T next() { + try { + return scanOp.retry( + "Scan Dynamo", + null, + true, + source::next); + } catch (IOException e) { + throw new WrappedIOException(e); + } + } + } + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTableAccess.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTableAccess.java index 0f33f4eb51c4d..7e8413b3dee7a 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTableAccess.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTableAccess.java @@ -45,6 +45,7 @@ import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.PARENT; import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.TABLE_VERSION; import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.itemToPathMetadata; +import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.pathToKey; /** * Package-scoped accessor to table state in S3Guard. @@ -143,6 +144,11 @@ void delete(Collection paths) { .forEach(table::deleteItem); } + @Retries.OnceRaw + void delete(Path path) { + table.deleteItem(pathToKey(path)); + } + /** * A collection which wraps the result of a query or scan. * Important: iterate through this only once; the outcome @@ -198,11 +204,13 @@ private DDBPathMetadataIterator(final IteratorSupport it) { } @Override + @Retries.OnceRaw public boolean hasNext() { return it.hasNext(); } @Override + @Retries.OnceRaw public DDBPathMetadata next() { Item item = it.next(); Pair key = primaryKey(item); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java index d2b6fc00a8f6d..6829d5f374b3b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java @@ -19,6 +19,7 @@ package org.apache.hadoop.fs.s3a.s3guard; import javax.annotation.Nullable; +import java.io.FileNotFoundException; import java.io.IOException; import java.util.ArrayList; import java.util.Collections; @@ -35,7 +36,6 @@ import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughputDescription; import com.amazonaws.services.dynamodbv2.xspec.ExpressionSpecBuilder; import org.assertj.core.api.Assertions; -import org.junit.AfterClass; import org.junit.Assume; import org.junit.FixMethodOrder; import org.junit.Test; @@ -55,6 +55,7 @@ import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3AStorageStatistics; +import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.Statistic; import org.apache.hadoop.fs.s3a.scale.AbstractITestS3AMetadataStoreScale; import org.apache.hadoop.io.IOUtils; @@ -94,7 +95,20 @@ public class ITestDynamoDBMetadataStoreScale private static final long MAXIMUM_READ_CAPACITY = 10; private static final long MAXIMUM_WRITE_CAPACITY = 15; - private static DynamoDBMetadataStore classMetastore; + /** + * Time in milliseconds to sleep after a test throttled: + * {@value}. + * This is to help isolate throttling to the test which failed, + * rather than have it surface in a followup test. + * Also the test reports will record durations more accurately, + * as JUnit doesn't include setup/teardown times in its reports. + * There's a cost: single test runs will sleep, and the last test + * run may throttle when it doesn't need to. + * The last test {}@link {@link #test_999_delete_all_entries()} + * doesn't do the sleep so a full batch run should not suffer here. + */ + public static final int THROTTLE_RECOVER_TIME_MILLIS = 5_000; + private DynamoDBMetadataStore ddbms; private DynamoDBMetadataStoreTableManager tableHandler; @@ -118,18 +132,15 @@ public class ITestDynamoDBMetadataStoreScale /** * Create the metadata store. The table and region are determined from * the attributes of the FS used in the tests. - * @return a new metadata store instanceFive + * @return a new metadata store instance * @throws IOException failure to instantiate * @throws AssumptionViolatedException if the FS isn't running S3Guard + DDB/ */ @Override - public MetadataStore createMetadataStore() throws IOException { + public DynamoDBMetadataStore createMetadataStore() throws IOException { S3AFileSystem fs = getFileSystem(); assumeTrue("S3Guard is disabled for " + fs.getUri(), fs.hasMetadataStore()); - if (classMetastore != null) { - return classMetastore; - } MetadataStore store = fs.getMetadataStore(); assumeTrue("Metadata store for " + fs.getUri() + " is " + store + " -not DynamoDBMetadataStore", @@ -165,8 +176,6 @@ public MetadataStore createMetadataStore() throws IOException { // wire up the owner FS so that we can make assertions about throttle // events ms.bindToOwnerFilesystem(fs); - // and update the class value - classMetastore = ms; return ms; } @@ -187,15 +196,29 @@ public void setup() throws Exception { || originalCapacity.getWriteCapacityUnits() > MAXIMUM_WRITE_CAPACITY); } - @AfterClass - public static void closeClassMetastore() { - IOUtils.cleanupWithLogger(LOG, classMetastore); + @Override + public void teardown() throws Exception { + IOUtils.cleanupWithLogger(LOG, ddbms); + super.teardown(); } + /** + * Is throttling likely? + * @return true if the DDB table has prepaid IO and is small enough to throttle. + */ private boolean expectThrottling() { return !isOverProvisionedForTest && !isOnDemandTable; } + /** + * Recover from throttling by sleeping briefly. + */ + private void recoverFromThrottling() throws InterruptedException { + LOG.info("Sleeping to recover from throttling for {} ms", + THROTTLE_RECOVER_TIME_MILLIS); + Thread.sleep(THROTTLE_RECOVER_TIME_MILLIS); + } + /** * The subclass expects the superclass to be throttled; sometimes it is. */ @@ -298,8 +321,8 @@ public void test_030_BatchedWrite() throws Exception { } } }); - if (expectThrottling()) { - result.assertThrottlingDetected(); + if (expectThrottling() && result.probeThrottlingDetected()) { + recoverFromThrottling(); } } finally { describe("Cleaning up table %s", tableName); @@ -340,7 +363,12 @@ public void test_050_getVersionMarkerItem() throws Throwable { execute("get", OPERATIONS_PER_THREAD * 2, expectThrottling(), - () -> tableHandler.getVersionMarkerItem() + () -> { + try { + tableHandler.getVersionMarkerItem(); + } catch (FileNotFoundException ignored) { + } + } ); } @@ -351,9 +379,8 @@ public void test_050_getVersionMarkerItem() throws Throwable { */ private void retryingDelete(final Path path) { try { - ddbms.getWriteOperationInvoker() - .retry("Delete ", path.toString(), true, - () -> ddbms.delete(path, null)); + ddbms.getWriteOperationInvoker().retry("Delete ", path.toString(), true, + () -> ddbms.delete(path, null)); } catch (IOException e) { LOG.warn("Failed to delete {}: ", path, e); } @@ -411,11 +438,10 @@ public void test_080_fullPathsToPut() throws Throwable { BulkOperationState.OperationType.Put, child)) { ddbms.put(new PathMetadata(makeDirStatus(base)), bulkUpdate); ddbms.put(new PathMetadata(makeDirStatus(child)), bulkUpdate); - ddbms.getWriteOperationInvoker() - .retry("set up directory tree", - base.toString(), - true, - () -> ddbms.put(pms, bulkUpdate)); + ddbms.getWriteOperationInvoker().retry("set up directory tree", + base.toString(), + true, + () -> ddbms.put(pms, bulkUpdate)); } try (BulkOperationState bulkUpdate = ddbms.initiateBulkWrite( @@ -470,7 +496,27 @@ public void test_100_forgetMetadata() throws Throwable { } @Test - public void test_200_delete_all_entries() throws Throwable { + public void test_900_instrumentation() throws Throwable { + describe("verify the owner FS gets updated after throttling events"); + Assume.assumeTrue("No throttling expected", expectThrottling()); + // we rely on the FS being shared + S3AFileSystem fs = getFileSystem(); + String fsSummary = fs.toString(); + + S3AStorageStatistics statistics = fs.getStorageStatistics(); + for (StorageStatistics.LongStatistic statistic : statistics) { + LOG.info("{}", statistic.toString()); + } + String retryKey = Statistic.S3GUARD_METADATASTORE_RETRY.getSymbol(); + assertTrue("No increment of " + retryKey + " in " + fsSummary, + statistics.getLong(retryKey) > 0); + String throttledKey = Statistic.S3GUARD_METADATASTORE_THROTTLED.getSymbol(); + assertTrue("No increment of " + throttledKey + " in " + fsSummary, + statistics.getLong(throttledKey) > 0); + } + + @Test + public void test_999_delete_all_entries() throws Throwable { describe("Delete all entries from the table"); S3GuardTableAccess tableAccess = new S3GuardTableAccess(ddbms); ExpressionSpecBuilder builder = new ExpressionSpecBuilder(); @@ -492,31 +538,14 @@ public void test_200_delete_all_entries() throws Throwable { throw e; } } - ddbms.getWriteOperationInvoker() - .retry("delete", - path, - true, - () -> tableAccess.delete(list)); - } - - @Test - public void test_900_instrumentation() throws Throwable { - describe("verify the owner FS gets updated after throttling events"); - Assume.assumeTrue("No throttling expected", expectThrottling()); - // we rely on the FS being shared - S3AFileSystem fs = getFileSystem(); - String fsSummary = fs.toString(); - - S3AStorageStatistics statistics = fs.getStorageStatistics(); - for (StorageStatistics.LongStatistic statistic : statistics) { - LOG.info("{}", statistic.toString()); + // sending this in one by one for more efficient retries + for (Path p : list) { + ddbms.getWriteOperationInvoker() + .retry("delete", + path, + true, + () -> tableAccess.delete(p)); } - String retryKey = Statistic.S3GUARD_METADATASTORE_RETRY.getSymbol(); - assertTrue("No increment of " + retryKey + " in " + fsSummary, - statistics.getLong(retryKey) > 0); - String throttledKey = Statistic.S3GUARD_METADATASTORE_THROTTLED.getSymbol(); - assertTrue("No increment of " + throttledKey + " in " + fsSummary, - statistics.getLong(throttledKey) > 0); } /** @@ -558,21 +587,18 @@ public ThrottleTracker execute(String operation, try { action.call(); outcome.completed++; + } catch (AWSServiceThrottledException e) { + // this is possibly OK + LOG.info("Operation [{}] raised a throttled exception " + e, j, e); + LOG.debug(e.toString(), e); + throttleExceptions.incrementAndGet(); + // consider it completed + outcome.throttleExceptions.add(e); + outcome.throttled++; } catch (Exception e) { - if (e instanceof AWSServiceThrottledException - || (e.getCause() instanceof AWSServiceThrottledException)) { - // this is possibly OK - LOG.info("Operation [{}] raised a throttled exception " + e, j, e); - LOG.debug(e.toString(), e); - throttleExceptions.incrementAndGet(); - // consider it completed - outcome.throttleExceptions.add(e); - outcome.throttled++; - } else { - LOG.error("Failed to execute {}", operation, e); - outcome.exceptions.add(e); - break; - } + LOG.error("Failed to execute {}", operation, e); + outcome.exceptions.add(e); + break; } tracker.probe(); } @@ -593,9 +619,8 @@ public ThrottleTracker execute(String operation, .describedAs("Futures of all tasks") .allMatch(Future::isDone); tracker.probe(); - - if (expectThrottling) { - tracker.assertThrottlingDetected(); + if (expectThrottling() && tracker.probeThrottlingDetected()) { + recoverFromThrottling(); } for (Future future : futures) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ThrottleTracker.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ThrottleTracker.java index 942d7fecaff30..4c2f2fd09eab2 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ThrottleTracker.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ThrottleTracker.java @@ -18,7 +18,8 @@ package org.apache.hadoop.fs.s3a.s3guard; -import org.junit.Assert; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Something to track throttles in DynamoDB metastores. @@ -34,13 +35,15 @@ */ class ThrottleTracker { + private static final Logger LOG = LoggerFactory.getLogger( + ThrottleTracker.class); private final DynamoDBMetadataStore ddbms; - private long writeThrottleEventOrig = 0; + private long writeThrottleEventOrig; - private long readThrottleEventOrig = 0; + private long readThrottleEventOrig; - private long batchWriteThrottleCountOrig = 0; + private long batchWriteThrottleCountOrig; private long scanThrottleCountOrig; @@ -102,12 +105,15 @@ public String toString() { } /** - * Assert that throttling has been detected. + * Check that throttling was detected; Warn if not. + * @return true if throttling took place. */ - public void assertThrottlingDetected() { - Assert.assertTrue("No throttling detected in " + this + - " against " + ddbms.toString(), - isThrottlingDetected()); + public boolean probeThrottlingDetected() { + if (!isThrottlingDetected()) { + LOG.warn("No throttling detected in {} against {}", this, ddbms.toString()); + return false; + } + return true; } /** From faae65b7ee391eef9dd753c2978e0e6dc281bc96 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 31 Jan 2020 18:55:10 +0000 Subject: [PATCH 06/17] HADOOP-16823 load test for throttling. Implies that from the (remote) test machine multiple smaller bulk deletes are handled better than few large ones from a retry perspective. Also imply: we should make sure that the backoff strategy we use in our code doesn't back off over-aggressively Proposed: retryUpToMaximumCountWithProportionalSleep rather than exponential for throttling oh, and the throttle detection code doesn't seem to be updating counters here... Change-Id: I163d128aa5ad5c203ade66bd4f049d3357d6a9d4 --- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 2 +- .../ILoadTestS3ABulkDeleteThrottling.java | 282 ++++++++++++++++++ 2 files changed, 283 insertions(+), 1 deletion(-) create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ILoadTestS3ABulkDeleteThrottling.java 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 d48814e00ac36..86f5d6dca53b9 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 @@ -2272,7 +2272,7 @@ private void noteDeleted(final int count, final boolean deleteFakeDir) { */ @VisibleForTesting @Retries.RetryMixed - void removeKeys( + public void removeKeys( final List keysToDelete, final boolean deleteFakeDir, final BulkOperationState operationState) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ILoadTestS3ABulkDeleteThrottling.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ILoadTestS3ABulkDeleteThrottling.java new file mode 100644 index 0000000000000..219a3de80c4a3 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ILoadTestS3ABulkDeleteThrottling.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.scale; + +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletionService; +import java.util.concurrent.ExecutorCompletionService; +import java.util.concurrent.ExecutorService; + +import com.amazonaws.services.s3.model.DeleteObjectsRequest; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.junit.Assume; +import org.junit.FixMethodOrder; +import org.junit.Test; +import org.junit.runners.MethodSorters; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.impl.FunctionsRaisingIOE; +import org.apache.hadoop.fs.impl.WrappedIOException; +import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.fs.s3a.auth.delegation.Csvout; +import org.apache.hadoop.test.GenericTestUtils; +import org.apache.hadoop.util.concurrent.HadoopExecutors; + +import static org.apache.hadoop.fs.s3a.Constants.ENABLE_MULTI_DELETE; + +/** + * Test some scalable operations related to file renaming and deletion. + * Much of the setup code is lifted from ILoadTestSessionCredentials; + * whereas that was designed to overload an STS endpoint, this just + * tries to overload a single S3 shard with too many bulk IO requests + * -and so see what happens. + */ +@FixMethodOrder(MethodSorters.NAME_ASCENDING) +public class ILoadTestS3ABulkDeleteThrottling extends S3AScaleTestBase { + + private static final Logger LOG = + LoggerFactory.getLogger(ILoadTestS3ABulkDeleteThrottling.class); + + protected static final int THREADS = 20; + + private static boolean previousTestThrottled; + + private final ExecutorService executor = + HadoopExecutors.newFixedThreadPool( + THREADS, + new ThreadFactoryBuilder() + .setNameFormat(" #%d") + .build()); + + private final CompletionService + completionService = new ExecutorCompletionService<>(executor); + + private File dataDir; + + @Override + public void setup() throws Exception { + super.setup(); + Assume.assumeTrue("multipart delete disabled", + getConf().getBoolean(ENABLE_MULTI_DELETE, true)); + dataDir = GenericTestUtils.getTestDir("throttling"); + dataDir.mkdirs(); + } + + @Test + public void test_010_DeleteThrottlingSmall() throws Throwable { + describe("test how S3 reacts to massive multipart deletion requests"); + maybeSleep(); + final File results = deleteFiles(100, 200); + LOG.info("Test run completed against {}:\n see {}", getFileSystem(), + results); + } + + @Test + public void test_011_sleep() throws Throwable { + maybeSleep(); + } + + @Test + public void test_020_DeleteThrottlingBigFiles() throws Throwable { + describe("test how S3 reacts to massive multipart deletion requests"); + final File results = deleteFiles(20, 1000); + LOG.info("Test run completed against {}:\n see {}", getFileSystem(), + results); + previousTestThrottled = true; + } + + private void maybeSleep() throws InterruptedException { + if (previousTestThrottled) { + LOG.info("Sleeping briefly to let store recover"); + Thread.sleep(30_000); + previousTestThrottled = false; + } + } + /** + * Fetch requests. + * @param requests number of requests. + * @throws Exception failure + * @return + */ + private File deleteFiles(final int requests, final int entries) + throws Exception { + File csvFile = new File(dataDir, + String.format("delete-%03d-%04d.csv", requests, entries)); + describe("Issuing %d requests of size %d, saving log to %s", + requests, entries, csvFile); + Path basePath = path("testDeleteObjectThrottling"); + final S3AFileSystem fs = getFileSystem(); + final String base = fs.pathToKey(basePath); + final List fileList + = buildDeleteRequest(base, entries); + final FileWriter out = new FileWriter(csvFile); + Csvout csvout = new Csvout(out, "\t", "\n"); + Outcome.writeSchema(csvout); + + + final ContractTestUtils.NanoTimer jobTimer = + new ContractTestUtils.NanoTimer(); + + + for (int i = 0; i < requests; i++) { + final int id = i; + completionService.submit(() -> { + final long startTime = System.currentTimeMillis(); + Thread.currentThread().setName("#" + id); + LOG.info("Issuing request {}", id); + final ContractTestUtils.NanoTimer timer = + new ContractTestUtils.NanoTimer(); + Exception ex = null; + try { + fs.removeKeys(fileList, false, null); + } catch (IOException e) { + ex = e; + } + timer.end("Request " + id); + return new Outcome(id, startTime, timer, + ex); + }); + } + + NanoTimerStats stats = new NanoTimerStats("Overall"); + NanoTimerStats success = new NanoTimerStats("Successful"); + NanoTimerStats throttled = new NanoTimerStats("Throttled"); + List throttledEvents = new ArrayList<>(); + for (int i = 0; i < requests; i++) { + Outcome outcome = completionService.take().get(); + ContractTestUtils.NanoTimer timer = outcome.timer; + Exception ex = outcome.exception; + outcome.writeln(csvout); + stats.add(timer); + if (ex != null) { + // throttling event occurred. + LOG.info("Throttled at event {}", i, ex); + throttled.add(timer); + throttledEvents.add(outcome); + } else { + success.add(timer); + } + } + + csvout.close(); + + jobTimer.end("Execution of operations"); + // now print the stats + LOG.info("Summary file is " + csvFile); + LOG.info("Made {} requests with {} throttle events\n: {}\n{}\n{}", + requests, + throttled.getCount(), + stats, + throttled, + success); + + double duration = jobTimer.duration(); + double iops = requests * entries * 1.0e9 / duration; + LOG.info(String.format("TPS %3f operations/second", + iops)); + // log at debug + if (LOG.isDebugEnabled()) { + throttledEvents.stream().forEach((outcome -> { + LOG.debug("{}: duration: {}", + outcome.id, outcome.timer.elapsedTimeMs()); + })); + } + return csvFile; + } + + + private List buildDeleteRequest( + String base, int count) { + List request = new ArrayList<>(count); + for (int i = 0; i < count; i++) { + request.add(new DeleteObjectsRequest.KeyVersion( + String.format("%s/file-%04d", base, i))); + } + return request; + } + + + private R wrap(FunctionsRaisingIOE.CallableRaisingIOE callable) { + try { + return callable.apply(); + } catch (IOException e) { + throw new WrappedIOException(e); + } + } + + /** + * Outcome of one of the load operations. + */ + private static class Outcome { + + private final int id; + + private final long startTime; + + private final ContractTestUtils.NanoTimer timer; + + private final Exception exception; + + Outcome(final int id, + final long startTime, + final ContractTestUtils.NanoTimer timer, + final Exception exception) { + this.id = id; + this.startTime = startTime; + this.timer = timer; + this.exception = exception; + } + + /** + * Write this record. + * @param out the csvout to write through. + * @return the csvout instance + * @throws IOException IO failure. + */ + public Csvout writeln(Csvout out) throws IOException { + return out.write( + id, + startTime, + exception == null ? 1 : 0, + timer.getStartTime(), + timer.getEndTime(), + timer.duration(), + '"' + (exception == null ? "" : exception.getMessage()) + '"') + .newline(); + } + + /** + * Write the schema of the outcome records. + * @param out CSV destinatin + * @throws IOException IO failure. + */ + public static void writeSchema(Csvout out) throws IOException { + out.write("id", "starttime", "success", "started", "ended", + "duration", "error").newline(); + } + } + +} From 6c49f4c9d716095d2b96c0cd33b66ef1ff469d87 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 3 Feb 2020 12:09:58 +0000 Subject: [PATCH 07/17] HADOOP-16283 tightening retry timeouts -trying to set up an explicit callback on retries in bulk deletes Change-Id: I456680bbbedf3f135508ae3960e83eb1baefbfc6 --- .../hadoop-common/src/main/resources/core-default.xml | 2 +- .../main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java | 5 +++++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml index c236b968c6f9f..fe0f8cc5407d3 100644 --- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml +++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml @@ -1706,7 +1706,7 @@ fs.s3a.retry.throttle.interval - 500ms + 100ms Initial between retry attempts on throttled requests, +/- 50%. chosen at random. i.e. for an intial value of 3000ms, the initial delay would be in the range 1500ms to 4500ms. 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 86f5d6dca53b9..00e9649255bc4 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 @@ -1950,6 +1950,11 @@ private DeleteObjectsResult deleteObjects(DeleteObjectsRequest deleteRequest) deleteRequest.getKeys().size())) { return invoker.retryUntranslated("delete", DELETE_CONSIDERED_IDEMPOTENT, + (text, e, r, i) -> { + // error triggering retry + operationRetried(e); + + }, () -> { incrementStatistic(OBJECT_DELETE_REQUESTS, 1); return s3.deleteObjects(deleteRequest); From c16a2b65c98fa6a0bc3f49b5826b4a046b6525b4 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 3 Feb 2020 19:29:46 +0000 Subject: [PATCH 08/17] HADOOP-16283 page size controllable * pulling out retry logic into its own class; only using this in delete() * tests are parameterized configured page size coming in as 0; no idea why not. To debug Change-Id: I7e45b897c0a8d09167e6e6148a8f3930f31ec5b0 --- .../org/apache/hadoop/fs/s3a/Constants.java | 53 ++++- .../hadoop/fs/s3a/DefaultS3ClientFactory.java | 6 +- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 14 +- .../hadoop/fs/s3a/impl/DeleteOperation.java | 2 +- .../hadoop/fs/s3a/impl/InternalConstants.java | 4 + .../hadoop/fs/s3a/impl/RenameOperation.java | 10 +- .../fs/s3a/impl/RetryCallbackHandler.java | 191 ++++++++++++++++++ .../site/markdown/tools/hadoop-aws/testing.md | 16 ++ .../ILoadTestS3ABulkDeleteThrottling.java | 132 +++++++++--- 9 files changed, 394 insertions(+), 34 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RetryCallbackHandler.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 eda8a38c8003b..f22685489ff12 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 @@ -174,10 +174,34 @@ private Constants() { public static final String PROXY_DOMAIN = "fs.s3a.proxy.domain"; public static final String PROXY_WORKSTATION = "fs.s3a.proxy.workstation"; - // number of times we should retry errors + /** + * Number of times the AWS client library should retry errors before + * escalating to the S3A code: {@value}. + */ public static final String MAX_ERROR_RETRIES = "fs.s3a.attempts.maximum"; + + /** + * Default number of times the AWS client library should retry errors before + * escalating to the S3A code: {@value}. + */ public static final int DEFAULT_MAX_ERROR_RETRIES = 10; + + /** + * Experimental/Unstable feature: should the AWS client library retry + * throttle responses before escalating to the S3A code: {@value}. + * + * When set to false, the S3A connector sees all S3 throttle events, + * And so can update it counters and the metrics, and use its own retry + * policy. + * However, this may have adverse effects on some operations where the S3A + * code cannot retry as efficiently as the AWS client library. + * + * This only applies to S3 operations, not to DynamoDB or other services. + */ + public static final String AWS_INTERNAL_THROTTLING = + "fs.s3a.experimental.aws.internal.throttling"; + // seconds until we give up trying to establish a connection to s3 public static final String ESTABLISH_TIMEOUT = "fs.s3a.connection.establish.timeout"; @@ -225,6 +249,33 @@ private Constants() { public static final String ENABLE_MULTI_DELETE = "fs.s3a.multiobjectdelete.enable"; + /** + * Number of objects to delete in a single multi-object delete {@value}. + * Max: 1000. + * + * A bigger value it means fewer POST requests when deleting a directory + * tree with many objects. + * However, as you are limited to only a a few thousand requests per + * second against a single partition of an S3 bucket, + * a large page size can easily overload the bucket and so trigger + * throttling. + * + * Furthermore, as the reaction to this request is being throttled + * is simply to retry it -it can take a while for the situation to go away. + * While a large value may give better numbers on tests and benchmarks + * where only a single operations being executed, once multiple + * applications start working with the same bucket these large + * deletes can be highly disruptive. + */ + public static final String BULK_DELETE_PAGE_SIZE = + "fs.s3a.bulk.delete.page.size"; + + /** + * Default Number of objects to delete in a single multi-object + * delete: {@value}. + */ + public static final int BULK_DELETE_PAGE_SIZE_DEFAULT = 200; + // comma separated list of directories public static final String BUFFER_DIR = "fs.s3a.buffer.dir"; 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 1b833c5bde47e..5f232a47bfda2 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 @@ -34,6 +34,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; +import static org.apache.hadoop.fs.s3a.Constants.AWS_INTERNAL_THROTTLING; import static org.apache.hadoop.fs.s3a.Constants.ENDPOINT; import static org.apache.hadoop.fs.s3a.Constants.PATH_STYLE_ACCESS; @@ -56,11 +57,12 @@ public AmazonS3 createS3Client(URI name, final String userAgentSuffix) throws IOException { Configuration conf = getConf(); final ClientConfiguration awsConf = S3AUtils - .createAwsConf(getConf(), bucket, Constants.AWS_SERVICE_IDENTIFIER_S3); + .createAwsConf(conf, bucket, Constants.AWS_SERVICE_IDENTIFIER_S3); // throttling is explicitly disabled on the S3 client so that // all failures are collected - awsConf.setUseThrottleRetries(false); + awsConf.setUseThrottleRetries( + conf.getBoolean(AWS_INTERNAL_THROTTLING, true)); if (!StringUtils.isEmpty(userAgentSuffix)) { awsConf.setUserAgentSuffix(userAgentSuffix); 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 00e9649255bc4..31cdab7503342 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 @@ -107,6 +107,7 @@ 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.RetryCallbackHandler; import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum; import org.apache.hadoop.fs.s3a.impl.StoreContext; import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; @@ -170,6 +171,7 @@ import static org.apache.hadoop.fs.s3a.auth.RolePolicies.allowS3Operations; import static org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens.TokenIssuingPolicy.NoTokensAvailable; import static org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens.hasDelegationTokenBinding; +import static org.apache.hadoop.fs.s3a.impl.InternalConstants.MAX_ENTRIES_TO_DELETE; 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.io.IOUtils.cleanupWithLogger; @@ -1388,7 +1390,9 @@ private long innerRename(Path source, Path dest) createStoreContext(), src, srcKey, p.getLeft(), dst, dstKey, p.getRight(), - operationCallbacks); + operationCallbacks, + intOption(getConf(), BULK_DELETE_PAGE_SIZE, + BULK_DELETE_PAGE_SIZE_DEFAULT, 0)); return renameOperation.execute(); } @@ -1945,6 +1949,8 @@ private void blockRootDelete(String key) throws InvalidRequestException { private DeleteObjectsResult deleteObjects(DeleteObjectsRequest deleteRequest) throws MultiObjectDeleteException, AmazonClientException, IOException { incrementWriteOperations(); + RetryCallbackHandler retryHandler = + new RetryCallbackHandler(createStoreContext()); try(DurationInfo ignored = new DurationInfo(LOG, false, "DELETE %d keys", deleteRequest.getKeys().size())) { @@ -1952,8 +1958,7 @@ private DeleteObjectsResult deleteObjects(DeleteObjectsRequest deleteRequest) DELETE_CONSIDERED_IDEMPOTENT, (text, e, r, i) -> { // error triggering retry - operationRetried(e); - + retryHandler.bulkDeleteRetried(deleteRequest, e); }, () -> { incrementStatistic(OBJECT_DELETE_REQUESTS, 1); @@ -2372,7 +2377,8 @@ public boolean delete(Path f, boolean recursive) throws IOException { innerGetFileStatus(f, true, StatusProbeEnum.ALL), recursive, operationCallbacks, - InternalConstants.MAX_ENTRIES_TO_DELETE); + intOption(getConf(), BULK_DELETE_PAGE_SIZE, + BULK_DELETE_PAGE_SIZE_DEFAULT, 0)); boolean outcome = deleteOperation.execute(); if (outcome) { try { 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 4ff1f8223b2df..49c5b313d0ff6 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 @@ -200,7 +200,7 @@ public DeleteOperation(final StoreContext context, this.callbacks = callbacks; checkArgument(pageSize > 0 && pageSize <= InternalConstants.MAX_ENTRIES_TO_DELETE, - "page size out of range: %d", pageSize); + "page size out of range: %s", pageSize); this.pageSize = pageSize; metadataStore = context.getMetadataStore(); executor = context.createThrottledExecutor(1); 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 82250afc05ecf..608617c592a66 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 @@ -85,4 +85,8 @@ private InternalConstants() { /** 404 error code. */ public static final int SC_404 = 404; + + /* Name of the log for throttling events. */ + public static final String THROTTLE_LOG_NAME = + "org.apache.hadoop.fs.s3a.throttled"; } 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 a3561072c12df..b89c0fba82ef3 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 @@ -50,7 +50,6 @@ 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.s3a.impl.InternalConstants.DEFAULT_BLOCKSIZE; -import static org.apache.hadoop.fs.s3a.impl.InternalConstants.MAX_ENTRIES_TO_DELETE; import static org.apache.hadoop.fs.s3a.impl.InternalConstants.RENAME_PARALLEL_LIMIT; /** @@ -101,6 +100,8 @@ public class RenameOperation extends ExecutingStoreOperation { */ private final AtomicLong bytesCopied = new AtomicLong(); + private final int pageSize; + /** * Rename tracker. */ @@ -137,6 +138,7 @@ public class RenameOperation extends ExecutingStoreOperation { * @param destKey destination key * @param destStatus destination status. * @param callbacks callback provider + * @param pageSize size of delete pages */ public RenameOperation( final StoreContext storeContext, @@ -146,7 +148,8 @@ public RenameOperation( final Path destPath, final String destKey, final S3AFileStatus destStatus, - final OperationCallbacks callbacks) { + final OperationCallbacks callbacks, + final int pageSize) { super(storeContext); this.sourcePath = sourcePath; this.sourceKey = sourceKey; @@ -157,6 +160,7 @@ public RenameOperation( this.callbacks = callbacks; blocksize = storeContext.getConfiguration() .getLongBytes(FS_S3A_BLOCK_SIZE, DEFAULT_BLOCKSIZE); + this.pageSize = pageSize; } /** @@ -360,7 +364,7 @@ protected void recursiveDirectoryRename() throws IOException { LOG.debug("Waiting for active copies to complete"); completeActiveCopies("batch threshold reached"); } - if (keysToDelete.size() == MAX_ENTRIES_TO_DELETE) { + if (keysToDelete.size() == pageSize) { // finish ongoing copies then delete all queued keys. // provided the parallel limit is a factor of the max entry // constant, this will not need to block for the copy, and diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RetryCallbackHandler.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RetryCallbackHandler.java new file mode 100644 index 0000000000000..5e18b2925e4d2 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RetryCallbackHandler.java @@ -0,0 +1,191 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl; + +import com.amazonaws.services.s3.model.DeleteObjectsRequest; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.s3a.Invoker; +import org.apache.hadoop.fs.s3a.S3AInstrumentation; +import org.apache.hadoop.fs.s3a.S3AStorageStatistics; +import org.apache.hadoop.fs.s3a.Statistic; + +import static org.apache.hadoop.fs.s3a.S3AUtils.isThrottleException; +import static org.apache.hadoop.fs.s3a.Statistic.IGNORED_ERRORS; +import static org.apache.hadoop.fs.s3a.Statistic.S3GUARD_METADATASTORE_RETRY; +import static org.apache.hadoop.fs.s3a.Statistic.S3GUARD_METADATASTORE_THROTTLED; +import static org.apache.hadoop.fs.s3a.Statistic.S3GUARD_METADATASTORE_THROTTLE_RATE; +import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_THROTTLED; +import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_THROTTLE_RATE; +import static org.apache.hadoop.fs.s3a.impl.InternalConstants.THROTTLE_LOG_NAME; + +/** + * Handler for retry callbacks. + * This is intended to be pulled out for all operations, but for now it is only + * used in delete objects. + */ +public class RetryCallbackHandler extends AbstractStoreOperation { + + + private static final Logger LOG = LoggerFactory.getLogger( + RetryCallbackHandler.class); + + private static final Logger THROTTLE_LOG = LoggerFactory.getLogger( + THROTTLE_LOG_NAME); + + private final S3AInstrumentation instrumentation; + + private final S3AStorageStatistics storageStatistics; + + public RetryCallbackHandler(final StoreContext storeContext) { + super(storeContext); + instrumentation = storeContext.getInstrumentation(); + storageStatistics = storeContext.getStorageStatistics(); + } + + /** + * Increment a statistic by 1. + * This increments both the instrumentation and storage statistics. + * @param statistic The operation to increment + */ + protected void incrementStatistic(Statistic statistic) { + incrementStatistic(statistic, 1); + } + + /** + * Increment a statistic by a specific value. + * This increments both the instrumentation and storage statistics. + * @param statistic The operation to increment + * @param count the count to increment + */ + protected void incrementStatistic(Statistic statistic, long count) { + instrumentation.incrementCounter(statistic, count); + storageStatistics.incrementCounter(statistic, count); + } + + /** + * Decrement a gauge by a specific value. + * @param statistic The operation to decrement + * @param count the count to decrement + */ + protected void decrementGauge(Statistic statistic, long count) { + instrumentation.decrementGauge(statistic, count); + } + + /** + * Increment a gauge by a specific value. + * @param statistic The operation to increment + * @param count the count to increment + */ + protected void incrementGauge(Statistic statistic, long count) { + instrumentation.incrementGauge(statistic, count); + } + + /** + * Callback when an operation was retried. + * Increments the statistics of ignored errors or throttled requests, + * depending up on the exception class. + * @param ex exception. + */ + public void operationRetried(Exception ex) { + if (isThrottleException(ex)) { + operationThrottled(false); + } else { + incrementStatistic(IGNORED_ERRORS); + } + } + + /** + * Callback from {@link Invoker} when an operation is retried. + * @param text text of the operation + * @param ex exception + * @param retries number of retries + * @param idempotent is the method idempotent + */ + public void operationRetried( + String text, + Exception ex, + int retries, + boolean idempotent) { + operationRetried(ex); + } + + /** + * Callback from {@link Invoker} when an operation against a metastore + * is retried. + * Always increments the {@link Statistic#S3GUARD_METADATASTORE_RETRY} + * statistic/counter; + * if it is a throttling exception will update the associated + * throttled metrics/statistics. + * + * @param ex exception + * @param retries number of retries + * @param idempotent is the method idempotent + */ + public void metastoreOperationRetried(Exception ex, + int retries, + boolean idempotent) { + incrementStatistic(S3GUARD_METADATASTORE_RETRY); + if (isThrottleException(ex)) { + operationThrottled(true); + } else { + incrementStatistic(IGNORED_ERRORS); + } + } + + + /** + * Handler for failure of bulk delete requests. + * @param deleteRequest request which was retried. + * @param ex exception + */ + public void bulkDeleteRetried( + DeleteObjectsRequest deleteRequest, + Exception ex) { + if (isThrottleException(ex)) { + final int size = deleteRequest.getKeys().size(); + incrementStatistic(STORE_IO_THROTTLED, size); + instrumentation.addValueToQuantiles(STORE_IO_THROTTLE_RATE, size); + THROTTLE_LOG.warn("Bulk delete {} keys throttled -first key = {}", + size, deleteRequest.getKeys().get(0)); + } else { + incrementStatistic(IGNORED_ERRORS); + } + } + + + /** + * Note that an operation was throttled -this will update + * specific counters/metrics. + * @param metastore was the throttling observed in the S3Guard metastore? + */ + private void operationThrottled(boolean metastore) { + THROTTLE_LOG.debug("Request throttled on {}", metastore ? "S3" : "DynamoDB"); + if (metastore) { + incrementStatistic(S3GUARD_METADATASTORE_THROTTLED); + instrumentation.addValueToQuantiles(S3GUARD_METADATASTORE_THROTTLE_RATE, + 1); + } else { + incrementStatistic(STORE_IO_THROTTLED); + instrumentation.addValueToQuantiles(STORE_IO_THROTTLE_RATE, 1); + } + } + +} diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md index ca35ce8f66c72..b4bd1b061cb69 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md @@ -464,6 +464,21 @@ Otherwise, set a large timeout in `fs.s3a.scale.test.timeout` The tests are executed in an order to only clean up created files after the end of all the tests. If the tests are interrupted, the test data will remain. +## Load tests. +There are a few tests which are designed to overload AWS services with more +requests per second than an AWS account is permitted. + +The operation of these test maybe observable to other users of the same +account -especially if they are working in the AWS region to which the +tests are targeted. + +There may also run up larger bills. + +These tests all have the prefix `ILoadTest` + +They do not run automatically: they must be explicitly run from the command line or an IDE. + +Look in the source for these and reads the Java docs before executing. ## Testing against non AWS S3 endpoints. @@ -1467,6 +1482,7 @@ Then see if complete successfully in roughly the same time once the upgrade is a to AWS services. * Try and get other people, especially anyone with their own endpoints, apps or different deployment environments, to run their own tests. +* Run the load tests, especially `ILoadTestS3ABulkDeleteThrottling`. ### Dealing with Deprecated APIs and New Features diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ILoadTestS3ABulkDeleteThrottling.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ILoadTestS3ABulkDeleteThrottling.java index 219a3de80c4a3..fb39ca1bab48f 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ILoadTestS3ABulkDeleteThrottling.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ILoadTestS3ABulkDeleteThrottling.java @@ -22,30 +22,42 @@ import java.io.FileWriter; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; import java.util.List; import java.util.concurrent.CompletionService; import java.util.concurrent.ExecutorCompletionService; import java.util.concurrent.ExecutorService; import com.amazonaws.services.s3.model.DeleteObjectsRequest; +import com.google.common.base.Preconditions; import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.junit.Assume; import org.junit.FixMethodOrder; import org.junit.Test; +import org.junit.runner.RunWith; import org.junit.runners.MethodSorters; +import org.junit.runners.Parameterized; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.impl.FunctionsRaisingIOE; import org.apache.hadoop.fs.impl.WrappedIOException; import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.auth.delegation.Csvout; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.util.concurrent.HadoopExecutors; +import static org.apache.hadoop.fs.s3a.Constants.AWS_INTERNAL_THROTTLING; +import static org.apache.hadoop.fs.s3a.Constants.BULK_DELETE_PAGE_SIZE; +import static org.apache.hadoop.fs.s3a.Constants.BULK_DELETE_PAGE_SIZE_DEFAULT; import static org.apache.hadoop.fs.s3a.Constants.ENABLE_MULTI_DELETE; +import static org.apache.hadoop.fs.s3a.Constants.USER_AGENT_PREFIX; +import static org.apache.hadoop.fs.s3a.impl.InternalConstants.MAX_ENTRIES_TO_DELETE; /** * Test some scalable operations related to file renaming and deletion. @@ -53,8 +65,11 @@ * whereas that was designed to overload an STS endpoint, this just * tries to overload a single S3 shard with too many bulk IO requests * -and so see what happens. + * Note: UA field includes the configuration tested for the benefit + * */ @FixMethodOrder(MethodSorters.NAME_ASCENDING) +@RunWith(Parameterized.class) public class ILoadTestS3ABulkDeleteThrottling extends S3AScaleTestBase { private static final Logger LOG = @@ -62,13 +77,20 @@ public class ILoadTestS3ABulkDeleteThrottling extends S3AScaleTestBase { protected static final int THREADS = 20; - private static boolean previousTestThrottled; + public static final int SMALL = BULK_DELETE_PAGE_SIZE_DEFAULT; + public static final int SMALL_REQS = 100; + + public static final int MAXIMUM = MAX_ENTRIES_TO_DELETE; + public static final int MAXIMUM_REQS = 20; + + // shared across test cases. + private static boolean testWasThrottled; private final ExecutorService executor = HadoopExecutors.newFixedThreadPool( THREADS, new ThreadFactoryBuilder() - .setNameFormat(" #%d") + .setNameFormat("#%d") .build()); private final CompletionService @@ -76,6 +98,63 @@ public class ILoadTestS3ABulkDeleteThrottling extends S3AScaleTestBase { private File dataDir; + private final boolean throttle; + private final int pageSize; + private final int requests; + + /** + * Test array for parameterized test runs. + *
    + *
  • AWS client throttle on/off
  • + *
  • Page size
  • + *
+ * + * @return a list of parameter tuples. + */ + @Parameterized.Parameters(name = "bulk-delete-client-retry={0}-requests={2}-size={1}") + public static Collection params() { + return Arrays.asList(new Object[][]{ + {false, SMALL, SMALL_REQS}, + {false, MAXIMUM, MAXIMUM_REQS}, + {true, SMALL, SMALL_REQS}, + {true, MAXIMUM, MAXIMUM_REQS}, + }); + } + + /** + * Parameterized constructor. + * @param throttle AWS client throttle on/off + * @param pageSize Page size + * @param requests request count; + */ + public ILoadTestS3ABulkDeleteThrottling( + final boolean throttle, + final int pageSize, + final int requests) { + this.throttle = throttle; + Preconditions.checkArgument(pageSize > 0, + "page size too low %s", pageSize); + + this.pageSize = pageSize; + this.requests = requests; + } + + @Override + protected Configuration createScaleConfiguration() { + Configuration conf = super.createScaleConfiguration(); + S3ATestUtils.removeBaseAndBucketOverrides(conf, + AWS_INTERNAL_THROTTLING, + BULK_DELETE_PAGE_SIZE, + USER_AGENT_PREFIX); + conf.setBoolean(AWS_INTERNAL_THROTTLING, throttle); + conf.setInt(BULK_DELETE_PAGE_SIZE, pageSize); + conf.set(USER_AGENT_PREFIX, + String.format("ILoadTestS3ABulkDeleteThrottling-%s-%04d", + throttle, pageSize)); + S3ATestUtils.disableFilesystemCaching(conf); + return conf; + } + @Override public void setup() throws Exception { super.setup(); @@ -86,45 +165,54 @@ public void setup() throws Exception { } @Test - public void test_010_DeleteThrottlingSmall() throws Throwable { - describe("test how S3 reacts to massive multipart deletion requests"); - maybeSleep(); - final File results = deleteFiles(100, 200); - LOG.info("Test run completed against {}:\n see {}", getFileSystem(), - results); - } + public void test_010_Reset() throws Throwable { + testWasThrottled = false; + final Path p = path("p"); - @Test - public void test_011_sleep() throws Throwable { - maybeSleep(); + final S3AFileSystem fs = getFileSystem(); + ContractTestUtils.touch(fs, p ); + fs.delete(p, false); } @Test - public void test_020_DeleteThrottlingBigFiles() throws Throwable { + public void test_020_DeleteThrottling() throws Throwable { describe("test how S3 reacts to massive multipart deletion requests"); - final File results = deleteFiles(20, 1000); + final File results = deleteFiles(requests, pageSize); LOG.info("Test run completed against {}:\n see {}", getFileSystem(), results); - previousTestThrottled = true; + if (testWasThrottled) { + LOG.warn("Test was throttled"); + } else { + LOG.info("No throttling recorded in filesystem"); + } } - private void maybeSleep() throws InterruptedException { - if (previousTestThrottled) { + + @Test + public void test_030_Sleep() throws Throwable { + maybeSleep(); + } + + private void maybeSleep() throws InterruptedException, IOException { + if (testWasThrottled) { LOG.info("Sleeping briefly to let store recover"); Thread.sleep(30_000); - previousTestThrottled = false; + getFileSystem().delete(path("recovery"), true); + testWasThrottled = false; } } + /** - * Fetch requests. + * delete files. * @param requests number of requests. * @throws Exception failure - * @return + * @return CSV filename */ private File deleteFiles(final int requests, final int entries) throws Exception { File csvFile = new File(dataDir, - String.format("delete-%03d-%04d.csv", requests, entries)); + String.format("delete-%03d-%04d-%s.csv", + requests, entries, throttle)); describe("Issuing %d requests of size %d, saving log to %s", requests, entries, csvFile); Path basePath = path("testDeleteObjectThrottling"); @@ -136,11 +224,9 @@ private File deleteFiles(final int requests, final int entries) Csvout csvout = new Csvout(out, "\t", "\n"); Outcome.writeSchema(csvout); - final ContractTestUtils.NanoTimer jobTimer = new ContractTestUtils.NanoTimer(); - for (int i = 0; i < requests; i++) { final int id = i; completionService.submit(() -> { From aee41b4929767aa8caae9bcef92e28b141308a79 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 4 Feb 2020 18:54:12 +0000 Subject: [PATCH 09/17] HADOOP-16823 tune directory operations This adds an option "fs.s3a.experimental.optimized.directory.operations" which says "optimize directory IO" without being explicit about what it does. In this release It only looks for and deletes parent dir entry when creating a file (we still do for mkdir though) This is dangerous as it goes against what is the fs spec for create (goes with createNonRecursive though :). If you create a file two levels under an empty dir, that empty dir marker stays there. but consider: if you create a file two levels under a file, s3a is happy. and nobody has noticed. Also * directory cleanup/parent marker recreation is done async * page size set to 250; seems to balance out better in the load tests. * HADOOP-16613. dir marker contentType = application/x-directory Change-Id: Id88a198f61beb3719aa4202d26f3634e5e9cc194 --- .../org/apache/hadoop/fs/s3a/Constants.java | 19 +++- .../hadoop/fs/s3a/DefaultS3ClientFactory.java | 4 +- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 105 ++++++++++++++---- .../hadoop/fs/s3a/WriteOperationHelper.java | 5 +- .../hadoop/fs/s3a/impl/CallableSupplier.java | 14 +++ .../hadoop/fs/s3a/impl/InternalConstants.java | 6 +- .../ILoadTestS3ABulkDeleteThrottling.java | 47 +++++--- 7 files changed, 155 insertions(+), 45 deletions(-) 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 f22685489ff12..9ba6d039b732a 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 @@ -199,9 +199,24 @@ private Constants() { * * This only applies to S3 operations, not to DynamoDB or other services. */ - public static final String AWS_INTERNAL_THROTTLING = + @InterfaceStability.Unstable + public static final String EXPERIMENTAL_AWS_INTERNAL_THROTTLING = "fs.s3a.experimental.aws.internal.throttling"; + /** + * Experimental/Unstable feature: should empty directory marker + * operations be optimized? Value {@value}. + * Default: false. + * + * This is an experimental feature for reducing operations related + * to looking for/deleting fake directory markers. + * The goals are better performance as well as fewer tombstone markers + * being created on versioned buckets. + */ + @InterfaceStability.Unstable + public static final String EXPERIMENTAL_OPTIMIZED_DIRECTORY_OPERATIONS = + "fs.s3a.experimental.optimized.directory.operations"; + // seconds until we give up trying to establish a connection to s3 public static final String ESTABLISH_TIMEOUT = "fs.s3a.connection.establish.timeout"; @@ -274,7 +289,7 @@ private Constants() { * Default Number of objects to delete in a single multi-object * delete: {@value}. */ - public static final int BULK_DELETE_PAGE_SIZE_DEFAULT = 200; + public static final int BULK_DELETE_PAGE_SIZE_DEFAULT = 250; // comma separated list of directories public static final String BUFFER_DIR = "fs.s3a.buffer.dir"; 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 5f232a47bfda2..5fc31e9646426 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 @@ -34,7 +34,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; -import static org.apache.hadoop.fs.s3a.Constants.AWS_INTERNAL_THROTTLING; +import static org.apache.hadoop.fs.s3a.Constants.EXPERIMENTAL_AWS_INTERNAL_THROTTLING; import static org.apache.hadoop.fs.s3a.Constants.ENDPOINT; import static org.apache.hadoop.fs.s3a.Constants.PATH_STYLE_ACCESS; @@ -62,7 +62,7 @@ public AmazonS3 createS3Client(URI name, // throttling is explicitly disabled on the S3 client so that // all failures are collected awsConf.setUseThrottleRetries( - conf.getBoolean(AWS_INTERNAL_THROTTLING, true)); + conf.getBoolean(EXPERIMENTAL_AWS_INTERNAL_THROTTLING, true)); if (!StringUtils.isEmpty(userAgentSuffix)) { awsConf.setUserAgentSuffix(userAgentSuffix); 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 31cdab7503342..1a53e3865409b 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 @@ -171,8 +171,11 @@ import static org.apache.hadoop.fs.s3a.auth.RolePolicies.allowS3Operations; import static org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens.TokenIssuingPolicy.NoTokensAvailable; import static org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens.hasDelegationTokenBinding; -import static org.apache.hadoop.fs.s3a.impl.InternalConstants.MAX_ENTRIES_TO_DELETE; +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.s3a.impl.CallableSupplier.waitForCompletionIgnoringExceptions; import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_404; +import static org.apache.hadoop.fs.s3a.impl.InternalConstants.X_DIRECTORY; import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.fixBucketRegion; import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; @@ -281,6 +284,11 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, private final S3AFileSystem.OperationCallbacksImpl operationCallbacks = new OperationCallbacksImpl(); + /** + * Should directory marker use be optimized? + */ + private boolean optimizeDirectoryOperations; + /** Add any deprecated keys. */ @SuppressWarnings("deprecation") private static void addDeprecatedKeys() { @@ -442,6 +450,11 @@ public void initialize(URI name, Configuration originalConf) } initMultipartUploads(conf); + optimizeDirectoryOperations = conf.getBoolean( + EXPERIMENTAL_OPTIMIZED_DIRECTORY_OPERATIONS, false); + if (optimizeDirectoryOperations) { + LOG.info("Using experimental optimized directory operations"); + } } catch (AmazonClientException e) { // amazon client exception: stop all services then throw the translation stopAllServices(); @@ -1487,8 +1500,21 @@ public void finishRename(final Path sourceRenamed, final Path destCreated) Path destParent = destCreated.getParent(); if (!sourceRenamed.getParent().equals(destParent)) { LOG.debug("source & dest parents are different; fix up dir markers"); - deleteUnnecessaryFakeDirectories(destParent); - maybeCreateFakeParentDirectory(sourceRenamed); + // kick off an async delete + List> ops = new ArrayList<>(2); + ops.add(submit( + boundedThreadPool, + () -> { + deleteUnnecessaryFakeDirectories(destParent, false); + return null; + })); + ops.add(submit( + boundedThreadPool, + () -> { + maybeCreateFakeParentDirectory(sourceRenamed);; + return null; + })); + waitForCompletion(ops); } } @@ -2372,13 +2398,14 @@ DeleteObjectsResult removeKeys( public boolean delete(Path f, boolean recursive) throws IOException { try { entryPoint(INVOCATION_DELETE); + final int pageSize = intOption(getConf(), BULK_DELETE_PAGE_SIZE, + BULK_DELETE_PAGE_SIZE_DEFAULT, 0); DeleteOperation deleteOperation = new DeleteOperation( createStoreContext(), innerGetFileStatus(f, true, StatusProbeEnum.ALL), recursive, operationCallbacks, - intOption(getConf(), BULK_DELETE_PAGE_SIZE, - BULK_DELETE_PAGE_SIZE_DEFAULT, 0)); + pageSize); boolean outcome = deleteOperation.execute(); if (outcome) { try { @@ -2859,7 +2886,7 @@ S3AFileStatus innerGetFileStatus(final Path f, S3AFileStatus s3GetFileStatus(final Path path, final String key, final Set probes, - final Set tombstones) throws IOException { + @Nullable Set tombstones) throws IOException { if (!key.isEmpty()) { if (probes.contains(StatusProbeEnum.Head) && !key.endsWith("/")) { try { @@ -3514,14 +3541,14 @@ private Optional generateSSECustomerKey() { /** * Perform post-write actions. - * Calls {@link #deleteUnnecessaryFakeDirectories(Path)} and then + * Calls {@link #deleteUnnecessaryFakeDirectories(Path, boolean)} and then * updates any metastore. * This operation MUST be called after any PUT/multipart PUT completes * successfully. * * The operations actions include *
    - *
  1. Calling {@link #deleteUnnecessaryFakeDirectories(Path)}
  2. + *
  3. Calling {@link #deleteUnnecessaryFakeDirectories(Path, boolean)}
  4. *
  5. Updating any metadata store with details on the newly created * object.
  6. *
@@ -3544,7 +3571,14 @@ void finishedWrite(String key, long length, String eTag, String versionId, key, length, eTag, versionId); Path p = keyToQualifiedPath(key); Preconditions.checkArgument(length >= 0, "content length is negative"); - deleteUnnecessaryFakeDirectories(p.getParent()); + final boolean isDir = objectRepresentsDirectory(key, length); + // kick off an async delete + final CompletableFuture deletion = submit( + boundedThreadPool, + () -> { + deleteUnnecessaryFakeDirectories(p.getParent(), isDir); + return null; + }); // this is only set if there is a metastore to update and the // operationState parameter passed in was null. BulkOperationState stateToClose = null; @@ -3563,7 +3597,6 @@ void finishedWrite(String key, long length, String eTag, String versionId, activeState = stateToClose; } S3Guard.addAncestors(metadataStore, p, ttlTimeProvider, activeState); - final boolean isDir = objectRepresentsDirectory(key, length); S3AFileStatus status = createUploadFileStatus(p, isDir, length, getDefaultBlockSize(p), username, eTag, versionId); @@ -3586,6 +3619,8 @@ void finishedWrite(String key, long length, String eTag, String versionId, activeState); } } + // and catch up with any delete operation. + waitForCompletionIgnoringExceptions(deletion); } catch (IOException e) { if (failOnMetadataWriteError) { throw new MetadataPersistenceException(p.toString(), e); @@ -3604,16 +3639,45 @@ void finishedWrite(String key, long length, String eTag, String versionId, * Delete mock parent directories which are no longer needed. * Retry policy: retrying; exceptions swallowed. * @param path path + * @param isMkDirOperation is this for a mkdir call? */ @Retries.RetryExceptionsSwallowed - private void deleteUnnecessaryFakeDirectories(Path path) { + private void deleteUnnecessaryFakeDirectories(Path path, + final boolean isMkDirOperation) { List keysToRemove = new ArrayList<>(); - while (!path.isRoot()) { - String key = pathToKey(path); - key = (key.endsWith("/")) ? key : (key + "/"); - LOG.trace("To delete unnecessary fake directory {} for {}", key, path); - keysToRemove.add(new DeleteObjectsRequest.KeyVersion(key)); - path = path.getParent(); + boolean deleteWholeTree = false; + if (optimizeDirectoryOperations && !isMkDirOperation) { + // this is a file creation/commit + // Assume that the parent directory exists either explicitly as a marker + // on implicitly (peer entries) + // only look for the dir marker in S3 -we don't care about DDB. + try { + String key = pathToKey(path); + s3GetFileStatus(path, key, StatusProbeEnum.DIR_MARKER_ONLY, null); + // here an entry exists. + LOG.debug("Removing marker {}", key); + keysToRemove.add(new DeleteObjectsRequest.KeyVersion(key)); + } catch (FileNotFoundException e) { + // no entry. Nothing to delete. + } catch (IOException e) { + instrumentation.errorIgnored(); + LOG.debug("Ignored when looking at directory marker {}", path, e); + // for now, fall back to a full delete. + deleteWholeTree = true; + } + } else { + deleteWholeTree = true; + } + if (deleteWholeTree) { + // traditional delete creates a delete request for + // all parents. + while (!path.isRoot()) { + String key = pathToKey(path); + key = (key.endsWith("/")) ? key : (key + "/"); + LOG.trace("To delete unnecessary fake directory {} for {}", key, path); + keysToRemove.add(new DeleteObjectsRequest.KeyVersion(key)); + path = path.getParent(); + } } try { removeKeys(keysToRemove, true, null); @@ -3661,8 +3725,10 @@ public int read() throws IOException { } }; + final ObjectMetadata metadata = newObjectMetadata(0L); + metadata.setContentType(X_DIRECTORY); PutObjectRequest putObjectRequest = newPutObjectRequest(objectName, - newObjectMetadata(0L), + metadata, im); invoker.retry("PUT 0-byte object ", objectName, true, @@ -3778,8 +3844,7 @@ public String toString() { if (committerIntegration != null) { sb.append(", magicCommitter=").append(isMagicCommitEnabled()); } - sb.append(", boundedExecutor=").append(boundedThreadPool); - sb.append(", unboundedExecutor=").append(unboundedThreadPool); + sb.append(", optimizeDirMarkers=").append(optimizeDirectoryOperations); sb.append(", credentials=").append(credentials); sb.append(", delegation tokens=") .append(delegationTokens.map(Objects::toString).orElse("disabled")); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java index 602732b6d3250..a1a7707b0f838 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 @@ -479,7 +479,9 @@ public UploadResult uploadObject(PutObjectRequest putObjectRequest) /** * Revert a commit by deleting the file. - * Relies on retry code in filesystem + * No attempt is made to create/rebuild the parent directory, on the + * assumption that reverting is part of a more complex sequence. + * Relies on retry code in filesystem. * @throws IOException on problems * @param destKey destination key * @param operationState operational state for a bulk update @@ -492,7 +494,6 @@ public void revertCommit(String destKey, Path destPath = owner.keyToQualifiedPath(destKey); owner.deleteObjectAtPath(destPath, destKey, true, operationState); - owner.maybeCreateFakeParentDirectory(destPath); } ); } 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 609eecee643c3..b753d4e2efdee 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 @@ -123,4 +123,18 @@ public static void waitForCompletion( } } + /** + * Wait for a single of future to complete, ignoring exceptions raised. + * @param future future to wait for. + */ + public static void waitForCompletionIgnoringExceptions( + final CompletableFuture future) { + try (DurationInfo ignore = + new DurationInfo(LOG, false, "Waiting for task completion")) { + future.join(); + } catch (Exception e) { + LOG.debug("Ignoring exception raised in task completion: "); + } + } + } 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 608617c592a66..c73580d19fccd 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 @@ -86,7 +86,11 @@ private InternalConstants() { /** 404 error code. */ public static final int SC_404 = 404; - /* Name of the log for throttling events. */ + /** Name of the log for throttling events. Value: {@value}. */ public static final String THROTTLE_LOG_NAME = "org.apache.hadoop.fs.s3a.throttled"; + + /** Directory marker attribute: see HADOOP-16613. Value: {@value}. */ + public static final String X_DIRECTORY = + "application/x-directory"; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ILoadTestS3ABulkDeleteThrottling.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ILoadTestS3ABulkDeleteThrottling.java index fb39ca1bab48f..06d822edcfeb4 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ILoadTestS3ABulkDeleteThrottling.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ILoadTestS3ABulkDeleteThrottling.java @@ -32,6 +32,7 @@ import com.amazonaws.services.s3.model.DeleteObjectsRequest; import com.google.common.base.Preconditions; import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.assertj.core.api.Assertions; import org.junit.Assume; import org.junit.FixMethodOrder; import org.junit.Test; @@ -52,7 +53,7 @@ import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.util.concurrent.HadoopExecutors; -import static org.apache.hadoop.fs.s3a.Constants.AWS_INTERNAL_THROTTLING; +import static org.apache.hadoop.fs.s3a.Constants.EXPERIMENTAL_AWS_INTERNAL_THROTTLING; import static org.apache.hadoop.fs.s3a.Constants.BULK_DELETE_PAGE_SIZE; import static org.apache.hadoop.fs.s3a.Constants.BULK_DELETE_PAGE_SIZE_DEFAULT; import static org.apache.hadoop.fs.s3a.Constants.ENABLE_MULTI_DELETE; @@ -66,7 +67,7 @@ * tries to overload a single S3 shard with too many bulk IO requests * -and so see what happens. * Note: UA field includes the configuration tested for the benefit - * + * of anyone looking through the server logs. */ @FixMethodOrder(MethodSorters.NAME_ASCENDING) @RunWith(Parameterized.class) @@ -75,15 +76,17 @@ public class ILoadTestS3ABulkDeleteThrottling extends S3AScaleTestBase { private static final Logger LOG = LoggerFactory.getLogger(ILoadTestS3ABulkDeleteThrottling.class); - protected static final int THREADS = 20; + protected static final int THREADS = 40; + public static final int TOTAL_KEYS = 15000; public static final int SMALL = BULK_DELETE_PAGE_SIZE_DEFAULT; - public static final int SMALL_REQS = 100; + public static final int SMALL_REQS = TOTAL_KEYS / SMALL; public static final int MAXIMUM = MAX_ENTRIES_TO_DELETE; - public static final int MAXIMUM_REQS = 20; + public static final int MAXIMUM_REQS = TOTAL_KEYS / MAXIMUM; // shared across test cases. + @SuppressWarnings("StaticNonFinalField") private static boolean testWasThrottled; private final ExecutorService executor = @@ -142,36 +145,44 @@ public ILoadTestS3ABulkDeleteThrottling( @Override protected Configuration createScaleConfiguration() { Configuration conf = super.createScaleConfiguration(); + S3ATestUtils.disableFilesystemCaching(conf); + return conf; + } + + @Override + public void setup() throws Exception { + final Configuration conf = getConf(); S3ATestUtils.removeBaseAndBucketOverrides(conf, - AWS_INTERNAL_THROTTLING, + EXPERIMENTAL_AWS_INTERNAL_THROTTLING, BULK_DELETE_PAGE_SIZE, USER_AGENT_PREFIX); - conf.setBoolean(AWS_INTERNAL_THROTTLING, throttle); + conf.setBoolean(EXPERIMENTAL_AWS_INTERNAL_THROTTLING, throttle); + Assertions.assertThat(pageSize) + .describedAs("page size") + .isGreaterThan(0); conf.setInt(BULK_DELETE_PAGE_SIZE, pageSize); conf.set(USER_AGENT_PREFIX, String.format("ILoadTestS3ABulkDeleteThrottling-%s-%04d", throttle, pageSize)); - S3ATestUtils.disableFilesystemCaching(conf); - return conf; - } - @Override - public void setup() throws Exception { super.setup(); Assume.assumeTrue("multipart delete disabled", - getConf().getBoolean(ENABLE_MULTI_DELETE, true)); + conf.getBoolean(ENABLE_MULTI_DELETE, true)); dataDir = GenericTestUtils.getTestDir("throttling"); dataDir.mkdirs(); + final String size = getFileSystem().getConf().get(BULK_DELETE_PAGE_SIZE); + Assertions.assertThat(size) + .describedAs("page size") + .isNotEmpty(); + Assertions.assertThat(getFileSystem().getConf() + .getInt(BULK_DELETE_PAGE_SIZE, -1)) + .isEqualTo(pageSize); + } @Test public void test_010_Reset() throws Throwable { testWasThrottled = false; - final Path p = path("p"); - - final S3AFileSystem fs = getFileSystem(); - ContractTestUtils.touch(fs, p ); - fs.delete(p, false); } @Test From cb78e00838a20e323b8e19c13fbbe6e822f031f4 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 5 Feb 2020 15:30:17 +0000 Subject: [PATCH 10/17] HADOOP-16823 throttling: cleanup of new code Change-Id: I7bb9a4a9cc0b5e1ee7a54be7c5f463621ca66bc1 --- .../contract/AbstractContractRenameTest.java | 4 +- .../hadoop/fs/contract/ContractTestUtils.java | 5 +- .../org/apache/hadoop/fs/s3a/Constants.java | 5 +- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 49 ++++++++++++++----- .../hadoop/fs/s3a/WriteOperationHelper.java | 5 +- .../hadoop/fs/s3a/impl/CallableSupplier.java | 29 ++++++++--- .../hadoop/fs/s3a/impl/DeleteOperation.java | 20 +------- ...r.java => StandardInvokeRetryHandler.java} | 6 +-- .../contract/s3a/ITestS3AContractRename.java | 2 +- .../fs/s3a/commit/ITestCommitOperations.java | 1 + 10 files changed, 74 insertions(+), 52 deletions(-) rename hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/{RetryCallbackHandler.java => StandardInvokeRetryHandler.java} (97%) diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRenameTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRenameTest.java index b509f32b64b5f..b8a5d5cc7d84b 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRenameTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRenameTest.java @@ -204,9 +204,7 @@ public void testRenameWithNonEmptySubDir() throws Throwable { assertPathExists("not created in src/sub dir", new Path(srcSubDir, "subfile.txt")); - boolean rename = fs.rename(srcDir, finalDir); - assertTrue("rename(" + srcDir + ", " + finalDir + ") failed", - rename); + rename(srcDir, finalDir); // Accept both POSIX rename behavior and CLI rename behavior if (renameRemoveEmptyDest) { 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 f61634943bb7f..4789630f95f1c 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 @@ -418,8 +418,9 @@ public static boolean rm(FileSystem fileSystem, public static void rename(FileSystem fileSystem, Path src, Path dst) throws IOException { rejectRootOperation(src, false); - assertTrue(fileSystem.rename(src, dst)); - assertPathDoesNotExist(fileSystem, "renamed", src); + assertTrue("rename(" + src + ", " + dst + ") failed", + fileSystem.rename(src, dst)); + assertPathDoesNotExist(fileSystem, "renamed source dir", src); } /** 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 9ba6d039b732a..96346dbc5e105 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 @@ -186,7 +186,6 @@ private Constants() { */ public static final int DEFAULT_MAX_ERROR_RETRIES = 10; - /** * Experimental/Unstable feature: should the AWS client library retry * throttle responses before escalating to the S3A code: {@value}. @@ -213,7 +212,7 @@ private Constants() { * The goals are better performance as well as fewer tombstone markers * being created on versioned buckets. */ - @InterfaceStability.Unstable + @InterfaceStability.Unstable public static final String EXPERIMENTAL_OPTIMIZED_DIRECTORY_OPERATIONS = "fs.s3a.experimental.optimized.directory.operations"; @@ -805,7 +804,7 @@ private Constants() { * Interval between retry attempts on throttled requests: {@value}. */ public static final String RETRY_THROTTLE_INTERVAL = - "fs.s3a.retry.throttle.interval"; + "fs.s3a.retry.throttle.interval"; /** * Default throttled retry interval: {@value}. 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 1a53e3865409b..a57849255b809 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 @@ -107,7 +107,7 @@ 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.RetryCallbackHandler; +import org.apache.hadoop.fs.s3a.impl.StandardInvokeRetryHandler; import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum; import org.apache.hadoop.fs.s3a.impl.StoreContext; import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; @@ -278,6 +278,11 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, private ITtlTimeProvider ttlTimeProvider; + /** + * Page size for deletions. + */ + private int pageSize; + /** * Specific operations used by rename and delete operations. */ @@ -450,6 +455,9 @@ public void initialize(URI name, Configuration originalConf) } initMultipartUploads(conf); + + pageSize = intOption(getConf(), BULK_DELETE_PAGE_SIZE, + BULK_DELETE_PAGE_SIZE_DEFAULT, 0); optimizeDirectoryOperations = conf.getBoolean( EXPERIMENTAL_OPTIMIZED_DIRECTORY_OPERATIONS, false); if (optimizeDirectoryOperations) { @@ -1404,8 +1412,7 @@ private long innerRename(Path source, Path dest) src, srcKey, p.getLeft(), dst, dstKey, p.getRight(), operationCallbacks, - intOption(getConf(), BULK_DELETE_PAGE_SIZE, - BULK_DELETE_PAGE_SIZE_DEFAULT, 0)); + pageSize); return renameOperation.execute(); } @@ -1511,7 +1518,7 @@ public void finishRename(final Path sourceRenamed, final Path destCreated) ops.add(submit( boundedThreadPool, () -> { - maybeCreateFakeParentDirectory(sourceRenamed);; + maybeCreateFakeParentDirectory(sourceRenamed); return null; })); waitForCompletion(ops); @@ -1975,8 +1982,8 @@ private void blockRootDelete(String key) throws InvalidRequestException { private DeleteObjectsResult deleteObjects(DeleteObjectsRequest deleteRequest) throws MultiObjectDeleteException, AmazonClientException, IOException { incrementWriteOperations(); - RetryCallbackHandler retryHandler = - new RetryCallbackHandler(createStoreContext()); + StandardInvokeRetryHandler retryHandler = + new StandardInvokeRetryHandler(createStoreContext()); try(DurationInfo ignored = new DurationInfo(LOG, false, "DELETE %d keys", deleteRequest.getKeys().size())) { @@ -2398,8 +2405,6 @@ DeleteObjectsResult removeKeys( public boolean delete(Path f, boolean recursive) throws IOException { try { entryPoint(INVOCATION_DELETE); - final int pageSize = intOption(getConf(), BULK_DELETE_PAGE_SIZE, - BULK_DELETE_PAGE_SIZE_DEFAULT, 0); DeleteOperation deleteOperation = new DeleteOperation( createStoreContext(), innerGetFileStatus(f, true, StatusProbeEnum.ALL), @@ -3548,7 +3553,8 @@ private Optional generateSSECustomerKey() { * * The operations actions include *
    - *
  1. Calling {@link #deleteUnnecessaryFakeDirectories(Path, boolean)}
  2. + *
  3. Calling + * {@link #deleteUnnecessaryFakeDirectories(Path, boolean)}
  4. *
  5. Updating any metadata store with details on the newly created * object.
  6. *
@@ -3663,6 +3669,8 @@ private void deleteUnnecessaryFakeDirectories(Path path, instrumentation.errorIgnored(); LOG.debug("Ignored when looking at directory marker {}", path, e); // for now, fall back to a full delete. + // if the failure was permissions or network this will probably fail + // too... deleteWholeTree = true; } } else { @@ -3942,25 +3950,40 @@ public boolean exists(Path f) throws IOException { } /** - * Override superclass so as to add statistic collection. + * An optimized check which only looks for directory markers. * {@inheritDoc} */ @Override @SuppressWarnings("deprecation") public boolean isDirectory(Path f) throws IOException { entryPoint(INVOCATION_IS_DIRECTORY); - return super.isDirectory(f); + try { + // against S3Guard, a full query; + // against S3 a HEAD + "/" then a LIST. + return innerGetFileStatus(f, false, + StatusProbeEnum.DIRECTORIES).isDirectory(); + } catch (FileNotFoundException e) { + return false; + } } /** - * Override superclass so as to add statistic collection. + * Override superclass so as to only poll for a file. + * Warning: may leave a 404 in the S3 load balancer cache. * {@inheritDoc} */ @Override @SuppressWarnings("deprecation") public boolean isFile(Path f) throws IOException { entryPoint(INVOCATION_IS_FILE); - return super.isFile(f); + try { + // against S3Guard, a full query; against S3 only a HEAD. + return innerGetFileStatus(f, false, + StatusProbeEnum.HEAD_ONLY).isFile(); + } catch (FileNotFoundException e) { + // no file or there is a directory there. + return false; + } } /** 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 a1a7707b0f838..602732b6d3250 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 @@ -479,9 +479,7 @@ public UploadResult uploadObject(PutObjectRequest putObjectRequest) /** * Revert a commit by deleting the file. - * No attempt is made to create/rebuild the parent directory, on the - * assumption that reverting is part of a more complex sequence. - * Relies on retry code in filesystem. + * Relies on retry code in filesystem * @throws IOException on problems * @param destKey destination key * @param operationState operational state for a bulk update @@ -494,6 +492,7 @@ public void revertCommit(String destKey, Path destPath = owner.keyToQualifiedPath(destKey); owner.deleteObjectAtPath(destPath, destKey, true, operationState); + owner.maybeCreateFakeParentDirectory(destPath); } ); } 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 b753d4e2efdee..1c61d30b08a60 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 @@ -18,6 +18,7 @@ package org.apache.hadoop.fs.s3a.impl; +import javax.annotation.Nullable; import java.io.IOException; import java.util.List; import java.util.concurrent.Callable; @@ -128,13 +129,29 @@ public static void waitForCompletion( * @param future future to wait for. */ public static void waitForCompletionIgnoringExceptions( - final CompletableFuture future) { - try (DurationInfo ignore = - new DurationInfo(LOG, false, "Waiting for task completion")) { - future.join(); - } catch (Exception e) { - LOG.debug("Ignoring exception raised in task completion: "); + @Nullable final CompletableFuture future) { + if (future != null) { + try (DurationInfo ignore = + new DurationInfo(LOG, false, "Waiting for task completion")) { + future.join(); + } catch (Exception e) { + LOG.debug("Ignoring exception raised in task completion: "); + } } } + /** + * Block awaiting completion for any non-null future passed in; + * No-op if a null arg was supplied. + * @param future future + * @throws IOException if one of the called futures raised an IOE. + * @throws RuntimeException if one of the futures raised one. + */ + public static void maybeAwaitCompletion( + @Nullable final CompletableFuture future) + throws IOException { + if (future != null) { + waitForCompletion(future); + } + } } 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 49c5b313d0ff6..8a1d37d7f6a6e 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 @@ -45,6 +45,7 @@ import org.apache.hadoop.util.DurationInfo; import static com.google.common.base.Preconditions.checkArgument; +import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.maybeAwaitCompletion; import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit; import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletion; @@ -186,7 +187,7 @@ public class DeleteOperation extends ExecutingStoreOperation { * @param status pre-fetched source status * @param recursive recursive delete? * @param callbacks callback provider - * @param pageSize number of entries in a page + * @param pageSize size of delete pages */ public DeleteOperation(final StoreContext context, final S3AFileStatus status, @@ -557,22 +558,5 @@ private void asyncDeleteAction( } } - /** - * Block awaiting completion for any non-null future passed in; - * No-op if a null arg was supplied. - * @param future future - * @throws IOException if one of the called futures raised an IOE. - * @throws RuntimeException if one of the futures raised one. - */ - private void maybeAwaitCompletion( - @Nullable final CompletableFuture future) - throws IOException { - if (future != null) { - try (DurationInfo ignored = - new DurationInfo(LOG, false, "delete completion")) { - waitForCompletion(future); - } - } - } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RetryCallbackHandler.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StandardInvokeRetryHandler.java similarity index 97% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RetryCallbackHandler.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StandardInvokeRetryHandler.java index 5e18b2925e4d2..9f3615632540b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RetryCallbackHandler.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StandardInvokeRetryHandler.java @@ -41,11 +41,11 @@ * This is intended to be pulled out for all operations, but for now it is only * used in delete objects. */ -public class RetryCallbackHandler extends AbstractStoreOperation { +public class StandardInvokeRetryHandler extends AbstractStoreOperation { private static final Logger LOG = LoggerFactory.getLogger( - RetryCallbackHandler.class); + StandardInvokeRetryHandler.class); private static final Logger THROTTLE_LOG = LoggerFactory.getLogger( THROTTLE_LOG_NAME); @@ -54,7 +54,7 @@ public class RetryCallbackHandler extends AbstractStoreOperation { private final S3AStorageStatistics storageStatistics; - public RetryCallbackHandler(final StoreContext storeContext) { + public StandardInvokeRetryHandler(final StoreContext storeContext) { super(storeContext); instrumentation = storeContext.getInstrumentation(); storageStatistics = storeContext.getStorageStatistics(); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRename.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRename.java index a631410d8bdd6..e623d5d27941b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRename.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRename.java @@ -156,7 +156,7 @@ public void testRenamePopulatesFileAncestors2() throws Exception { S3ATestUtils.MetricDiff fileCopyBytes = new S3ATestUtils.MetricDiff(fs, Statistic.FILES_COPIED_BYTES); - fs.rename(src, dest); + rename(src, dest); describe("Rename has completed, examining data under " + base); fileCopyDiff.assertDiffEquals("Number of files copied", 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 455a8a3ebd1c6..74fe45d72d4e8 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 @@ -525,6 +525,7 @@ public void testRevertMissingCommit() throws Throwable { commit.setDestinationKey(fs.pathToKey(destFile)); fullThrottle(); actions.revertCommit(commit, null); + resetFailures(); assertPathExists("parent of reverted (nonexistent) commit", destFile.getParent()); } From 86fd17d3687f93e30e58a8bcb13650e77736965f Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 6 Feb 2020 15:27:07 +0000 Subject: [PATCH 11/17] HADOOP-16823 remove experimentak dir optimization Change-Id: I4df1d47c0865604bbb17083b08cd5a3bc4e1d9f4 --- .../org/apache/hadoop/fs/s3a/Constants.java | 14 --- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 95 ++++--------------- 2 files changed, 18 insertions(+), 91 deletions(-) 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 96346dbc5e105..55250338475f0 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 @@ -202,20 +202,6 @@ private Constants() { public static final String EXPERIMENTAL_AWS_INTERNAL_THROTTLING = "fs.s3a.experimental.aws.internal.throttling"; - /** - * Experimental/Unstable feature: should empty directory marker - * operations be optimized? Value {@value}. - * Default: false. - * - * This is an experimental feature for reducing operations related - * to looking for/deleting fake directory markers. - * The goals are better performance as well as fewer tombstone markers - * being created on versioned buckets. - */ - @InterfaceStability.Unstable - public static final String EXPERIMENTAL_OPTIMIZED_DIRECTORY_OPERATIONS = - "fs.s3a.experimental.optimized.directory.operations"; - // seconds until we give up trying to establish a connection to s3 public static final String ESTABLISH_TIMEOUT = "fs.s3a.connection.establish.timeout"; 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 a57849255b809..251eec9417816 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 @@ -175,7 +175,6 @@ import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletion; import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletionIgnoringExceptions; import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_404; -import static org.apache.hadoop.fs.s3a.impl.InternalConstants.X_DIRECTORY; import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.fixBucketRegion; import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; @@ -289,11 +288,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, private final S3AFileSystem.OperationCallbacksImpl operationCallbacks = new OperationCallbacksImpl(); - /** - * Should directory marker use be optimized? - */ - private boolean optimizeDirectoryOperations; - /** Add any deprecated keys. */ @SuppressWarnings("deprecation") private static void addDeprecatedKeys() { @@ -458,11 +452,6 @@ public void initialize(URI name, Configuration originalConf) pageSize = intOption(getConf(), BULK_DELETE_PAGE_SIZE, BULK_DELETE_PAGE_SIZE_DEFAULT, 0); - optimizeDirectoryOperations = conf.getBoolean( - EXPERIMENTAL_OPTIMIZED_DIRECTORY_OPERATIONS, false); - if (optimizeDirectoryOperations) { - LOG.info("Using experimental optimized directory operations"); - } } catch (AmazonClientException e) { // amazon client exception: stop all services then throw the translation stopAllServices(); @@ -1512,7 +1501,7 @@ public void finishRename(final Path sourceRenamed, final Path destCreated) ops.add(submit( boundedThreadPool, () -> { - deleteUnnecessaryFakeDirectories(destParent, false); + deleteUnnecessaryFakeDirectories(destParent); return null; })); ops.add(submit( @@ -3546,15 +3535,14 @@ private Optional generateSSECustomerKey() { /** * Perform post-write actions. - * Calls {@link #deleteUnnecessaryFakeDirectories(Path, boolean)} and then + * Calls {@link #deleteUnnecessaryFakeDirectories(Path)} and then * updates any metastore. * This operation MUST be called after any PUT/multipart PUT completes * successfully. * * The operations actions include *
    - *
  1. Calling - * {@link #deleteUnnecessaryFakeDirectories(Path, boolean)}
  2. + *
  3. Calling {@link #deleteUnnecessaryFakeDirectories(Path)}
  4. *
  5. Updating any metadata store with details on the newly created * object.
  6. *
@@ -3582,7 +3570,7 @@ void finishedWrite(String key, long length, String eTag, String versionId, final CompletableFuture deletion = submit( boundedThreadPool, () -> { - deleteUnnecessaryFakeDirectories(p.getParent(), isDir); + deleteUnnecessaryFakeDirectories(p.getParent()); return null; }); // this is only set if there is a metastore to update and the @@ -3645,47 +3633,16 @@ void finishedWrite(String key, long length, String eTag, String versionId, * Delete mock parent directories which are no longer needed. * Retry policy: retrying; exceptions swallowed. * @param path path - * @param isMkDirOperation is this for a mkdir call? */ @Retries.RetryExceptionsSwallowed - private void deleteUnnecessaryFakeDirectories(Path path, - final boolean isMkDirOperation) { + private void deleteUnnecessaryFakeDirectories(Path path) { List keysToRemove = new ArrayList<>(); - boolean deleteWholeTree = false; - if (optimizeDirectoryOperations && !isMkDirOperation) { - // this is a file creation/commit - // Assume that the parent directory exists either explicitly as a marker - // on implicitly (peer entries) - // only look for the dir marker in S3 -we don't care about DDB. - try { - String key = pathToKey(path); - s3GetFileStatus(path, key, StatusProbeEnum.DIR_MARKER_ONLY, null); - // here an entry exists. - LOG.debug("Removing marker {}", key); - keysToRemove.add(new DeleteObjectsRequest.KeyVersion(key)); - } catch (FileNotFoundException e) { - // no entry. Nothing to delete. - } catch (IOException e) { - instrumentation.errorIgnored(); - LOG.debug("Ignored when looking at directory marker {}", path, e); - // for now, fall back to a full delete. - // if the failure was permissions or network this will probably fail - // too... - deleteWholeTree = true; - } - } else { - deleteWholeTree = true; - } - if (deleteWholeTree) { - // traditional delete creates a delete request for - // all parents. - while (!path.isRoot()) { - String key = pathToKey(path); - key = (key.endsWith("/")) ? key : (key + "/"); - LOG.trace("To delete unnecessary fake directory {} for {}", key, path); - keysToRemove.add(new DeleteObjectsRequest.KeyVersion(key)); - path = path.getParent(); - } + while (!path.isRoot()) { + String key = pathToKey(path); + key = (key.endsWith("/")) ? key : (key + "/"); + LOG.trace("To delete unnecessary fake directory {} for {}", key, path); + keysToRemove.add(new DeleteObjectsRequest.KeyVersion(key)); + path = path.getParent(); } try { removeKeys(keysToRemove, true, null); @@ -3733,10 +3690,8 @@ public int read() throws IOException { } }; - final ObjectMetadata metadata = newObjectMetadata(0L); - metadata.setContentType(X_DIRECTORY); PutObjectRequest putObjectRequest = newPutObjectRequest(objectName, - metadata, + newObjectMetadata(0L), im); invoker.retry("PUT 0-byte object ", objectName, true, @@ -3852,7 +3807,8 @@ public String toString() { if (committerIntegration != null) { sb.append(", magicCommitter=").append(isMagicCommitEnabled()); } - sb.append(", optimizeDirMarkers=").append(optimizeDirectoryOperations); + sb.append(", boundedExecutor=").append(boundedThreadPool); + sb.append(", unboundedExecutor=").append(unboundedThreadPool); sb.append(", credentials=").append(credentials); sb.append(", delegation tokens=") .append(delegationTokens.map(Objects::toString).orElse("disabled")); @@ -3950,40 +3906,25 @@ public boolean exists(Path f) throws IOException { } /** - * An optimized check which only looks for directory markers. + * Override superclass so as to add statistic collection. * {@inheritDoc} */ @Override @SuppressWarnings("deprecation") public boolean isDirectory(Path f) throws IOException { entryPoint(INVOCATION_IS_DIRECTORY); - try { - // against S3Guard, a full query; - // against S3 a HEAD + "/" then a LIST. - return innerGetFileStatus(f, false, - StatusProbeEnum.DIRECTORIES).isDirectory(); - } catch (FileNotFoundException e) { - return false; - } + return super.isDirectory(f); } /** - * Override superclass so as to only poll for a file. - * Warning: may leave a 404 in the S3 load balancer cache. + * Override superclass so as to add statistic collection. * {@inheritDoc} */ @Override @SuppressWarnings("deprecation") public boolean isFile(Path f) throws IOException { entryPoint(INVOCATION_IS_FILE); - try { - // against S3Guard, a full query; against S3 only a HEAD. - return innerGetFileStatus(f, false, - StatusProbeEnum.HEAD_ONLY).isFile(); - } catch (FileNotFoundException e) { - // no file or there is a directory there. - return false; - } + return super.isFile(f); } /** From 10f459e9242b5265ec616066b28c0db4da48f546 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 6 Feb 2020 18:54:38 +0000 Subject: [PATCH 12/17] HADOOP-16823 address intermittent test issues These tests can fail even before this patch went in, but it was as I was working on this where some of the problems were happening often enough that I could track down problems. Key: testDeleteTable() was not deleting a test table, it was deleting whichever table the bucket was bonded to, so potentially interfering with every other test. The stack traces were actually appearing in the next test which was run, testAncestorOverwriteConflict(), which would spin for 30-60s in setup before failing. Change-Id: I5e942d3854a5e1e496405c5be620768d2f81a83a --- .../s3guard/ITestDynamoDBMetadataStore.java | 27 ++++++++++++++----- .../ITestDynamoDBMetadataStoreScale.java | 1 - 2 files changed, 20 insertions(+), 8 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java index 4ad789965a06f..fc81c8d854109 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java @@ -130,8 +130,10 @@ public ITestDynamoDBMetadataStore() { private String bucket; + @SuppressWarnings("StaticNonFinalField") private static DynamoDBMetadataStore ddbmsStatic; + @SuppressWarnings("StaticNonFinalField") private static String testDynamoDBTableName; private static final List UNCHANGED_ENTRIES = Collections.emptyList(); @@ -166,13 +168,17 @@ public void setUp() throws Exception { try{ super.setUp(); - tableHandler = getDynamoMetadataStore().getTableHandler(); } catch (FileNotFoundException e){ LOG.warn("MetadataStoreTestBase setup failed. Waiting for table to be " - + "deleted before trying again."); - ddbmsStatic.getTable().waitForDelete(); + + "deleted before trying again.", e); + try { + ddbmsStatic.getTable().waitForDelete(); + } catch (IllegalArgumentException | InterruptedException ex) { + LOG.warn("When awaiting a table to be cleaned up", e); + } super.setUp(); } + tableHandler = getDynamoMetadataStore().getTableHandler(); } @BeforeClass @@ -780,10 +786,16 @@ private void removeVersionMarkerTag(Table table, AmazonDynamoDB addb) { .withTagKeys(VERSION_MARKER_TAG_NAME)); } - private void deleteVersionMarkerItem(Table table) { + /** + * Deletes a version marker; spins briefly to await it disappearing. + * @param table table to delete the key + * @throws Exception failure + */ + private void deleteVersionMarkerItem(Table table) throws Exception { table.deleteItem(VERSION_MARKER_PRIMARY_KEY); - assertNull("Version marker should be null after deleting it " + - "from the table.", table.getItem(VERSION_MARKER_PRIMARY_KEY)); + eventually(30_000, 1_0, () -> + assertNull("Version marker should be null after deleting it " + + "from the table.", table.getItem(VERSION_MARKER_PRIMARY_KEY))); } /** @@ -1003,7 +1015,8 @@ public void testDeleteTable() throws Exception { final String tableName = getTestTableName("testDeleteTable"); Path testPath = new Path(new Path(fsUri), "/" + tableName); final S3AFileSystem s3afs = getFileSystem(); - final Configuration conf = getTableCreationConfig(); + // patch the filesystem config as this is one read in initialize() + final Configuration conf = s3afs.getConf(); conf.set(S3GUARD_DDB_TABLE_NAME_KEY, tableName); enableOnDemand(conf); DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore(); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java index 6829d5f374b3b..ec50ac7cad25d 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java @@ -55,7 +55,6 @@ import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3AStorageStatistics; -import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.Statistic; import org.apache.hadoop.fs.s3a.scale.AbstractITestS3AMetadataStoreScale; import org.apache.hadoop.io.IOUtils; From 0441ad88a1f36d54f43bdd729d286d45d97f4e8c Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 10 Feb 2020 15:53:39 +0000 Subject: [PATCH 13/17] HADOOP-16823. throttling -improve retry handler. the new load test was not picking up throttle events in the retry handler. This is because the failure we are seeing is actually the XML parser error we've seen before when an opening connection is broken and the AWS SDK client's XML parser simply sees and report a failure of XML parsing, rather than change in the network or remote system. We have assumed until now there was a sign of network issues. The fact that is happening consistently when performing bulk delete operations makes me suspect that it is actually the S3 front end rejecting the caller. We are retrying on it, but treating it as a symptom of throttling and so updating the relevant counters. Change-Id: I5b6907ddd7d3eaec65d12064b10c89d953d85e46 --- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 17 +- .../fs/s3a/impl/BulkDeleteRetryHandler.java | 138 +++++++++++++ .../s3a/impl/StandardInvokeRetryHandler.java | 191 ------------------ .../ILoadTestS3ABulkDeleteThrottling.java | 24 +-- 4 files changed, 162 insertions(+), 208 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteRetryHandler.java delete mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StandardInvokeRetryHandler.java 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 251eec9417816..f4b29bf32b569 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 @@ -99,6 +99,7 @@ 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; +import org.apache.hadoop.fs.s3a.impl.BulkDeleteRetryHandler; import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; import org.apache.hadoop.fs.s3a.impl.ContextAccessors; import org.apache.hadoop.fs.s3a.impl.CopyOutcome; @@ -107,7 +108,6 @@ 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.StandardInvokeRetryHandler; import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum; import org.apache.hadoop.fs.s3a.impl.StoreContext; import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; @@ -1961,6 +1961,12 @@ private void blockRootDelete(String key) throws InvalidRequestException { * Increments the {@code OBJECT_DELETE_REQUESTS} and write * operation statistics. * Retry policy: retry untranslated; delete considered idempotent. + * If the request is throttled, this is logged in the throttle statistics, + * with the counter set to the number of keys, rather than the number of invocations + * of the delete operation. + * This is because S3 considers each key as one mutating operation on the store + * when updating its load counters on a specific partition of an S3 bucket. + * If only the request was measured, this operation would under-report. * @param deleteRequest keys to delete on the s3-backend * @return the AWS response * @throws MultiObjectDeleteException one or more of the keys could not @@ -1971,15 +1977,15 @@ private void blockRootDelete(String key) throws InvalidRequestException { private DeleteObjectsResult deleteObjects(DeleteObjectsRequest deleteRequest) throws MultiObjectDeleteException, AmazonClientException, IOException { incrementWriteOperations(); - StandardInvokeRetryHandler retryHandler = - new StandardInvokeRetryHandler(createStoreContext()); + BulkDeleteRetryHandler retryHandler = + new BulkDeleteRetryHandler(createStoreContext()); try(DurationInfo ignored = new DurationInfo(LOG, false, "DELETE %d keys", deleteRequest.getKeys().size())) { return invoker.retryUntranslated("delete", DELETE_CONSIDERED_IDEMPOTENT, (text, e, r, i) -> { - // error triggering retry + // handle the failure retryHandler.bulkDeleteRetried(deleteRequest, e); }, () -> { @@ -1987,7 +1993,8 @@ private DeleteObjectsResult deleteObjects(DeleteObjectsRequest deleteRequest) return s3.deleteObjects(deleteRequest); }); } catch (MultiObjectDeleteException e) { - // one or more of the operations failed. + // one or more of the keys could not be deleted. + // log and rethrow List errors = e.getErrors(); LOG.debug("Partial failure of delete, {} errors", errors.size(), e); for (MultiObjectDeleteException.DeleteError error : errors) { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteRetryHandler.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteRetryHandler.java new file mode 100644 index 0000000000000..748ea470a6287 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteRetryHandler.java @@ -0,0 +1,138 @@ +/* + * 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.util.List; + +import com.amazonaws.SdkClientException; +import com.amazonaws.services.s3.model.DeleteObjectsRequest; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.s3a.AWSClientIOException; +import org.apache.hadoop.fs.s3a.S3AInstrumentation; +import org.apache.hadoop.fs.s3a.S3AStorageStatistics; +import org.apache.hadoop.fs.s3a.Statistic; + +import static org.apache.hadoop.fs.s3a.S3AUtils.isThrottleException; +import static org.apache.hadoop.fs.s3a.Statistic.IGNORED_ERRORS; +import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_THROTTLED; +import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_THROTTLE_RATE; +import static org.apache.hadoop.fs.s3a.impl.InternalConstants.THROTTLE_LOG_NAME; + +/** + * Handler for bulk delete retry events. + */ +public class BulkDeleteRetryHandler extends AbstractStoreOperation { + + private static final Logger LOG = LoggerFactory.getLogger( + BulkDeleteRetryHandler.class); + + private static final Logger THROTTLE_LOG = LoggerFactory.getLogger( + THROTTLE_LOG_NAME); + + public static final String XML_PARSE_BROKEN = "Failed to parse XML document"; + + private final S3AInstrumentation instrumentation; + + private final S3AStorageStatistics storageStatistics; + + /** + * Constructor. + * @param storeContext context + */ + public BulkDeleteRetryHandler(final StoreContext storeContext) { + super(storeContext); + instrumentation = storeContext.getInstrumentation(); + storageStatistics = storeContext.getStorageStatistics(); + } + + /** + * Increment a statistic by 1. + * This increments both the instrumentation and storage statistics. + * @param statistic The operation to increment + */ + protected void incrementStatistic(Statistic statistic) { + incrementStatistic(statistic, 1); + } + + /** + * Increment a statistic by a specific value. + * This increments both the instrumentation and storage statistics. + * @param statistic The operation to increment + * @param count the count to increment + */ + protected void incrementStatistic(Statistic statistic, long count) { + instrumentation.incrementCounter(statistic, count); + storageStatistics.incrementCounter(statistic, count); + } + + /** + * Handler for failure of bulk delete requests. + * @param deleteRequest request which was retried. + * @param ex exception + */ + public void bulkDeleteRetried( + DeleteObjectsRequest deleteRequest, + Exception ex) { + LOG.debug("Retrying on error during bulk delete", ex); + if (isThrottleException(ex)) { + onDeleteThrottled(deleteRequest); + } else if (isSymptomOfBrokenConnection(ex)) { + // this is one which surfaces when an HTTPS connection is broken while + // the service is reading the result. + // it is treated as a throttle event for statistics + LOG.warn("Bulk delete operation interrupted: {}", ex.getMessage()); + onDeleteThrottled(deleteRequest); + } else { + incrementStatistic(IGNORED_ERRORS); + } + } + + /** + * Handle a delete throttling event. + * @param deleteRequest request which failed. + */ + private void onDeleteThrottled(final DeleteObjectsRequest deleteRequest) { + final List keys = deleteRequest.getKeys(); + final int size = keys.size(); + incrementStatistic(STORE_IO_THROTTLED, size); + instrumentation.addValueToQuantiles(STORE_IO_THROTTLE_RATE, size); + THROTTLE_LOG.warn( + "Bulk delete {} keys throttled -first key = {}; last = {}", + size, + keys.get(0).getKey(), + keys.get(size - 1).getKey()); + } + + /** + * Does this error indicate that the connection was ultimately broken while + * the XML Response was parsed? As this seems a symptom of the far end + * blocking the response (i.e. server-side throttling) while + * the client eventually times out. + * @param ex exception received. + * @return true if this exception is considered a sign of a broken connection. + */ + private boolean isSymptomOfBrokenConnection(final Exception ex) { + return ex instanceof AWSClientIOException + && ex.getCause() instanceof SdkClientException + && ex.getMessage().contains(XML_PARSE_BROKEN); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StandardInvokeRetryHandler.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StandardInvokeRetryHandler.java deleted file mode 100644 index 9f3615632540b..0000000000000 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StandardInvokeRetryHandler.java +++ /dev/null @@ -1,191 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.s3a.impl; - -import com.amazonaws.services.s3.model.DeleteObjectsRequest; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hadoop.fs.s3a.Invoker; -import org.apache.hadoop.fs.s3a.S3AInstrumentation; -import org.apache.hadoop.fs.s3a.S3AStorageStatistics; -import org.apache.hadoop.fs.s3a.Statistic; - -import static org.apache.hadoop.fs.s3a.S3AUtils.isThrottleException; -import static org.apache.hadoop.fs.s3a.Statistic.IGNORED_ERRORS; -import static org.apache.hadoop.fs.s3a.Statistic.S3GUARD_METADATASTORE_RETRY; -import static org.apache.hadoop.fs.s3a.Statistic.S3GUARD_METADATASTORE_THROTTLED; -import static org.apache.hadoop.fs.s3a.Statistic.S3GUARD_METADATASTORE_THROTTLE_RATE; -import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_THROTTLED; -import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_THROTTLE_RATE; -import static org.apache.hadoop.fs.s3a.impl.InternalConstants.THROTTLE_LOG_NAME; - -/** - * Handler for retry callbacks. - * This is intended to be pulled out for all operations, but for now it is only - * used in delete objects. - */ -public class StandardInvokeRetryHandler extends AbstractStoreOperation { - - - private static final Logger LOG = LoggerFactory.getLogger( - StandardInvokeRetryHandler.class); - - private static final Logger THROTTLE_LOG = LoggerFactory.getLogger( - THROTTLE_LOG_NAME); - - private final S3AInstrumentation instrumentation; - - private final S3AStorageStatistics storageStatistics; - - public StandardInvokeRetryHandler(final StoreContext storeContext) { - super(storeContext); - instrumentation = storeContext.getInstrumentation(); - storageStatistics = storeContext.getStorageStatistics(); - } - - /** - * Increment a statistic by 1. - * This increments both the instrumentation and storage statistics. - * @param statistic The operation to increment - */ - protected void incrementStatistic(Statistic statistic) { - incrementStatistic(statistic, 1); - } - - /** - * Increment a statistic by a specific value. - * This increments both the instrumentation and storage statistics. - * @param statistic The operation to increment - * @param count the count to increment - */ - protected void incrementStatistic(Statistic statistic, long count) { - instrumentation.incrementCounter(statistic, count); - storageStatistics.incrementCounter(statistic, count); - } - - /** - * Decrement a gauge by a specific value. - * @param statistic The operation to decrement - * @param count the count to decrement - */ - protected void decrementGauge(Statistic statistic, long count) { - instrumentation.decrementGauge(statistic, count); - } - - /** - * Increment a gauge by a specific value. - * @param statistic The operation to increment - * @param count the count to increment - */ - protected void incrementGauge(Statistic statistic, long count) { - instrumentation.incrementGauge(statistic, count); - } - - /** - * Callback when an operation was retried. - * Increments the statistics of ignored errors or throttled requests, - * depending up on the exception class. - * @param ex exception. - */ - public void operationRetried(Exception ex) { - if (isThrottleException(ex)) { - operationThrottled(false); - } else { - incrementStatistic(IGNORED_ERRORS); - } - } - - /** - * Callback from {@link Invoker} when an operation is retried. - * @param text text of the operation - * @param ex exception - * @param retries number of retries - * @param idempotent is the method idempotent - */ - public void operationRetried( - String text, - Exception ex, - int retries, - boolean idempotent) { - operationRetried(ex); - } - - /** - * Callback from {@link Invoker} when an operation against a metastore - * is retried. - * Always increments the {@link Statistic#S3GUARD_METADATASTORE_RETRY} - * statistic/counter; - * if it is a throttling exception will update the associated - * throttled metrics/statistics. - * - * @param ex exception - * @param retries number of retries - * @param idempotent is the method idempotent - */ - public void metastoreOperationRetried(Exception ex, - int retries, - boolean idempotent) { - incrementStatistic(S3GUARD_METADATASTORE_RETRY); - if (isThrottleException(ex)) { - operationThrottled(true); - } else { - incrementStatistic(IGNORED_ERRORS); - } - } - - - /** - * Handler for failure of bulk delete requests. - * @param deleteRequest request which was retried. - * @param ex exception - */ - public void bulkDeleteRetried( - DeleteObjectsRequest deleteRequest, - Exception ex) { - if (isThrottleException(ex)) { - final int size = deleteRequest.getKeys().size(); - incrementStatistic(STORE_IO_THROTTLED, size); - instrumentation.addValueToQuantiles(STORE_IO_THROTTLE_RATE, size); - THROTTLE_LOG.warn("Bulk delete {} keys throttled -first key = {}", - size, deleteRequest.getKeys().get(0)); - } else { - incrementStatistic(IGNORED_ERRORS); - } - } - - - /** - * Note that an operation was throttled -this will update - * specific counters/metrics. - * @param metastore was the throttling observed in the S3Guard metastore? - */ - private void operationThrottled(boolean metastore) { - THROTTLE_LOG.debug("Request throttled on {}", metastore ? "S3" : "DynamoDB"); - if (metastore) { - incrementStatistic(S3GUARD_METADATASTORE_THROTTLED); - instrumentation.addValueToQuantiles(S3GUARD_METADATASTORE_THROTTLE_RATE, - 1); - } else { - incrementStatistic(STORE_IO_THROTTLED); - instrumentation.addValueToQuantiles(STORE_IO_THROTTLE_RATE, 1); - } - } - -} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ILoadTestS3ABulkDeleteThrottling.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ILoadTestS3ABulkDeleteThrottling.java index 06d822edcfeb4..32415894ba36f 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ILoadTestS3ABulkDeleteThrottling.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ILoadTestS3ABulkDeleteThrottling.java @@ -76,8 +76,8 @@ public class ILoadTestS3ABulkDeleteThrottling extends S3AScaleTestBase { private static final Logger LOG = LoggerFactory.getLogger(ILoadTestS3ABulkDeleteThrottling.class); - protected static final int THREADS = 40; - public static final int TOTAL_KEYS = 15000; + protected static final int THREADS = 20; + public static final int TOTAL_KEYS = 25000; public static final int SMALL = BULK_DELETE_PAGE_SIZE_DEFAULT; public static final int SMALL_REQS = TOTAL_KEYS / SMALL; @@ -198,7 +198,6 @@ public void test_020_DeleteThrottling() throws Throwable { } } - @Test public void test_030_Sleep() throws Throwable { maybeSleep(); @@ -215,17 +214,18 @@ private void maybeSleep() throws InterruptedException, IOException { /** * delete files. - * @param requests number of requests. + * @param requestCount number of requests. * @throws Exception failure * @return CSV filename */ - private File deleteFiles(final int requests, final int entries) + private File deleteFiles(final int requestCount, + final int entries) throws Exception { File csvFile = new File(dataDir, String.format("delete-%03d-%04d-%s.csv", - requests, entries, throttle)); + requestCount, entries, throttle)); describe("Issuing %d requests of size %d, saving log to %s", - requests, entries, csvFile); + requestCount, entries, csvFile); Path basePath = path("testDeleteObjectThrottling"); final S3AFileSystem fs = getFileSystem(); final String base = fs.pathToKey(basePath); @@ -238,7 +238,7 @@ private File deleteFiles(final int requests, final int entries) final ContractTestUtils.NanoTimer jobTimer = new ContractTestUtils.NanoTimer(); - for (int i = 0; i < requests; i++) { + for (int i = 0; i < requestCount; i++) { final int id = i; completionService.submit(() -> { final long startTime = System.currentTimeMillis(); @@ -262,7 +262,7 @@ private File deleteFiles(final int requests, final int entries) NanoTimerStats success = new NanoTimerStats("Successful"); NanoTimerStats throttled = new NanoTimerStats("Throttled"); List throttledEvents = new ArrayList<>(); - for (int i = 0; i < requests; i++) { + for (int i = 0; i < requestCount; i++) { Outcome outcome = completionService.take().get(); ContractTestUtils.NanoTimer timer = outcome.timer; Exception ex = outcome.exception; @@ -284,19 +284,19 @@ private File deleteFiles(final int requests, final int entries) // now print the stats LOG.info("Summary file is " + csvFile); LOG.info("Made {} requests with {} throttle events\n: {}\n{}\n{}", - requests, + requestCount, throttled.getCount(), stats, throttled, success); double duration = jobTimer.duration(); - double iops = requests * entries * 1.0e9 / duration; + double iops = requestCount * entries * 1.0e9 / duration; LOG.info(String.format("TPS %3f operations/second", iops)); // log at debug if (LOG.isDebugEnabled()) { - throttledEvents.stream().forEach((outcome -> { + throttledEvents.forEach((outcome -> { LOG.debug("{}: duration: {}", outcome.id, outcome.timer.elapsedTimeMs()); })); From fda06de6615cbc5c7ae6e86fea7c9447e9c620d1 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 10 Feb 2020 16:48:04 +0000 Subject: [PATCH 14/17] HADOOP-16823 throttling -gabor's comments Change-Id: I55a7d6d77accdf7393e147db2866300495d11f5b --- .../main/java/org/apache/hadoop/fs/s3a/Constants.java | 10 +++++++++- .../apache/hadoop/fs/s3a/DefaultS3ClientFactory.java | 9 +++++++-- 2 files changed, 16 insertions(+), 3 deletions(-) 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 55250338475f0..ccdd02e2d3d96 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 @@ -200,7 +200,15 @@ private Constants() { */ @InterfaceStability.Unstable public static final String EXPERIMENTAL_AWS_INTERNAL_THROTTLING = - "fs.s3a.experimental.aws.internal.throttling"; + "fs.s3a.experimental.aws.s3.throttling"; + + /** + * Default value of {@link #EXPERIMENTAL_AWS_INTERNAL_THROTTLING}, + * value: {@value}. + */ + @InterfaceStability.Unstable + public static final boolean EXPERIMENTAL_AWS_INTERNAL_THROTTLING_DEFAULT = + true; // seconds until we give up trying to establish a connection to s3 public static final String ESTABLISH_TIMEOUT = 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 5fc31e9646426..276961bf8b7e1 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 @@ -36,6 +36,7 @@ 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; /** @@ -59,10 +60,14 @@ public AmazonS3 createS3Client(URI name, final ClientConfiguration awsConf = S3AUtils .createAwsConf(conf, bucket, Constants.AWS_SERVICE_IDENTIFIER_S3); + // When EXPERIMENTAL_AWS_INTERNAL_THROTTLING is false // throttling is explicitly disabled on the S3 client so that - // all failures are collected + // all failures are collected in S3A instrumentation, and its + // retry policy is the only one used. + // This may cause problems in copy/rename. awsConf.setUseThrottleRetries( - conf.getBoolean(EXPERIMENTAL_AWS_INTERNAL_THROTTLING, true)); + conf.getBoolean(EXPERIMENTAL_AWS_INTERNAL_THROTTLING, + EXPERIMENTAL_AWS_INTERNAL_THROTTLING_DEFAULT)); if (!StringUtils.isEmpty(userAgentSuffix)) { awsConf.setUserAgentSuffix(userAgentSuffix); From 6caa05fb8a1d8638567ee15cf4cc6be5ea16ae86 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 11 Feb 2020 15:29:10 +0000 Subject: [PATCH 15/17] HADOOP-16283: cleanup and deadlock avoidance * remove the async stuff from the end of rename() * keep dir marker delete operations in finishedWrite() async, but use the unbounded thread pool. * Cleanup + enhancement of ITestS3ADeleteManyFiles so that it tests src and dest paths more rigorously, * and sets a page size of 50 for better coverage of the paged rename sequence. Change-Id: I334d70cc52c73bd926ccd1414e11a0ba740d9b89 --- .../org/apache/hadoop/fs/s3a/Constants.java | 2 +- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 32 ++---- .../hadoop/fs/s3a/impl/DeleteOperation.java | 1 - .../hadoop/fs/s3a/impl/RenameOperation.java | 6 +- .../fs/s3a/s3guard/DynamoDBMetadataStore.java | 4 +- .../s3a/s3guard/PurgeS3GuardDynamoTable.java | 3 +- .../fs/s3a/s3guard/RetryingCollection.java | 51 +++++---- .../site/markdown/tools/hadoop-aws/testing.md | 8 +- .../s3a/impl/ITestPartialRenamesDeletes.java | 13 ++- .../ITestDynamoDBMetadataStoreScale.java | 9 +- .../fs/s3a/s3guard/ThrottleTracker.java | 3 +- .../ILoadTestS3ABulkDeleteThrottling.java | 3 +- .../fs/s3a/scale/ITestS3ADeleteManyFiles.java | 102 +++++++++++++----- 13 files changed, 154 insertions(+), 83 deletions(-) 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 ccdd02e2d3d96..561ab4a84a7c3 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 @@ -275,7 +275,7 @@ private Constants() { * applications start working with the same bucket these large * deletes can be highly disruptive. */ - public static final String BULK_DELETE_PAGE_SIZE = + public static final String BULK_DELETE_PAGE_SIZE = "fs.s3a.bulk.delete.page.size"; /** 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 f4b29bf32b569..37131a6a6cf7d 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 @@ -1496,21 +1496,8 @@ public void finishRename(final Path sourceRenamed, final Path destCreated) Path destParent = destCreated.getParent(); if (!sourceRenamed.getParent().equals(destParent)) { LOG.debug("source & dest parents are different; fix up dir markers"); - // kick off an async delete - List> ops = new ArrayList<>(2); - ops.add(submit( - boundedThreadPool, - () -> { - deleteUnnecessaryFakeDirectories(destParent); - return null; - })); - ops.add(submit( - boundedThreadPool, - () -> { - maybeCreateFakeParentDirectory(sourceRenamed); - return null; - })); - waitForCompletion(ops); + deleteUnnecessaryFakeDirectories(destParent); + maybeCreateFakeParentDirectory(sourceRenamed); } } @@ -1962,10 +1949,11 @@ private void blockRootDelete(String key) throws InvalidRequestException { * operation statistics. * Retry policy: retry untranslated; delete considered idempotent. * If the request is throttled, this is logged in the throttle statistics, - * with the counter set to the number of keys, rather than the number of invocations - * of the delete operation. - * This is because S3 considers each key as one mutating operation on the store - * when updating its load counters on a specific partition of an S3 bucket. + * with the counter set to the number of keys, rather than the number + * of invocations of the delete operation. + * This is because S3 considers each key as one mutating operation on + * the store when updating its load counters on a specific partition + * of an S3 bucket. * If only the request was measured, this operation would under-report. * @param deleteRequest keys to delete on the s3-backend * @return the AWS response @@ -3575,7 +3563,7 @@ void finishedWrite(String key, long length, String eTag, String versionId, final boolean isDir = objectRepresentsDirectory(key, length); // kick off an async delete final CompletableFuture deletion = submit( - boundedThreadPool, + unboundedThreadPool, () -> { deleteUnnecessaryFakeDirectories(p.getParent()); return null; @@ -3593,7 +3581,9 @@ void finishedWrite(String key, long length, String eTag, String versionId, // information gleaned from addAncestors is preserved into the // subsequent put. stateToClose = S3Guard.initiateBulkWrite(metadataStore, - BulkOperationState.OperationType.Mkdir, + isDir + ? BulkOperationState.OperationType.Mkdir + : BulkOperationState.OperationType.Put, keyToPath(key)); activeState = stateToClose; } 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 8a1d37d7f6a6e..daf93d99bd9be 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 @@ -47,7 +47,6 @@ import static com.google.common.base.Preconditions.checkArgument; import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.maybeAwaitCompletion; import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit; -import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletion; /** * Implementation of the delete() operation. 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 b89c0fba82ef3..750aebf500a4b 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 @@ -98,8 +98,12 @@ public class RenameOperation extends ExecutingStoreOperation { /** * Counter of bytes copied. */ + private final AtomicLong bytesCopied = new AtomicLong(); + /** + * Page size for bulk deletes. + */ private final int pageSize; /** @@ -138,7 +142,7 @@ public class RenameOperation extends ExecutingStoreOperation { * @param destKey destination key * @param destStatus destination status. * @param callbacks callback provider - * @param pageSize size of delete pages + * @param pageSize size of delete requests */ public RenameOperation( final StoreContext storeContext, 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 42538cb7ba928..38b38fb7f93e7 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 @@ -2081,7 +2081,7 @@ public long getBatchWriteCapacityExceededCount() { * Get the operation invoker for write operations. * @return an invoker for retrying mutating operations on a store. */ - public Invoker getWriteOperationInvoker() { + public Invoker getInvoker() { return writeOp; } @@ -2094,7 +2094,7 @@ public Invoker getWriteOperationInvoker() { */ public Iterable wrapWithRetries( final Iterable source) { - return new RetryingCollection<>(scanOp, source); + return new RetryingCollection<>("scan dynamoDB table", scanOp, source); } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PurgeS3GuardDynamoTable.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PurgeS3GuardDynamoTable.java index 7caac5c140678..ad298c222adb2 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PurgeS3GuardDynamoTable.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PurgeS3GuardDynamoTable.java @@ -174,13 +174,12 @@ public int execute() throws ServiceLaunchException, IOException { count, ddbms.toString()); // sending this in one by one for more efficient retries for (Path path: list) { - ddbms.getWriteOperationInvoker() + ddbms.getInvoker() .retry("delete", prefix, true, () -> tableAccess.delete(path)); } - duration.close(); long durationMillis = duration.value(); long timePerEntry = durationMillis / count; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RetryingCollection.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RetryingCollection.java index 36667db7b92bb..43ba0c522de70 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RetryingCollection.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RetryingCollection.java @@ -27,14 +27,12 @@ /** * A collection which wraps the result of a query or scan - * with retries; the {@link #scanThrottleEvents} count is - * then updated. + * with retries. * Important: iterate through this only once; the outcome * of repeating an iteration is "undefined" * @param type of outcome. */ -class RetryingCollection - implements Iterable { +class RetryingCollection implements Iterable { /** * Source iterable. @@ -44,35 +42,48 @@ class RetryingCollection /** * Invoker for retries. */ - private Invoker invoker; + private final Invoker invoker; + /** + * Operation name for invoker.retry messages. + */ + private final String operation; + + /** + * Constructor, + * @param operation Operation name for invoker.retry messages. + * @param invoker Invoker for retries. + * @param source Source iterable. + */ RetryingCollection( + final String operation, final Invoker invoker, final Iterable source) { + this.operation = operation; this.source = source; this.invoker = invoker; } - + /** + * Demand creates a new iterator which will retry all hasNext/next + * operations through the invoker supplied in the constructor. + * @return a new iterator. + */ @Override public Iterator iterator() { - return new RetryingIterator<>(invoker, source.iterator()); + return new RetryingIterator(source.iterator()); } /** * An iterator which wraps a non-retrying iterator of scan results * (i.e {@code S3GuardTableAccess.DDBPathMetadataIterator}. */ - private static final class RetryingIterator implements Iterator { + private final class RetryingIterator implements Iterator { - private final Iterator source; + private final Iterator iterator; - private Invoker scanOp; - - private RetryingIterator(final Invoker scanOp, - final Iterator source) { - this.source = source; - this.scanOp = scanOp; + private RetryingIterator(final Iterator iterator) { + this.iterator = iterator; } /** @@ -83,11 +94,11 @@ private RetryingIterator(final Invoker scanOp, @Retries.RetryTranslated public boolean hasNext() { try { - return scanOp.retry( - "Scan Dynamo", + return invoker.retry( + operation, null, true, - source::hasNext); + iterator::hasNext); } catch (IOException e) { throw new WrappedIOException(e); } @@ -101,11 +112,11 @@ public boolean hasNext() { @Retries.RetryTranslated public T next() { try { - return scanOp.retry( + return invoker.retry( "Scan Dynamo", null, true, - source::next); + iterator::next); } catch (IOException e) { throw new WrappedIOException(e); } diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md index b4bd1b061cb69..780b8aa02307c 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md @@ -465,7 +465,8 @@ The tests are executed in an order to only clean up created files after the end of all the tests. If the tests are interrupted, the test data will remain. ## Load tests. -There are a few tests which are designed to overload AWS services with more + +Some are designed to overload AWS services with more requests per second than an AWS account is permitted. The operation of these test maybe observable to other users of the same @@ -478,7 +479,7 @@ These tests all have the prefix `ILoadTest` They do not run automatically: they must be explicitly run from the command line or an IDE. -Look in the source for these and reads the Java docs before executing. +Look in the source for these and reads the Javadocs before executing. ## Testing against non AWS S3 endpoints. @@ -1389,6 +1390,9 @@ as it may take a couple of SDK updates before it is ready. in `fs.s3a.assumed.role.arn` for testing assumed roles, and `fs.s3a.server-side-encryption.key` for encryption, for full coverage. If you can, scale up the scale tests. +1. Run the `ILoadTest*` load tests from your IDE or via maven through + `mvn verify -Dtest=skip -Dit.test=ILoadTest\*` ; look for regressions in performance + as much as failures. 1. Create the site with `mvn site -DskipTests`; look in `target/site` for the report. 1. Review *every single `-output.txt` file in `hadoop-tools/hadoop-aws/target/failsafe-reports`, paying particular attention to diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestPartialRenamesDeletes.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestPartialRenamesDeletes.java index 69ca553b9ce93..1b53ef5a9185b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestPartialRenamesDeletes.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestPartialRenamesDeletes.java @@ -818,16 +818,25 @@ private static void buildPaths( } // create the file paths for (int i = 0; i < fileCount; i++) { - String name = PREFIX + i; + String name = filenameOfIndex(i); Path p = new Path(destDir, name); filePaths.add(p); } for (int i = 0; i < dirCount; i++) { - String name = "dir-" + i; + String name = String.format("dir-%03d", i); Path p = new Path(destDir, name); dirPaths.add(p); buildPaths(filePaths, dirPaths, p, depth - 1, fileCount, dirCount); } } + + /** + * Given an index, return a string to use as the filename. + * @param i index + * @return name + */ + public static String filenameOfIndex(final int i) { + return String.format("%s%03d", PREFIX, i); + } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java index ec50ac7cad25d..75b630ae22a4b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java @@ -203,7 +203,8 @@ public void teardown() throws Exception { /** * Is throttling likely? - * @return true if the DDB table has prepaid IO and is small enough to throttle. + * @return true if the DDB table has prepaid IO and is small enough + * to throttle. */ private boolean expectThrottling() { return !isOverProvisionedForTest && !isOnDemandTable; @@ -378,7 +379,7 @@ public void test_050_getVersionMarkerItem() throws Throwable { */ private void retryingDelete(final Path path) { try { - ddbms.getWriteOperationInvoker().retry("Delete ", path.toString(), true, + ddbms.getInvoker().retry("Delete ", path.toString(), true, () -> ddbms.delete(path, null)); } catch (IOException e) { LOG.warn("Failed to delete {}: ", path, e); @@ -437,7 +438,7 @@ public void test_080_fullPathsToPut() throws Throwable { BulkOperationState.OperationType.Put, child)) { ddbms.put(new PathMetadata(makeDirStatus(base)), bulkUpdate); ddbms.put(new PathMetadata(makeDirStatus(child)), bulkUpdate); - ddbms.getWriteOperationInvoker().retry("set up directory tree", + ddbms.getInvoker().retry("set up directory tree", base.toString(), true, () -> ddbms.put(pms, bulkUpdate)); @@ -539,7 +540,7 @@ public void test_999_delete_all_entries() throws Throwable { } // sending this in one by one for more efficient retries for (Path p : list) { - ddbms.getWriteOperationInvoker() + ddbms.getInvoker() .retry("delete", path, true, diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ThrottleTracker.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ThrottleTracker.java index 4c2f2fd09eab2..0dad1bf03d7bd 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ThrottleTracker.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ThrottleTracker.java @@ -110,7 +110,8 @@ public String toString() { */ public boolean probeThrottlingDetected() { if (!isThrottlingDetected()) { - LOG.warn("No throttling detected in {} against {}", this, ddbms.toString()); + LOG.warn("No throttling detected in {} against {}", + this, ddbms); return false; } return true; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ILoadTestS3ABulkDeleteThrottling.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ILoadTestS3ABulkDeleteThrottling.java index 32415894ba36f..a1d5c46159aaa 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ILoadTestS3ABulkDeleteThrottling.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ILoadTestS3ABulkDeleteThrottling.java @@ -114,7 +114,8 @@ public class ILoadTestS3ABulkDeleteThrottling extends S3AScaleTestBase { * * @return a list of parameter tuples. */ - @Parameterized.Parameters(name = "bulk-delete-client-retry={0}-requests={2}-size={1}") + @Parameterized.Parameters( + name = "bulk-delete-aws-retry={0}-requests={2}-size={1}") public static Collection params() { return Arrays.asList(new Object[][]{ {false, SMALL, SMALL_REQS}, diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADeleteManyFiles.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADeleteManyFiles.java index bc3224a6c2dce..efaec5f4fa5f0 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADeleteManyFiles.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADeleteManyFiles.java @@ -18,27 +18,58 @@ package org.apache.hadoop.fs.s3a.scale; +import java.util.Arrays; + +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; 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.impl.ITestPartialRenamesDeletes; +import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.util.DurationInfo; +import org.assertj.core.api.Assertions; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.hadoop.fs.contract.ContractTestUtils.rm; +import static org.apache.hadoop.fs.s3a.Constants.BULK_DELETE_PAGE_SIZE; +import static org.apache.hadoop.fs.s3a.Constants.EXPERIMENTAL_AWS_INTERNAL_THROTTLING; +import static org.apache.hadoop.fs.s3a.Constants.USER_AGENT_PREFIX; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.lsR; import static org.apache.hadoop.fs.s3a.impl.ITestPartialRenamesDeletes.createFiles; +import static org.apache.hadoop.fs.s3a.impl.ITestPartialRenamesDeletes.filenameOfIndex; /** * Test some scalable operations related to file renaming and deletion. + * We set a bulk page size low enough that even the default test scale will + * issue multiple delete requests during a delete sequence -so test that + * operation more efficiently. */ public class ITestS3ADeleteManyFiles extends S3AScaleTestBase { + private static final Logger LOG = LoggerFactory.getLogger(ITestS3ADeleteManyFiles.class); - public static final String PREFIX = ITestPartialRenamesDeletes.PREFIX; + /** + * Delete Page size: {@value}. + */ + static final int DELETE_PAGE_SIZE = 50; + + + @Override + protected Configuration createScaleConfiguration() { + Configuration conf = super.createScaleConfiguration(); + S3ATestUtils.disableFilesystemCaching(conf); + S3ATestUtils.removeBaseAndBucketOverrides(conf, + EXPERIMENTAL_AWS_INTERNAL_THROTTLING, + BULK_DELETE_PAGE_SIZE, + USER_AGENT_PREFIX); + conf.setBoolean(EXPERIMENTAL_AWS_INTERNAL_THROTTLING, false); + conf.setInt(BULK_DELETE_PAGE_SIZE, DELETE_PAGE_SIZE); + return conf; + } /** * CAUTION: If this test starts failing, please make sure that the @@ -46,33 +77,37 @@ public class ITestS3ADeleteManyFiles extends S3AScaleTestBase { * set too low. Alternatively, consider reducing the * scale.test.operation.count parameter in * getOperationCount(). - * + * If it is slow: look at the size of any S3Guard Table used. * @see #getOperationCount() */ @Test public void testBulkRenameAndDelete() throws Throwable { - final Path scaleTestDir = path("testBulkRenameAndDelete"); - final Path srcDir = new Path(scaleTestDir, "src"); - final Path finalDir = new Path(scaleTestDir, "final"); final int count = getConf().getInt(KEY_FILE_COUNT, DEFAULT_FILE_COUNT); + describe("Testing bulk rename and delete of %d files", count); + + final Path scaleTestDir = path("testBulkRenameAndDelete"); + final Path srcParentDir = new Path(scaleTestDir, "srcParent"); + final Path srcDir = new Path(srcParentDir, "src"); + final Path finalParentDir = new Path(scaleTestDir, "finalParent"); + final Path finalDir = new Path(finalParentDir, "final"); final S3AFileSystem fs = getFileSystem(); - ContractTestUtils.rm(fs, scaleTestDir, true, false); + rm(fs, scaleTestDir, true, false); fs.mkdirs(srcDir); + fs.mkdirs(finalParentDir); createFiles(fs, srcDir, 1, count, 0); FileStatus[] statuses = fs.listStatus(srcDir); int nSrcFiles = statuses.length; - long sourceSize = 0; - for (FileStatus status : statuses) { - sourceSize += status.getLen(); - } + long sourceSize = Arrays.stream(statuses) + .mapToLong(FileStatus::getLen) + .sum(); assertEquals("Source file Count", count, nSrcFiles); ContractTestUtils.NanoTimer renameTimer = new ContractTestUtils.NanoTimer(); try (DurationInfo ignored = new DurationInfo(LOG, "Rename %s to %s", srcDir, finalDir)) { - assertTrue("Rename failed", fs.rename(srcDir, finalDir)); + rename(srcDir, finalDir); } renameTimer.end(); LOG.info("Effective rename bandwidth {} MB/s", @@ -80,19 +115,31 @@ public void testBulkRenameAndDelete() throws Throwable { LOG.info(String.format( "Time to rename a file: %,03f milliseconds", (renameTimer.nanosPerOperation(count) * 1.0f) / 1.0e6)); - assertEquals(nSrcFiles, fs.listStatus(finalDir).length); - ContractTestUtils.assertPathDoesNotExist(fs, "not deleted after rename", - new Path(srcDir, PREFIX + 0)); - ContractTestUtils.assertPathDoesNotExist(fs, "not deleted after rename", - new Path(srcDir, PREFIX + count / 2)); - ContractTestUtils.assertPathDoesNotExist(fs, "not deleted after rename", - new Path(srcDir, PREFIX + (count - 1))); - ContractTestUtils.assertPathExists(fs, "not renamed to dest dir", - new Path(finalDir, PREFIX + 0)); - ContractTestUtils.assertPathExists(fs, "not renamed to dest dir", - new Path(finalDir, PREFIX + count/2)); - ContractTestUtils.assertPathExists(fs, "not renamed to dest dir", - new Path(finalDir, PREFIX + (count-1))); + Assertions.assertThat(lsR(fs, srcParentDir, true)) + .describedAs("Recursive listing of source dir %s", srcParentDir) + .isEqualTo(0); + + assertPathDoesNotExist("not deleted after rename", + new Path(srcDir, filenameOfIndex(0))); + assertPathDoesNotExist("not deleted after rename", + new Path(srcDir, filenameOfIndex(count / 2))); + assertPathDoesNotExist("not deleted after rename", + new Path(srcDir, filenameOfIndex(count - 1))); + + // audit destination + Assertions.assertThat(lsR(fs, finalDir, true)) + .describedAs("size of recursive destination listFiles(%s)", finalDir) + .isEqualTo(count); + Assertions.assertThat(fs.listStatus(finalDir)) + .describedAs("size of destination listStatus(%s)", finalDir) + .hasSize(count); + + assertPathExists("not renamed to dest dir", + new Path(finalDir, filenameOfIndex(0))); + assertPathExists("not renamed to dest dir", + new Path(finalDir, filenameOfIndex(count / 2))); + assertPathExists("not renamed to dest dir", + new Path(finalDir, filenameOfIndex(count - 1))); ContractTestUtils.NanoTimer deleteTimer = new ContractTestUtils.NanoTimer(); @@ -104,6 +151,11 @@ public void testBulkRenameAndDelete() throws Throwable { LOG.info(String.format( "Time to delete an object %,03f milliseconds", (deleteTimer.nanosPerOperation(count) * 1.0f) / 1.0e6)); + Assertions.assertThat(lsR(fs, finalParentDir, true)) + .describedAs("Recursive listing of deleted rename destination %s", + finalParentDir) + .isEqualTo(0); + } } From 3e5ca3e12d196572da74539018dbbf0dfab6b820 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 11 Feb 2020 22:47:31 +0000 Subject: [PATCH 16/17] HADOOP-16823. Checkstyles Change-Id: I5fe8caab3b490904ef50522ca1dc0c7888fc79dc --- .../src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java | 1 - .../org/apache/hadoop/fs/s3a/s3guard/RetryingCollection.java | 2 +- 2 files changed, 1 insertion(+), 2 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 37131a6a6cf7d..ce7729fa396ea 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 @@ -172,7 +172,6 @@ import static org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens.TokenIssuingPolicy.NoTokensAvailable; import static org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens.hasDelegationTokenBinding; 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.s3a.impl.CallableSupplier.waitForCompletionIgnoringExceptions; import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_404; import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.fixBucketRegion; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RetryingCollection.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RetryingCollection.java index 43ba0c522de70..394f393c57ec5 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RetryingCollection.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RetryingCollection.java @@ -50,7 +50,7 @@ class RetryingCollection implements Iterable { private final String operation; /** - * Constructor, + * Constructor. * @param operation Operation name for invoker.retry messages. * @param invoker Invoker for retries. * @param source Source iterable. From fbf3d497d233c1e8f38ee2fdafe7124c0c034116 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 13 Feb 2020 18:46:12 +0000 Subject: [PATCH 17/17] HADOOP-16823 add a javadoc for the XML_PARSE_BROKEN string Change-Id: I833700b25f4c8cfb16a89f843d441edfbf440e59 --- .../org/apache/hadoop/fs/s3a/impl/BulkDeleteRetryHandler.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteRetryHandler.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteRetryHandler.java index 748ea470a6287..71b8cb71e5931 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteRetryHandler.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteRetryHandler.java @@ -47,6 +47,10 @@ public class BulkDeleteRetryHandler extends AbstractStoreOperation { private static final Logger THROTTLE_LOG = LoggerFactory.getLogger( THROTTLE_LOG_NAME); + /** + * This is an error string we see in exceptions when the XML parser + * failed: {@value}. + */ public static final String XML_PARSE_BROKEN = "Failed to parse XML document"; private final S3AInstrumentation instrumentation;