From 14ddf92309e281a0fcc6ebcb35a9a4338520429a Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 19 Apr 2019 18:36:59 -0700 Subject: [PATCH 01/22] HADOOP-15183: previous patch coalesced into a new PR Change-Id: I606370726a929516bde298c8973fbf617ff85df6 --- .../hadoop/fs/contract/ContractTestUtils.java | 4 +- hadoop-tools/hadoop-aws/pom.xml | 5 + .../apache/hadoop/fs/s3a/S3AFileSystem.java | 136 +++++- .../hadoop/fs/s3a/S3AInstrumentation.java | 3 +- .../org/apache/hadoop/fs/s3a/S3AUtils.java | 35 +- .../org/apache/hadoop/fs/s3a/Statistic.java | 2 + .../fs/s3a/impl/MultiObjectDeleteSupport.java | 201 ++++++++ .../hadoop/fs/s3a/impl/StoreContext.java | 273 +++++++++++ .../apache/hadoop/fs/s3a/s3guard/S3Guard.java | 3 + .../fs/s3a/ITestS3AFailureHandling.java | 101 +++- .../apache/hadoop/fs/s3a/S3ATestUtils.java | 4 + .../hadoop/fs/s3a/auth/ITestAssumeRole.java | 203 +------- .../hadoop/fs/s3a/auth/RoleTestUtils.java | 49 +- .../s3a/impl/ITestPartialRenamesDeletes.java | 446 ++++++++++++++++++ .../s3a/impl/TestPartialDeleteFailures.java | 326 +++++++++++++ .../hadoop/fs/s3a/test/ExtraAssertions.java | 124 +++++ 16 files changed, 1650 insertions(+), 265 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteSupport.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestPartialRenamesDeletes.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/ExtraAssertions.java 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 b4db3a5803ad8..4d975a338cd61 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 @@ -662,11 +662,13 @@ public static void appendFile(FileSystem fs, * Touch a file. * @param fs filesystem * @param path path + * @return the patch created. * @throws IOException IO problems */ - public static void touch(FileSystem fs, + public static Path touch(FileSystem fs, Path path) throws IOException { createFile(fs, path, true, null); + return path; } /** diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml index 9419e48532eb1..7464ee36564b0 100644 --- a/hadoop-tools/hadoop-aws/pom.xml +++ b/hadoop-tools/hadoop-aws/pom.xml @@ -422,6 +422,11 @@ wildfly-openssl runtime + + org.assertj + assertj-core + test + junit junit 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 7c82aa6b90d6e..af524fd71aa7f 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 @@ -89,6 +89,8 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; +import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport; +import org.apache.hadoop.fs.s3a.impl.StoreContext; import org.apache.hadoop.fs.s3a.select.InternalSelectConstants; import org.apache.hadoop.util.LambdaUtils; import org.apache.hadoop.fs.FileAlreadyExistsException; @@ -242,6 +244,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, private S3Guard.ITtlTimeProvider ttlTimeProvider; + private String bucketLocation; + /** Add any deprecated keys. */ @SuppressWarnings("deprecation") private static void addDeprecatedKeys() { @@ -358,6 +362,13 @@ public void initialize(URI name, Configuration originalConf) // the encryption algorithms) bindAWSClient(name, delegationTokensEnabled); + try { + bucketLocation = getBucketLocation(); + } catch (IOException e) { + LOG.warn("Location of bucket {} unknown: {} ", + getUri(), e.toString()); + LOG.debug("getBucketLocation() failed", e); + } initTransferManager(); initCannedAcls(conf); @@ -1284,9 +1295,7 @@ private boolean innerRename(Path source, Path dest) removeKeys(keysToDelete, true, false); } } - if (!keysToDelete.isEmpty()) { - removeKeys(keysToDelete, false, false); - } + removeKeys(keysToDelete, false, false); // We moved all the children, now move the top-level dir // Empty directory should have been added as the object summary @@ -1903,7 +1912,7 @@ public void incrementPutProgressStatistics(String key, long bytes) { } /** - * A helper method to delete a list of keys on a s3-backend. + * Delete a list of keys on a s3-backend. * Retry policy: retry untranslated; delete considered idempotent. * @param keysToDelete collection of keys to delete on the s3-backend. * if empty, no request is made of the object store. @@ -1914,11 +1923,12 @@ public void incrementPutProgressStatistics(String key, long bytes) { * a mistaken attempt to delete the root directory. * @throws MultiObjectDeleteException one or more of the keys could not * be deleted in a multiple object delete operation. - * @throws AmazonClientException amazon-layer failure. + * The number of rejected objects will be added to the metric + * {@link Statistic#FILES_DELETE_REJECTED}. + * @throws AmazonClientException other amazon-layer failure. */ - @VisibleForTesting @Retries.RetryRaw - void removeKeys(List keysToDelete, + private void removeKeysS3(List keysToDelete, boolean clearKeys, boolean deleteFakeDir) throws MultiObjectDeleteException, AmazonClientException, IOException { @@ -1929,22 +1939,78 @@ void removeKeys(List keysToDelete, for (DeleteObjectsRequest.KeyVersion keyVersion : keysToDelete) { blockRootDelete(keyVersion.getKey()); } - if (enableMultiObjectsDelete) { - deleteObjects(new DeleteObjectsRequest(bucket) - .withKeys(keysToDelete) - .withQuiet(true)); - } else { - for (DeleteObjectsRequest.KeyVersion keyVersion : keysToDelete) { - deleteObject(keyVersion.getKey()); + try { + if (enableMultiObjectsDelete) { + deleteObjects(new DeleteObjectsRequest(bucket) + .withKeys(keysToDelete) + .withQuiet(true)); + } else { + for (DeleteObjectsRequest.KeyVersion keyVersion : keysToDelete) { + deleteObject(keyVersion.getKey()); + } } + } catch (MultiObjectDeleteException ex) { + // partial delete. + // Update the stats with the count of the actual number of successful + // deletions. + int rejected = ex.getErrors().size(); + noteDeleted(keysToDelete.size() - rejected, deleteFakeDir); + incrementStatistic(FILES_DELETE_REJECTED, rejected); + throw ex; + } + noteDeleted(keysToDelete.size(), deleteFakeDir); + if (clearKeys) { + keysToDelete.clear(); } + } + + /** + * Note the deletion of files or fake directories deleted. + * @param count count of keys deleted. + * @param deleteFakeDir are the deletions fake directories? + */ + private void noteDeleted(final int count, final boolean deleteFakeDir) { if (!deleteFakeDir) { - instrumentation.fileDeleted(keysToDelete.size()); + instrumentation.fileDeleted(count); } else { - instrumentation.fakeDirsDeleted(keysToDelete.size()); + instrumentation.fakeDirsDeleted(count); } - if (clearKeys) { - keysToDelete.clear(); + } + + /** + * Invoke {@link #removeKeysS3(List, boolean, boolean)} with handling of + * {@code MultiObjectDeleteException} in which S3Guard is updated with all + * deleted entries, before the exception is rethrown. + * + * @param keysToDelete collection of keys to delete on the s3-backend. + * if empty, no request is made of the object store. + * @param clearKeys clears the keysToDelete-list after processing the list + * when set to true + * @param deleteFakeDir indicates whether this is for deleting fake dirs + * @throws InvalidRequestException if the request was rejected due to + * a mistaken attempt to delete the root directory. + * @throws MultiObjectDeleteException one or more of the keys could not + * be deleted in a multiple object delete operation. + * @throws AmazonClientException amazon-layer failure. + * @throws IOException other IO Exception. + */ + @VisibleForTesting + @Retries.RetryMixed + void removeKeys( + final List keysToDelete, + final boolean clearKeys, + final boolean deleteFakeDir) + throws MultiObjectDeleteException, AmazonClientException, + IOException { + try { + removeKeysS3(keysToDelete, clearKeys, deleteFakeDir); + } catch (MultiObjectDeleteException ex) { + LOG.debug("Partial delete failure"); + // what to do if an IOE was raised? Given an exception was being + // raised anyway, and the failures are logged, do nothing. + new MultiObjectDeleteSupport(createStoreContext()) + .processDeleteFailure(ex, keysToDelete); + throw ex; } } @@ -2048,10 +2114,7 @@ private boolean innerDelete(S3AFileStatus status, boolean recursive) if (objects.isTruncated()) { objects = continueListObjects(request, objects); } else { - if (!keys.isEmpty()) { - // TODO: HADOOP-13761 S3Guard: retries - removeKeys(keys, false, false); - } + removeKeys(keys, false, false); break; } } @@ -2272,6 +2335,35 @@ public UserGroupInformation getOwner() { return owner; } + /** + * Build an immutable store context. + * If called while the FS is being initialized, + * some of the context will be incomplete. + * new store context instances should be created as appropriate. + * @return the store context of this FS. + */ + public StoreContext createStoreContext() { + return new StoreContext( + getUri(), + getBucket(), + getConf(), + getUsername(), + owner, + boundedThreadPool, + invoker, + directoryAllocator, + getInstrumentation(), + getStorageStatistics(), + getInputPolicy(), + changeDetectionPolicy, + enableMultiObjectsDelete, + metadataStore, + this::keyToQualifiedPath, + bucketLocation, + useListV1, + false); + } + /** * * Make the given path and all non-existent parents into 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 2bb8f682d8d42..42f0335550820 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 @@ -186,7 +186,8 @@ public class S3AInstrumentation implements Closeable, MetricsSource { S3GUARD_METADATASTORE_RETRY, S3GUARD_METADATASTORE_THROTTLED, STORE_IO_THROTTLED, - DELEGATION_TOKENS_ISSUED + DELEGATION_TOKENS_ISSUED, + FILES_DELETE_REJECTED }; private static final Statistic[] GAUGES_TO_CREATE = { 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 85181c3af8904..6b3505e4361e4 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 @@ -87,6 +87,7 @@ import static org.apache.commons.lang3.StringUtils.isEmpty; import static org.apache.hadoop.fs.s3a.Constants.*; +import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.translateMultiObjectDeleteException; /** * Utility methods for S3A code. @@ -451,40 +452,6 @@ public static IOException translateDynamoDBException(final String path, return result; } - /** - * A MultiObjectDeleteException is raised if one or more delete objects - * listed in a bulk DELETE operation failed. - * The top-level exception is therefore just "something wasn't deleted", - * but doesn't include the what or the why. - * This translation will extract an AccessDeniedException if that's one of - * the causes, otherwise grabs the status code and uses it in the - * returned exception. - * @param message text for the exception - * @param ex exception to translate - * @return an IOE with more detail. - */ - public static IOException translateMultiObjectDeleteException(String message, - MultiObjectDeleteException ex) { - List keys; - StringBuffer result = new StringBuffer(ex.getErrors().size() * 100); - result.append(message).append(": "); - String exitCode = ""; - for (MultiObjectDeleteException.DeleteError error : ex.getErrors()) { - String code = error.getCode(); - result.append(String.format("%s: %s: %s%n", code, error.getKey(), - error.getMessage())); - if (exitCode.isEmpty() || "AccessDenied".equals(code)) { - exitCode = code; - } - } - if ("AccessDenied".equals(exitCode)) { - return (IOException) new AccessDeniedException(result.toString()) - .initCause(ex); - } else { - return new AWSS3IOException(result.toString(), ex); - } - } - /** * Get low level details of an amazon exception for logging; multi-line. * @param e exception 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 54a2c60254167..7a5672830ede2 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 @@ -42,6 +42,8 @@ public enum Statistic { "Total number of files created through the object store."), FILES_DELETED("files_deleted", "Total number of files deleted from the object store."), + FILES_DELETE_REJECTED("files_delete_rejected", + "Total number of files whose delete request was rejected"), FAKE_DIRECTORIES_CREATED("fake_directories_created", "Total number of fake directory entries created in the object store."), FAKE_DIRECTORIES_DELETED("fake_directories_deleted", diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteSupport.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteSupport.java new file mode 100644 index 0000000000000..8d5949af557a7 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteSupport.java @@ -0,0 +1,201 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl; + +import java.io.IOException; +import java.nio.file.AccessDeniedException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.function.Function; +import java.util.stream.Collectors; + +import com.amazonaws.services.s3.model.DeleteObjectsRequest; +import com.amazonaws.services.s3.model.MultiObjectDeleteException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.commons.lang3.tuple.Pair; +import org.apache.commons.lang3.tuple.Triple; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.AWSS3IOException; +import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.fs.s3a.s3guard.MetadataStore; + +import static com.google.common.base.Preconditions.checkNotNull; + +/** + * Support for Multi Object Deletion. + */ +public final class MultiObjectDeleteSupport { + + private static final Logger LOG = LoggerFactory.getLogger( + MultiObjectDeleteSupport.class); + + private final StoreContext context; + + public MultiObjectDeleteSupport(final StoreContext context) { + this.context = context; + } + + /** + * This is the exception exit code if access was denied on a delete. + * {@value}. + */ + public static final String ACCESS_DENIED = "AccessDenied"; + + /** + * A {@code }MultiObjectDeleteException} is raised if one or more + * paths listed in a bulk DELETE operation failed. + * The top-level exception is therefore just "something wasn't deleted", + * but doesn't include the what or the why. + * This translation will extract an AccessDeniedException if that's one of + * the causes, otherwise grabs the status code and uses it in the + * returned exception. + * @param message text for the exception + * @param deleteException the delete exception. to translate + * @return an IOE with more detail. + */ + public static IOException translateMultiObjectDeleteException( + final String message, + final MultiObjectDeleteException deleteException) { + final StringBuilder result = new StringBuilder( + deleteException.getErrors().size() * 256); + result.append(message).append(": "); + String exitCode = ""; + for (MultiObjectDeleteException.DeleteError error : + deleteException.getErrors()) { + String code = error.getCode(); + result.append(String.format("%s: %s: %s%n", code, error.getKey(), + error.getMessage())); + if (exitCode.isEmpty() || ACCESS_DENIED.equals(code)) { + exitCode = code; + } + } + if (ACCESS_DENIED.equals(exitCode)) { + return (IOException) new AccessDeniedException(result.toString()) + .initCause(deleteException); + } else { + return new AWSS3IOException(result.toString(), deleteException); + } + } + + /** + * Build a list of undeleted paths from a {@code MultiObjectDeleteException}. + * Outside of unit tests, the qualifier function should be + * {@link S3AFileSystem#keyToQualifiedPath(String)}. + * @param deleteException the delete exception. + * @param qualifierFn function to qualify paths + * @return the possibly empty list of paths. + */ + public static List extractUndeletedPaths( + final MultiObjectDeleteException deleteException, + final Function qualifierFn) { + return deleteException.getErrors().stream() + .map((e) -> qualifierFn.apply(e.getKey())) + .collect(Collectors.toList()); + } + + /** + * Process a multi object delete exception by removing all undeleted paths + * from the list of paths being deleted. + * The original list is updated, and so becomes the list of successfully + * deleted paths. + * @param deleteException the delete exception. + * @param pathsBeingDeleted list of paths which were being deleted. + * This has all undeleted paths removed, leaving only those deleted. + * @return the list of undeleted entries + */ + public static List splitUndeleted( + final MultiObjectDeleteException deleteException, + final Collection pathsBeingDeleted, + final Function qualifier) { + List undeleted = extractUndeletedPaths(deleteException, qualifier); + pathsBeingDeleted.removeAll(undeleted); + return undeleted; + } + + /** + * Process a multi object delete exception by building two paths from + * the delete request: one of all deleted files, one of all undeleted values. + * The latter are those rejected in the delete call. + * @param deleteException the delete exception. + * @param keysToDelete the keys in the delete request + * @return tuple of (undeleted, deleted) paths. + */ + public Pair, List> splitUndeletedKeys( + final MultiObjectDeleteException deleteException, + final Collection keysToDelete) { + LOG.debug("Processing delete failure; keys to delete count = {};" + + " errors in exception {}; successful deletions = {}", + keysToDelete.size(), + deleteException.getErrors().size(), + deleteException.getDeletedObjects().size()); + Function qualifier = context.getKeyToPathQualifier(); + // convert the collection of keys being deleted into paths + final List pathsBeingDeleted = keysToDelete.stream() + .map((keyVersion) -> qualifier.apply(keyVersion.getKey())) + .collect(Collectors.toList()); + // Take this is list of paths + // extract all undeleted entries contained in the exception and + // then removes them from the original list. + List undeleted = splitUndeleted(deleteException, pathsBeingDeleted, + qualifier); + return Pair.of(undeleted, pathsBeingDeleted); + } + + /** + * Process a delete failure by removing from the metastore all entries + * which where deleted, as inferred from the delete failures exception + * and the original list of files to delete declares to have been delted. + * @param deleteException the delete exception. + * @param keysToDelete collection of keys which had been requested. + * @param qualifierFn qualifier to convert keys to paths + * @return a tuple of (undeleted, deleted, failures) + */ + public Triple, List, List>> + processDeleteFailure( + final MultiObjectDeleteException deleteException, + final List keysToDelete) { + final MetadataStore metadataStore = + checkNotNull(context.getMetadataStore(), "context metadatastore"); + final List> failures = new ArrayList<>(); + final Pair, List> outcome = splitUndeletedKeys( + deleteException, keysToDelete); + List deleted = outcome.getRight(); + List undeleted = outcome.getLeft(); + // delete the paths but recover + deleted.forEach(path -> { + try { + metadataStore.delete(path); + } catch (IOException e) { + // trouble: we failed to delete the far end entry + // try with the next one. + // if this is a big network failure, this is going to be noisy. + LOG.warn("Failed to update S3Guard store with deletion of {}", path); + failures.add(Pair.of(path, e)); + } + }); + if (LOG.isDebugEnabled()) { + undeleted.forEach(p -> LOG.debug("Deleted {}", p)); + } + return Triple.of(undeleted, deleted, failures); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java new file mode 100644 index 0000000000000..cf0834b57ae58 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java @@ -0,0 +1,273 @@ +/* + * 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.net.URI; +import java.util.Optional; +import java.util.function.Function; + +import com.google.common.util.concurrent.ListeningExecutorService; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.LocalDirAllocator; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.Invoker; +import org.apache.hadoop.fs.s3a.S3AEncryptionMethods; +import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.fs.s3a.S3AInputPolicy; +import org.apache.hadoop.fs.s3a.S3AInstrumentation; +import org.apache.hadoop.fs.s3a.S3AStorageStatistics; +import org.apache.hadoop.fs.s3a.Statistic; +import org.apache.hadoop.fs.s3a.s3guard.MetadataStore; +import org.apache.hadoop.security.UserGroupInformation; + +/** + * This class provides the core context of the S3A filesystem to subsidiary + * components, without exposing the entire parent class. + * This is eliminate explicit recursive coupling. + * + * Warning: this really is private and unstable. Do not use + * outside the org.apache.hadoop.fs.s3a package. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public class StoreContext { + + /* + * Foundational fields. + */ + /** Filesystem URI. */ + private final URI fsURI; + + /** Bucket name */ + private final String bucket; + + /* FS configuration after all per-bucket overrides applied. */ + private final Configuration configuration; + + /** Username. */ + private final String username; + + /** Principal who created the FS*/ + private final UserGroupInformation owner; + + /** + * Location of a bucket. + * Optional as the AWS call to evaluate this may fail from a permissions + * or other IOE. + */ + public final Optional bucketLocation; + + /** + * Bounded thread pool for async operations. + */ + private final ListeningExecutorService boundedThreadPool; + + /** Invoker of operations. */ + private final Invoker invoker; + + private final LocalDirAllocator directoryAllocator; + + /* Instrumentation and statistics. */ + private final S3AInstrumentation instrumentation; + private final S3AStorageStatistics storageStatistics; + + /** Seek policy. */ + private final S3AInputPolicy inputPolicy; + + /** How to react to changes in etags and versions. */ + private final ChangeDetectionPolicy changeDetectionPolicy; + + /** Evaluated options. */ + private final boolean enableMultiObjectsDelete; + + /** List algorithm. */ + private final boolean useListV1; + + /** Is the store versioned? */ + private final boolean versioned; + + + /** + * To allow this context to be passed down to the metastore, this field + * wll be null until initialized. + */ + private final MetadataStore metadataStore; + + /** Function to take a key and return a path. */ + private final Function keyToPathQualifier; + + public StoreContext(final URI fsURI, + final String bucket, + final Configuration configuration, + final String username, + final UserGroupInformation owner, + final ListeningExecutorService boundedThreadPool, + final Invoker invoker, + final LocalDirAllocator directoryAllocator, + final S3AInstrumentation instrumentation, + final S3AStorageStatistics storageStatistics, + final S3AInputPolicy inputPolicy, + final ChangeDetectionPolicy changeDetectionPolicy, + final boolean enableMultiObjectsDelete, + final MetadataStore metadataStore, + final Function keyToPathQualifier, + final String bucketLocation, + final boolean useListV1, + final boolean versioned) { + this.fsURI = fsURI; + this.bucket = bucket; + this.configuration = configuration; + this.username = username; + this.owner = owner; + this.boundedThreadPool = boundedThreadPool; + this.invoker = invoker; + this.directoryAllocator = directoryAllocator; + this.instrumentation = instrumentation; + this.storageStatistics = storageStatistics; + this.inputPolicy = inputPolicy; + this.changeDetectionPolicy = changeDetectionPolicy; + this.enableMultiObjectsDelete = enableMultiObjectsDelete; + this.metadataStore = metadataStore; + this.keyToPathQualifier = keyToPathQualifier; + this.bucketLocation = Optional.ofNullable(bucketLocation); + this.useListV1 = useListV1; + this.versioned = versioned; + } + + @Override + protected Object clone() throws CloneNotSupportedException { + return super.clone(); + } + + public URI getFsURI() { + return fsURI; + } + + public String getBucket() { + return bucket; + } + + public Configuration getConfiguration() { + return configuration; + } + + public String getUsername() { + return username; + } + + public Optional getBucketLocation() { + return bucketLocation; + } + + public ListeningExecutorService getBoundedThreadPool() { + return boundedThreadPool; + } + + public Invoker getInvoker() { + return invoker; + } + + public LocalDirAllocator getDirectoryAllocator() { + return directoryAllocator; + } + + public S3AInstrumentation getInstrumentation() { + return instrumentation; + } + + public S3AInputPolicy getInputPolicy() { + return inputPolicy; + } + + public ChangeDetectionPolicy getChangeDetectionPolicy() { + return changeDetectionPolicy; + } + + public boolean isEnableMultiObjectsDelete() { + return enableMultiObjectsDelete; + } + + public MetadataStore getMetadataStore() { + return metadataStore; + } + + public boolean isUseListV1() { + return useListV1; + } + + public boolean isVersioned() { + return versioned; + } + + public Function getKeyToPathQualifier() { + return keyToPathQualifier; + } + + /** + * Get the storage statistics of this filesystem. + * @return the storage statistics + */ + public S3AStorageStatistics getStorageStatistics() { + return storageStatistics; + } + + + /** + * Increment a statistic by 1. + * This increments both the instrumentation and storage statistics. + * @param statistic The operation to increment + */ + public 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 + */ + public 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 + */ + public 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 + */ + public void incrementGauge(Statistic statistic, long count) { + instrumentation.incrementGauge(statistic, count); + } + + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java index 8234777c3b4a2..5728fb10ac2f5 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java @@ -28,11 +28,14 @@ import java.util.Set; import java.util.stream.Collectors; +import com.amazonaws.services.s3.model.DeleteObjectsRequest; +import com.amazonaws.services.s3.model.MultiObjectDeleteException; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java index 8f8d8605653b1..660b62cdc015b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java @@ -20,9 +20,14 @@ import com.amazonaws.services.s3.model.DeleteObjectsRequest; import com.amazonaws.services.s3.model.MultiObjectDeleteException; +import com.google.common.collect.Lists; +import org.junit.Assume; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport; import org.junit.Test; import org.slf4j.Logger; @@ -31,13 +36,16 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.nio.file.AccessDeniedException; import static org.apache.hadoop.fs.contract.ContractTestUtils.*; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.getLandsatCSVPath; +import static org.apache.hadoop.fs.s3a.test.ExtraAssertions.failIf; +import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.*; +import static org.apache.hadoop.fs.s3a.impl.TestPartialDeleteFailures.keysToDelete; import static org.apache.hadoop.test.LambdaTestUtils.*; /** - * Test S3A Failure translation. + * ITest for failure handling, primarily multipart deletion. */ public class ITestS3AFailureHandling extends AbstractS3ATestBase { private static final Logger LOG = @@ -69,12 +77,17 @@ public void testMultiObjectDeleteNoFile() throws Throwable { private void removeKeys(S3AFileSystem fileSystem, String... keys) throws IOException { + fileSystem.removeKeys(buildDeleteRequest(keys), false, false); + } + + private List buildDeleteRequest( + final String[] keys) { List request = new ArrayList<>( keys.length); for (String key : keys) { request.add(new DeleteObjectsRequest.KeyVersion(key)); } - fileSystem.removeKeys(request, false, false); + return request; } @Test @@ -87,12 +100,86 @@ public void testMultiObjectDeleteSomeFiles() throws Throwable { timer.end("removeKeys"); } + + private Path maybeGetCsvPath() { + Configuration conf = getConfiguration(); + String csvFile = conf.getTrimmed(KEY_CSVTEST_FILE, DEFAULT_CSVTEST_FILE); + Assume.assumeTrue("CSV test file is not the default", + DEFAULT_CSVTEST_FILE.equals(csvFile)); + return new Path(csvFile); + } + + /** + * Test low-level failure handling with low level delete request. + */ @Test public void testMultiObjectDeleteNoPermissions() throws Throwable { - Path testFile = getLandsatCSVPath(getConfiguration()); - S3AFileSystem fs = (S3AFileSystem)testFile.getFileSystem( + describe("Delete the landsat CSV file and expect it to fail"); + Path csvPath = maybeGetCsvPath(); + S3AFileSystem fs = (S3AFileSystem)csvPath.getFileSystem(getConfiguration()); + List keys + = buildDeleteRequest( + new String[]{ + fs.pathToKey(csvPath), + "missing-key.csv" + }); + MultiObjectDeleteException ex = intercept( + MultiObjectDeleteException.class, + () -> fs.removeKeys(keys, false, false)); + + final List undeleted + = extractUndeletedPaths(ex, fs::keyToQualifiedPath); + String undeletedFiles = join(undeleted); + failIf(undeleted.size() != 2, + "undeleted list size wrong: " + undeletedFiles, + ex); + assertTrue("no CSV in " +undeletedFiles, undeleted.contains(csvPath)); + + // and a full split, after adding a new key + String marker = "/marker"; + Path markerPath = fs.keyToQualifiedPath(marker); + keys.add(new DeleteObjectsRequest.KeyVersion(marker)); + + Pair, List> pair = + new MultiObjectDeleteSupport(fs.createStoreContext()) + .splitUndeletedKeys(ex, keys); + assertEquals(undeleted, pair.getLeft()); + List right = pair.getRight(); + assertEquals("Wrong size for " + join(right), 1, right.size()); + assertEquals(markerPath, right.get(0)); + } + + /** + * See what happens when you delete two entries which do not exist. + * The call succeeds; if + */ + @Test + public void testMultiObjectDeleteMissingEntriesSucceeds() throws Throwable { + describe("Delete keys which don't exist"); + Path base = path("missing"); + S3AFileSystem fs = getFileSystem(); + List keys = keysToDelete( + Lists.newArrayList(new Path(base, "1"), new Path(base, "2"))); + fs.removeKeys(keys, false, false); + } + + private String join(final Iterable iterable) { + return "[" + StringUtils.join(iterable, ",") + "]"; + } + + /** + * Test low-level failure handling with a single-entry file. + * This is deleted as a single call, so isn't that useful. + */ + @Test + public void testSingleObjectDeleteNoPermissionsTranslated() throws Throwable { + describe("Delete the landsat CSV file and expect it to fail"); + Path csvPath = maybeGetCsvPath(); + S3AFileSystem fs = (S3AFileSystem) csvPath.getFileSystem( getConfiguration()); - intercept(MultiObjectDeleteException.class, - () -> removeKeys(fs, fs.pathToKey(testFile))); + AccessDeniedException aex = intercept(AccessDeniedException.class, + () -> fs.delete(csvPath, false)); + Throwable cause = aex.getCause(); + failIf(cause == null, "no nested exception", aex); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java index 260ecdd71d25e..3d39259f39804 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java @@ -28,6 +28,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.s3a.auth.MarshalledCredentialBinding; import org.apache.hadoop.fs.s3a.auth.MarshalledCredentials; @@ -44,6 +45,7 @@ import org.apache.hadoop.util.ReflectionUtils; import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.services.s3.model.DeleteObjectsRequest; import org.hamcrest.core.Is; import org.junit.Assert; import org.junit.Assume; @@ -60,6 +62,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.Callable; +import java.util.stream.Collectors; import static com.google.common.base.Preconditions.checkNotNull; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CREDENTIAL_PROVIDER_PATH; @@ -1221,4 +1224,5 @@ public static boolean metadataStorePersistsAuthoritativeBit(MetadataStore ms) } return Boolean.valueOf(persists); } + } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java index 1ac52c4e3a239..4a146b117c383 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java @@ -23,9 +23,7 @@ import java.io.IOException; import java.net.URI; import java.nio.file.AccessDeniedException; -import java.util.ArrayList; import java.util.List; -import java.util.stream.Collectors; import java.util.stream.IntStream; import com.amazonaws.auth.AWSCredentials; @@ -53,7 +51,6 @@ import org.apache.hadoop.fs.s3a.commit.files.PendingSet; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; -import static org.apache.hadoop.fs.contract.ContractTestUtils.assertRenameOutcome; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; @@ -524,78 +521,6 @@ public Path methodPath() throws IOException { return path(getMethodName()); } - @Test - public void testRestrictedRename() throws Throwable { - describe("rename with parent paths not writeable"); - executeRestrictedRename(createAssumedRoleConfig()); - } - - @Test - public void testRestrictedSingleDeleteRename() throws Throwable { - describe("rename with parent paths not writeable" - + " and multi-object delete disabled"); - Configuration conf = createAssumedRoleConfig(); - conf.setBoolean(ENABLE_MULTI_DELETE, false); - executeRestrictedRename(conf); - } - - /** - * Execute a sequence of rename operations with access locked down. - * @param conf FS configuration - */ - public void executeRestrictedRename(final Configuration conf) - throws IOException { - Path basePath = methodPath(); - Path restrictedDir = new Path(basePath, "renameSrc"); - Path destPath = new Path(basePath, "renameDest"); - Path child = new Path(restrictedDir, "child"); - // the full FS - S3AFileSystem fs = getFileSystem(); - fs.delete(basePath, true); - - bindRolePolicyStatements(conf, - STATEMENT_S3GUARD_CLIENT, - STATEMENT_ALLOW_SSE_KMS_RW, - STATEMENT_ALL_BUCKET_READ_ACCESS, - new Statement(Effects.Allow) - .addActions(S3_PATH_RW_OPERATIONS) - .addResources(directory(restrictedDir)) - .addResources(directory(destPath)) - ); - roleFS = (S3AFileSystem) restrictedDir.getFileSystem(conf); - - roleFS.getFileStatus(ROOT); - roleFS.mkdirs(restrictedDir); - // you can create an adjacent child - touch(roleFS, child); - - roleFS.delete(destPath, true); - // as dest doesn't exist, this will map child -> dest - assertRenameOutcome(roleFS, child, destPath, true); - - assertIsFile(destPath); - assertIsDirectory(restrictedDir); - Path renamedDestPath = new Path(restrictedDir, destPath.getName()); - assertRenameOutcome(roleFS, destPath, restrictedDir, true); - assertIsFile(renamedDestPath); - roleFS.delete(restrictedDir, true); - roleFS.delete(destPath, true); - } - - @Test - public void testRestrictedRenameReadOnlyData() throws Throwable { - describe("rename with source read only, multidelete"); - executeRenameReadOnlyData(createAssumedRoleConfig()); - } - - @Test - public void testRestrictedRenameReadOnlySingleDelete() throws Throwable { - describe("rename with source read only single delete"); - Configuration conf = createAssumedRoleConfig(); - conf.setBoolean(ENABLE_MULTI_DELETE, false); - executeRenameReadOnlyData(conf); - } - /** * Without simulation of STS failures, and with STS overload likely to * be very rare, there'll be no implicit test coverage of @@ -615,102 +540,6 @@ public void testAssumedRoleRetryHandler() throws Throwable { } } - /** - * Execute a sequence of rename operations where the source - * data is read only to the client calling rename(). - * This will cause the inner delete() operations to fail, whose outcomes - * are explored. - * Multiple files are created (in parallel) for some renames, so exploring - * the outcome on bulk delete calls, including verifying that a - * MultiObjectDeleteException is translated to an AccessDeniedException. - *
    - *
  1. The exception raised is AccessDeniedException, - * from single and multi DELETE calls.
  2. - *
  3. It happens after the COPY. Not ideal, but, well, we can't pretend - * it's a filesystem forever.
  4. - *
- * @param conf FS configuration - */ - public void executeRenameReadOnlyData(final Configuration conf) - throws Exception { - assume("Does not work with S3Guard", !getFileSystem().hasMetadataStore()); - Path basePath = methodPath(); - Path destDir = new Path(basePath, "renameDest"); - Path readOnlyDir = new Path(basePath, "readonlyDir"); - Path readOnlyFile = new Path(readOnlyDir, "readonlyChild"); - - // the full FS - S3AFileSystem fs = getFileSystem(); - fs.delete(basePath, true); - - // this file is readable by the roleFS, but cannot be deleted - touch(fs, readOnlyFile); - - bindRolePolicyStatements(conf, - STATEMENT_S3GUARD_CLIENT, - STATEMENT_ALL_BUCKET_READ_ACCESS, - new Statement(Effects.Allow) - .addActions(S3_PATH_RW_OPERATIONS) - .addResources(directory(destDir)) - ); - roleFS = (S3AFileSystem) destDir.getFileSystem(conf); - - roleFS.delete(destDir, true); - roleFS.mkdirs(destDir); - // rename will fail in the delete phase - forbidden(readOnlyFile.toString(), - () -> roleFS.rename(readOnlyFile, destDir)); - - // and the source file is still there - assertIsFile(readOnlyFile); - - // but so is the copied version, because there's no attempt - // at rollback, or preflight checking on the delete permissions - Path renamedFile = new Path(destDir, readOnlyFile.getName()); - - assertIsFile(renamedFile); - - ContractTestUtils.assertDeleted(roleFS, renamedFile, true); - assertFileCount("Empty Dest Dir", roleFS, - destDir, 0); - // create a set of files - // this is done in parallel as it is 10x faster on a long-haul test run. - int range = 10; - touchFiles(fs, readOnlyDir, range); - // don't forget about that original file! - final long createdFiles = range + 1; - // are they all there? - assertFileCount("files ready to rename", roleFS, - readOnlyDir, createdFiles); - - // try to rename the directory - LOG.info("Renaming readonly files {} to {}", readOnlyDir, destDir); - AccessDeniedException ex = forbidden("", - () -> roleFS.rename(readOnlyDir, destDir)); - LOG.info("Result of renaming read-only files is AccessDeniedException", ex); - assertFileCount("files copied to the destination", roleFS, - destDir, createdFiles); - assertFileCount("files in the source directory", roleFS, - readOnlyDir, createdFiles); - - // and finally (so as to avoid the delay of POSTing some more objects, - // delete that r/o source - forbidden("", () -> roleFS.delete(readOnlyDir, true)); - } - - /** - * Parallel-touch a set of files in the destination directory. - * @param fs filesystem - * @param destDir destination - * @param range range 1..range inclusive of files to create. - */ - public void touchFiles(final S3AFileSystem fs, - final Path destDir, - final int range) { - IntStream.rangeClosed(1, range).parallel().forEach( - (i) -> eval(() -> touch(fs, new Path(destDir, "file-" + i)))); - } - @Test public void testRestrictedCommitActions() throws Throwable { describe("Attempt commit operations against a path with restricted rights"); @@ -840,24 +669,25 @@ public void writeCSVData(final File localSrc) throws IOException { @Test public void testPartialDelete() throws Throwable { describe("delete with part of the child tree read only; multidelete"); - executePartialDelete(createAssumedRoleConfig()); + executePartialDelete(createAssumedRoleConfig(), false); } @Test public void testPartialDeleteSingleDelete() throws Throwable { describe("delete with part of the child tree read only"); - Configuration conf = createAssumedRoleConfig(); - conf.setBoolean(ENABLE_MULTI_DELETE, false); - executePartialDelete(conf); + executePartialDelete(createAssumedRoleConfig(), true); } /** * Have a directory with full R/W permissions, but then remove * write access underneath, and try to delete it. * @param conf FS configuration + * @param singleDelete flag to indicate this is a single delete operation */ - public void executePartialDelete(final Configuration conf) + public void executePartialDelete(final Configuration conf, + final boolean singleDelete) throws Exception { + conf.setBoolean(ENABLE_MULTI_DELETE, !singleDelete); Path destDir = methodPath(); Path readOnlyDir = new Path(destDir, "readonlyDir"); @@ -888,25 +718,4 @@ public void executePartialDelete(final Configuration conf) roleFS.delete(pathWhichDoesntExist, true)); } - /** - * Assert that the number of files in a destination matches that expected. - * @param text text to use in the message - * @param fs filesystem - * @param path path to list (recursively) - * @param expected expected count - * @throws IOException IO problem - */ - private static void assertFileCount(String text, FileSystem fs, - Path path, long expected) - throws IOException { - List files = new ArrayList<>(); - applyLocatedFiles(fs.listFiles(path, true), - (status) -> files.add(status.getPath().toString())); - long actual = files.size(); - if (actual != expected) { - String ls = files.stream().collect(Collectors.joining("\n")); - fail(text + ": expected " + expected + " files in " + path - + " but got " + actual + "\n" + ls); - } - } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/RoleTestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/RoleTestUtils.java index dbbaee5f8a9d1..71acb57a75dba 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/RoleTestUtils.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/RoleTestUtils.java @@ -18,8 +18,14 @@ package org.apache.hadoop.fs.s3a.auth; +import java.io.IOException; import java.nio.file.AccessDeniedException; +import java.util.List; import java.util.concurrent.Callable; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; import com.fasterxml.jackson.core.JsonProcessingException; import org.junit.Assume; @@ -38,6 +44,7 @@ import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching; import static org.apache.hadoop.fs.s3a.auth.RoleModel.*; import static org.apache.hadoop.fs.s3a.auth.RolePolicies.*; +import static org.apache.hadoop.test.LambdaTestUtils.eval; import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -160,18 +167,35 @@ public static Configuration newAssumedRoleConfig( /** * Assert that an operation is forbidden. + * @param type of closure * @param contained contained text, may be null * @param eval closure to evaluate + * @return the access denied exception + * @throws Exception any other exception + */ + public static AccessDeniedException forbidden( + final String contained, + final Callable eval) + throws Exception { + return forbidden("", contained, eval); + } + + /** + * Assert that an operation is forbidden. * @param type of closure + * @param message error message + * @param contained contained text, may be null + * @param eval closure to evaluate * @return the access denied exception * @throws Exception any other exception */ public static AccessDeniedException forbidden( - String contained, - Callable eval) + final String message, + final String contained, + final Callable eval) throws Exception { return intercept(AccessDeniedException.class, - contained, eval); + contained, message, eval); } /** @@ -209,4 +233,23 @@ public static void assertCredentialsEqual(final String message, actual.getSessionToken()); } + + /** + * Parallel-touch a set of files in the destination directory. + * @param fs filesystem + * @param destDir destination + * @param range range 1..range inclusive of files to create. + * @return the list of paths created. + */ + public static List touchFiles(final FileSystem fs, + final Path destDir, + final int range) throws IOException { + List paths = IntStream.rangeClosed(1, range) + .mapToObj((i) -> new Path(destDir, "file-" + i)) + .collect(Collectors.toList()); + for (Path path : paths) { + touch(fs, path); + } + return paths; + } } 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 new file mode 100644 index 0000000000000..83bf08cc2a78b --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestPartialRenamesDeletes.java @@ -0,0 +1,446 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl; + +import java.io.IOException; +import java.nio.file.AccessDeniedException; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Set; +import java.util.TreeSet; +import java.util.stream.Collectors; + +import com.amazonaws.services.s3.model.MultiObjectDeleteException; +import org.assertj.core.api.Assertions; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.s3a.AbstractS3ATestBase; +import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.fs.s3a.S3AUtils; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.assertRenameOutcome; +import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; +import static org.apache.hadoop.fs.s3a.Constants.ASSUMED_ROLE_ARN; +import static org.apache.hadoop.fs.s3a.Constants.ENABLE_MULTI_DELETE; +import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.touchFiles; +import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.extractUndeletedPaths; +import static org.apache.hadoop.fs.s3a.test.ExtraAssertions.assertFileCount; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.MetricDiff; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.reset; +import static org.apache.hadoop.fs.s3a.S3AUtils.applyLocatedFiles; +import static org.apache.hadoop.fs.s3a.Statistic.FILES_DELETE_REJECTED; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_DELETE_REQUESTS; +import static org.apache.hadoop.fs.s3a.auth.RoleModel.Effects; +import static org.apache.hadoop.fs.s3a.auth.RoleModel.Statement; +import static org.apache.hadoop.fs.s3a.auth.RoleModel.directory; +import static org.apache.hadoop.fs.s3a.auth.RoleModel.statement; +import static org.apache.hadoop.fs.s3a.auth.RolePolicies.*; +import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.bindRolePolicyStatements; +import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.forbidden; +import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.newAssumedRoleConfig; +import static org.apache.hadoop.fs.s3a.test.ExtraAssertions.extractCause; +import static org.apache.hadoop.test.LambdaTestUtils.eval; + +/** + * Test partial failures of delete and rename operations, especially + * that the S3Guard tables are consistent with the state of + * the filesystem. + * + * All these test have a unique path for each run, with a roleFS having + * full RW access to part of it, and R/O access to a restricted subdirectory + */ +@SuppressWarnings({"IOResourceOpenedButNotSafelyClosed", "ThrowableNotThrown"}) +@RunWith(Parameterized.class) +public class ITestPartialRenamesDeletes extends AbstractS3ATestBase { + + private static final Logger LOG = + LoggerFactory.getLogger(ITestPartialRenamesDeletes.class); + + private static final Path ROOT = new Path("/"); + + private static final Statement STATEMENT_ALL_BUCKET_READ_ACCESS + = statement(true, S3_ALL_BUCKETS, S3_BUCKET_READ_OPERATIONS); + + /** + * A role FS; if non-null it is closed in teardown. + */ + private S3AFileSystem roleFS; + + private Path basePath; + + private Path destDir; + + private Path readonlyChild; + + private Path readOnlyDir; + + /** delete policy: single or multi? */ + private final boolean multiDelete; + + private Configuration assumedRoleConfig; + + /** + * Test array for parameterized test runs. + * @return a list of parameter tuples. + */ + @Parameterized.Parameters + public static Collection params() { + return Arrays.asList(new Object[][]{ + {true}, + {false}, + }); + } + + public ITestPartialRenamesDeletes(final boolean multiDelete) { + this.multiDelete = multiDelete; + } + + @Override + public void setup() throws Exception { + super.setup(); + assumeRoleTests(); + basePath = uniquePath(); + readOnlyDir = new Path(basePath, "readonlyDir"); + destDir = new Path(basePath, "renameDest"); + readonlyChild = new Path(readOnlyDir, "child"); + // the full FS + S3AFileSystem fs = getFileSystem(); + fs.delete(basePath, true); + fs.mkdirs(destDir); + assumedRoleConfig = createAssumedRoleConfig(); + bindRolePolicyStatements(assumedRoleConfig, + STATEMENT_S3GUARD_CLIENT, + STATEMENT_ALL_BUCKET_READ_ACCESS, + new Statement(Effects.Allow) + .addActions(S3_PATH_RW_OPERATIONS) + .addResources(directory(destDir)) + ); + roleFS = (S3AFileSystem) readOnlyDir.getFileSystem(assumedRoleConfig); + } + + @Override + public void teardown() throws Exception { + S3AUtils.closeAll(LOG, roleFS); + super.teardown(); + } + + private void assumeRoleTests() { + assume("No ARN for role tests", !getAssumedRoleARN().isEmpty()); + } + + private String getAssumedRoleARN() { + return getContract().getConf().getTrimmed(ASSUMED_ROLE_ARN, ""); + } + + /** + * Create the assumed role configuration. + * @return a config bonded to the ARN of the assumed role + */ + public Configuration createAssumedRoleConfig() { + return createAssumedRoleConfig(getAssumedRoleARN()); + } + + /** + * Create a config for an assumed role; it also disables FS caching + * and sets the multi delete option to that of the current mode. + * @param roleARN ARN of role + * @return the new configuration + */ + private Configuration createAssumedRoleConfig(String roleARN) { + Configuration conf = newAssumedRoleConfig(getContract().getConf(), + roleARN); + conf.setBoolean(ENABLE_MULTI_DELETE, multiDelete); + return conf; + } + + /** + * Create a unique path, which includes method name, + * multdelete flag and a random UUID. + * @return a string to use for paths. + * @throws IOException path creation failure. + */ + private Path uniquePath() throws IOException { + return path( + String.format("%s-%s-%04d", + getMethodName(), + multiDelete ? "multi" : "single", + System.currentTimeMillis() % 10000)); + } + + /** + * This is here to verify role and path setup. + */ + @Test + public void testCannotTouchUnderRODir() throws Throwable { + forbidden("touching the empty child " + readonlyChild, + "", + () -> ContractTestUtils.touch(roleFS, readonlyChild)); + } + + /** + * Execute a sequence of rename operations with access locked down. + */ + @Test + public void testRenameParentPathNotWriteable() throws Throwable { + describe("rename with parent paths not writeable; multi=%s", multiDelete); + final Configuration conf = createAssumedRoleConfig(); + bindRolePolicyStatements(conf, + STATEMENT_S3GUARD_CLIENT, + STATEMENT_ALLOW_SSE_KMS_RW, + STATEMENT_ALL_BUCKET_READ_ACCESS, + new Statement(Effects.Allow) + .addActions(S3_PATH_RW_OPERATIONS) + .addResources(directory(readOnlyDir)) + .addResources(directory(destDir))); + roleFS = (S3AFileSystem) readOnlyDir.getFileSystem(conf); + + S3AFileSystem fs = getFileSystem(); + roleFS.getFileStatus(ROOT); + fs.mkdirs(readOnlyDir); + // you can create an adjacent child + touch(fs, readonlyChild); + + fs.delete(destDir, true); + // as dest doesn't exist, this will map child -> dest + assertRenameOutcome(roleFS, readonlyChild, destDir, true); + + assertIsFile(destDir); + assertIsDirectory(readOnlyDir); + Path renamedDestPath = new Path(readOnlyDir, destDir.getName()); + assertRenameOutcome(roleFS, destDir, readOnlyDir, true); + assertIsFile(renamedDestPath); + roleFS.delete(readOnlyDir, true); + roleFS.delete(destDir, true); + } + + @Test + public void testRenameSingleFileFailsLeavingSource() throws Throwable { + describe("rename with source read only; multi=%s", multiDelete); + Path readOnlyFile = readonlyChild; + + // the full FS + S3AFileSystem fs = getFileSystem(); + fs.delete(basePath, true); + + // this file is readable by the roleFS, but cannot be deleted + touch(fs, readOnlyFile); + + roleFS.delete(destDir, true); + roleFS.mkdirs(destDir); + // rename will fail in the delete phase + AccessDeniedException deniedException = expectRenameForbidden( + readOnlyFile, destDir); + if (multiDelete) { + // look in that exception for a multidelete + Throwable cause = deniedException.getCause(); + if (!(cause instanceof MultiObjectDeleteException)) { + throw new AssertionError("Expected a MultiObjectDeleteException " + + "as the cause ", deniedException); + } + MultiObjectDeleteException mde = (MultiObjectDeleteException) cause; + final List undeleted + = extractUndeletedPaths(mde, fs::keyToQualifiedPath); + Assertions.assertThat(undeleted) + .as("files which could not be deleted") + .containsExactly(readOnlyFile); + } + + // and the source file is still there + assertIsFile(readOnlyFile); + + // but so is the copied version, because there's no attempt + // at rollback, or preflight checking on the delete permissions + Path renamedFile = new Path(destDir, readOnlyFile.getName()); + + assertIsFile(renamedFile); + + ContractTestUtils.assertDeleted(roleFS, renamedFile, true); + assertFileCount("Empty Dest Dir", roleFS, + destDir, 0); + } + + /** + * Execute a sequence of rename operations where the source + * data is read only to the client calling rename(). + * This will cause the inner delete() operations to fail, whose outcomes + * are explored. + * Multiple files are created (in parallel) for some renames, so exploring + * the outcome on bulk delete calls, including verifying that a + * MultiObjectDeleteException is translated to an AccessDeniedException. + *
    + *
  1. The exception raised is AccessDeniedException, + * from single and multi DELETE calls.
  2. + *
  3. It happens after the COPY. Not ideal, but, well, we can't pretend + * it's a filesystem forever.
  4. + *
+ */ + @Test + public void testRenameFilesetFailsLeavingSourceUnchanged() throws Throwable { + describe("rename with source read only; multi=%s", multiDelete); + + // the full FS + S3AFileSystem fs = getFileSystem(); + fs.delete(basePath, true); + + roleFS.mkdirs(destDir); + + // create a set of files + // this is done in parallel as it is 10x faster on a long-haul test run. + int filecount = 10; + touchFiles(fs, readOnlyDir, filecount); + // are they all there? + assertFileCount("files ready to rename", roleFS, + readOnlyDir, (long) filecount); + + // try to rename the directory + LOG.info("Renaming readonly files {} to {}", readOnlyDir, destDir); + + AccessDeniedException ex = expectRenameForbidden(readOnlyDir, destDir); + LOG.info("Result of renaming read-only files is AccessDeniedException", ex); + assertFileCount("files in the source directory", roleFS, + readOnlyDir, (long) filecount); + + } + + /** + * Have a directory with full R/W permissions, but then remove + * write access underneath, and try to delete it. + * + */ + @Test + public void testPartialDelete() throws Throwable { + describe("delete with part of the child tree read only; multidelete"); + + // the full FS + S3AFileSystem fs = getFileSystem(); + + int range = 10; + List readOnlyFiles = touchFiles(fs, readOnlyDir, range); + + // this set can be deleted by the role FS + MetricDiff rejectionCount = new MetricDiff(roleFS, FILES_DELETE_REJECTED); + MetricDiff deleteVerbCount = new MetricDiff(roleFS, OBJECT_DELETE_REQUESTS); + + describe("Trying to delete read only directory with files %s", + ls(readOnlyDir)); + AccessDeniedException ex = expectDeleteForbidden(readOnlyDir); + if (multiDelete) { + // multi-delete status checks + extractCause(MultiObjectDeleteException.class, ex); + rejectionCount.assertDiffEquals("Wrong rejection count", range); + deleteVerbCount.assertDiffEquals("Wrong delete count", 1); + reset(rejectionCount, deleteVerbCount); + } + // all the files are still there + readOnlyFiles.forEach(this::pathMustExist); + + describe("Trying to delete upper-level directory"); + ex = expectDeleteForbidden(basePath); + if (multiDelete) { + // multi-delete status checks + extractCause(MultiObjectDeleteException.class, ex); + rejectionCount.assertDiffEquals("Wrong rejection count", range); + deleteVerbCount.assertDiffEquals("Wrong delete count", 1); + reset(rejectionCount, deleteVerbCount); + } + + // build the set of all paths under the directory tree through + // a directory listing (i.e. not getFileStatus()). + // small risk of observed inconsistency here on unguarded stores. + final Set roFListing = filesUnderPath(readOnlyDir, true); + + String directoryList = roFListing + .stream() + .map(Path::toString) + .collect(Collectors.joining(", ", "[", "]")); + + Assertions.assertThat(roFListing) + .as("ReadOnly directory " + directoryList) + .containsAll(readOnlyFiles); + } + + private AccessDeniedException expectDeleteForbidden(Path path) throws Exception { + return forbidden("Expected an error deleting " + path, + "", + () -> { + boolean r = roleFS.delete(path, true); + return " delete=" + r + " " + ls(path.getParent()); + } + ); + } + + /** + * Expect that a rename will fail with an exception. + * @param src source + * @param dest dest + * @return the exception raised. + * @throws Exception any other failure + */ + private AccessDeniedException expectRenameForbidden(Path src, Path dest) + throws Exception { + return forbidden( + "Renaming " + src + " to " + dest, + "", + () -> { + roleFS.rename(src, dest); + return ContractTestUtils.ls(getFileSystem(), src.getParent()); + }); + } + + /** + * Assert that a path must exist, map IOEs to RTEs for loops. + * @param p path. + */ + private void pathMustExist(Path p) { + eval(() -> assertPathExists("Missing path", p)); + } + + /** + * Assert that a path must exist, map IOEs to RTEs for loops. + * @param p path. + */ + private void pathMustNotExist(Path p) { + eval(() -> assertPathDoesNotExist("Path should not exist", p)); + } + + /** + * List all files under a path. + * @param path path to list + * @param recursive recursive listing? + * @return an unordered set of the paths. + * @throws IOException failure + */ + private Set filesUnderPath(Path path, boolean recursive) throws IOException { + Set files = new TreeSet<>(); + applyLocatedFiles(getFileSystem().listFiles(path, recursive), + (status) -> { + files.add(status.getPath()); + }); + return files; + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java new file mode 100644 index 0000000000000..27db030f139d2 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java @@ -0,0 +1,326 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl; + +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +import com.amazonaws.services.s3.model.DeleteObjectsRequest; +import com.amazonaws.services.s3.model.MultiObjectDeleteException; +import com.google.common.collect.Lists; +import org.assertj.core.api.Assertions; +import org.junit.Before; +import org.junit.Test; + +import org.apache.commons.lang3.tuple.Pair; +import org.apache.commons.lang3.tuple.Triple; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.Invoker; +import org.apache.hadoop.fs.s3a.S3AInputPolicy; +import org.apache.hadoop.fs.s3a.S3AInstrumentation; +import org.apache.hadoop.fs.s3a.S3AStorageStatistics; +import org.apache.hadoop.fs.s3a.s3guard.DirListingMetadata; +import org.apache.hadoop.fs.s3a.s3guard.MetadataStore; +import org.apache.hadoop.fs.s3a.s3guard.PathMetadata; +import org.apache.hadoop.io.retry.RetryPolicies; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.util.BlockingThreadPoolExecutorService; + +import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.ACCESS_DENIED; +import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.splitUndeleted; +import static org.junit.Assert.assertEquals; + +/** + * Unit test suite covering translation of AWS SDK exceptions to S3A exceptions, + * and retry/recovery policies. + */ +public class TestPartialDeleteFailures { + + private StoreContext context; + + private static Path qualify(String k) { + return new Path("s3a://bucket/" + k); + } + + @Before + public void setUp() throws Exception { + context = craateMockStoreContext(true, + new OperationTrackingStore()); + } + + @Test + public void testDeleteExtraction() { + List src = pathList("a", "a/b", "a/c"); + List rejected = pathList("a/b"); + MultiObjectDeleteException ex = createDeleteException(ACCESS_DENIED, + rejected); + List undeleted = splitUndeleted(ex, src, + TestPartialDeleteFailures::qualify); + assertEquals(rejected, undeleted); + } + + @Test + public void testSplitKeysFromResults() throws Throwable { + List src = pathList("a", "a/b", "a/c"); + List rejected = pathList("a/b"); + List keys = keysToDelete(src); + MultiObjectDeleteException ex = createDeleteException(ACCESS_DENIED, + rejected); + Pair, List> pair = + new MultiObjectDeleteSupport(context) + .splitUndeletedKeys(ex, keys); + List undeleted = pair.getLeft(); + List deleted = pair.getRight(); + assertEquals(rejected, undeleted); + // now check the deleted list to verify that it is valid + src.remove(rejected.get(0)); + assertEquals(src, deleted); + } + + /** + * Build a list of paths from a vararg stream list.DistributedFileSystem + * @param paths paths in + * @return same paths as a list. + */ + private List pathList(String... paths) { + return Arrays.stream(paths) + .map(TestPartialDeleteFailures::qualify) + .collect(Collectors.toList()); + } + + /** + * Build a delete exception containing all the rejected paths. + * The list of successful entries is empty. + * @param rejected the rejected paths. + * @return a new exception + */ + private MultiObjectDeleteException createDeleteException( + final String code, + final List rejected) { + List errors = rejected.stream() + .map((p) -> { + MultiObjectDeleteException.DeleteError e + = new MultiObjectDeleteException.DeleteError(); + e.setKey(p.toUri().getPath()); + e.setCode(code); + e.setMessage("forbidden"); + return e; + }).collect(Collectors.toList()); + return new MultiObjectDeleteException(errors, Collections.emptyList()); + } + + /** + * From a list of paths, build up the list of keys for a delete request. + * @param paths path list + * @return a key list suitable for a delete request. + */ + public static List keysToDelete( + List paths) { + return paths.stream() + .map((p) -> p.toUri().getPath()) + .map(DeleteObjectsRequest.KeyVersion::new) + .collect(Collectors.toList()); + } + + /** + * Verify that on a partial delete, the S3Guard tables are updated + * with deleted items. And only them. + */ + @Test + public void testProcessDeleteFailure() throws Throwable { + Path pathA = qualify("/a"); + Path pathAB = qualify("/a/b"); + Path pathAC = qualify("/a/c"); + List src = Lists.newArrayList(pathA, pathAB, pathAC); + List keyList = keysToDelete(src); + List deleteForbidden = Lists.newArrayList(pathAB); + final List deleteAllowed = Lists.newArrayList(pathA, pathAC); + MultiObjectDeleteException ex = createDeleteException(ACCESS_DENIED, + deleteForbidden); + OperationTrackingStore store + = new OperationTrackingStore(); + StoreContext storeContext = craateMockStoreContext(true, store); + MultiObjectDeleteSupport deleteSupport + = new MultiObjectDeleteSupport(storeContext); + Triple, List, List>> + triple = deleteSupport.processDeleteFailure(ex, keyList); + Assertions.assertThat(triple.getRight()) + .as("failure list") + .isEmpty(); + List undeleted = triple.getLeft(); + List deleted = triple.getMiddle(); + Assertions.assertThat(deleted). + as("deleted files") + .containsAll(deleteAllowed) + .doesNotContainAnyElementsOf(deleteForbidden); + Assertions.assertThat(undeleted). + as("undeleted store entries") + .containsAll(deleteForbidden) + .doesNotContainAnyElementsOf(deleteAllowed); + } + + + StoreContext craateMockStoreContext(boolean multiDelete, + OperationTrackingStore store) throws URISyntaxException, IOException { + URI name = new URI("s3a://bucket"); + return new StoreContext( + name, + "bucket", + new Configuration(), + "alice", + UserGroupInformation.getCurrentUser(), + BlockingThreadPoolExecutorService.newInstance( + 4, + 4, + 10, TimeUnit.SECONDS, + "s3a-transfer-shared"), + new Invoker(RetryPolicies.TRY_ONCE_THEN_FAIL, Invoker.LOG_EVENT), + null, + new S3AInstrumentation(name), + new S3AStorageStatistics(), + S3AInputPolicy.Normal, + ChangeDetectionPolicy.createPolicy(ChangeDetectionPolicy.Mode.None, + ChangeDetectionPolicy.Source.ETag, false), + multiDelete, + store, + TestPartialDeleteFailures::qualify, + "us-west", + false, + false); + } + + /** + * MetadataStore which tracks what is deleted and added. + */ + private static class OperationTrackingStore implements MetadataStore { + + private final List deleted = new ArrayList<>(); + + private final List created = new ArrayList<>(); + + @Override + public void initialize(final FileSystem fs) { + } + + @Override + public void initialize(final Configuration conf) { + } + + @Override + public void delete(final Path path) { + deleted.add(path); + } + + @Override + public void forgetMetadata(final Path path) { + + } + + @Override + public void deleteSubtree(final Path path) { + + } + + @Override + public PathMetadata get(final Path path) { + return null; + } + + @Override + public PathMetadata get(final Path path, + final boolean wantEmptyDirectoryFlag) { + return null; + } + + @Override + public DirListingMetadata listChildren(final Path path) { + return null; + } + + @Override + public void move(final Collection pathsToDelete, + final Collection pathsToCreate) { + + } + + @Override + public void put(final PathMetadata meta) { + created.add(meta.getFileStatus().getPath()); + } + + @Override + public void put(final Collection metas) { + metas.stream().forEach(this::put); + } + + @Override + public void put(final DirListingMetadata meta) { + created.add(meta.getPath()); + } + + @Override + public void destroy() { + + } + + @Override + public void prune(final long modTime) + throws IOException, UnsupportedOperationException { + + } + + @Override + public void prune(final long modTime, final String keyPrefix) + throws IOException, UnsupportedOperationException { + } + + @Override + public Map getDiagnostics() { + return null; + } + + @Override + public void updateParameters(final Map parameters) { + } + + @Override + public void close() { + } + + public List getDeleted() { + return deleted; + } + + public List getCreated() { + return created; + } + } + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/ExtraAssertions.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/ExtraAssertions.java new file mode 100644 index 0000000000000..f1a4c5554594f --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/ExtraAssertions.java @@ -0,0 +1,124 @@ +/* + * 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.test; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; + +import org.junit.Assert; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; + +import static org.apache.hadoop.fs.s3a.S3AUtils.applyLocatedFiles; +import static org.junit.Assert.assertTrue; + +/** + * Some extra assertions for tests. + */ +@InterfaceAudience.Private +public class ExtraAssertions { + + /** + * Assert that the number of files in a destination matches that expected. + * @param text text to use in the message + * @param fs filesystem + * @param path path to list (recursively) + * @param expected expected count + * @throws IOException IO problem + */ + public static void assertFileCount(String text, FileSystem fs, + Path path, long expected) + throws IOException { + List files = new ArrayList<>(); + applyLocatedFiles(fs.listFiles(path, true), + (status) -> files.add(status.getPath().toString())); + long actual = files.size(); + if (actual != expected) { + String ls = files.stream().collect(Collectors.joining("\n")); + Assert.fail(text + ": expected " + expected + " files in " + path + + " but got " + actual + "\n" + ls); + } + } + + /** + * Assert that a string contains a piece of text. + * @param text text to can. + * @param contained text to look for. + */ + public static void assertTextContains(String text, String contained) { + assertTrue("string \"" + contained + "\" not found in \"" + text + "\"", + text != null && text.contains(contained)); + } + + /** + * If the condition is met, throw an AssertionError with the message + * and any nested exception. + * @param condition condition + * @param message text to use in the exception + * @param cause a (possibly null) throwable to init the cause with + * @throws AssertionError with the text and throwable if condition == true. + */ + public static void failIf(boolean condition, + String message, + Throwable cause) { + if (condition) { + ContractTestUtils.fail(message, cause); + } + } + + /** + * If the condition is met, throw an AssertionError with the message + * and any nested exception. + * @param condition condition + * @param message text to use in the exception + * @param cause a (possibly null) throwable to init the cause with + * @throws AssertionError with the text and throwable if condition == true. + */ + public static void failUnless(boolean condition, + String message, + Throwable cause) { + failIf(!condition, message, cause); + } + + /** + * Extract the inner cause of an exception. + * @param expected expected class of the cuse + * @param thrown thrown exception. + * @param type of the cause + * @return the extracted exception. + * @throws AssertionError with the text and throwable if the cause is not as + * expected + */ + public static T extractCause(Class expected, + Throwable thrown) { + Throwable cause = thrown.getCause(); + failIf(cause == null, + "No inner cause", + thrown); + failUnless(cause.getClass().equals(expected), + "Inner cause is of wrong type : expected " + expected, + thrown); + return (T)cause; + } +} From 0b1902c9acfe2f3630eee8c6b9fd52890789e663 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 24 Apr 2019 16:27:52 +0100 Subject: [PATCH 02/22] HADOOP-15183 getting ITestPartialRenamesDeletes tests to work by putting check for multidelete into the correct test case Change-Id: Ie8ca305b79a1e3bfb963cf10c917ce59c2b0c39f --- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 1 + .../hadoop/fs/s3a/impl/StoreContext.java | 12 ++-- .../s3a/impl/ITestPartialRenamesDeletes.java | 59 +++++++++++++------ 3 files changed, 47 insertions(+), 25 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 af524fd71aa7f..2121b0be3da24 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 @@ -2342,6 +2342,7 @@ public UserGroupInformation getOwner() { * new store context instances should be created as appropriate. * @return the store context of this FS. */ + @InterfaceAudience.Private public StoreContext createStoreContext() { return new StoreContext( getUri(), diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java index cf0834b57ae58..ffd2a8a95fe7f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java @@ -30,8 +30,6 @@ import org.apache.hadoop.fs.LocalDirAllocator; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.Invoker; -import org.apache.hadoop.fs.s3a.S3AEncryptionMethods; -import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3AInputPolicy; import org.apache.hadoop.fs.s3a.S3AInstrumentation; import org.apache.hadoop.fs.s3a.S3AStorageStatistics; @@ -97,7 +95,7 @@ public class StoreContext { private final ChangeDetectionPolicy changeDetectionPolicy; /** Evaluated options. */ - private final boolean enableMultiObjectsDelete; + private final boolean multiObjectDeleteEnabled; /** List algorithm. */ private final boolean useListV1; @@ -127,7 +125,7 @@ public StoreContext(final URI fsURI, final S3AStorageStatistics storageStatistics, final S3AInputPolicy inputPolicy, final ChangeDetectionPolicy changeDetectionPolicy, - final boolean enableMultiObjectsDelete, + final boolean multiObjectDeleteEnabled, final MetadataStore metadataStore, final Function keyToPathQualifier, final String bucketLocation, @@ -145,7 +143,7 @@ public StoreContext(final URI fsURI, this.storageStatistics = storageStatistics; this.inputPolicy = inputPolicy; this.changeDetectionPolicy = changeDetectionPolicy; - this.enableMultiObjectsDelete = enableMultiObjectsDelete; + this.multiObjectDeleteEnabled = multiObjectDeleteEnabled; this.metadataStore = metadataStore; this.keyToPathQualifier = keyToPathQualifier; this.bucketLocation = Optional.ofNullable(bucketLocation); @@ -202,8 +200,8 @@ public ChangeDetectionPolicy getChangeDetectionPolicy() { return changeDetectionPolicy; } - public boolean isEnableMultiObjectsDelete() { - return enableMultiObjectsDelete; + public boolean isMultiObjectDeleteEnabled() { + return multiObjectDeleteEnabled; } public MetadataStore getMetadataStore() { 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 83bf08cc2a78b..dc62d196e4e2f 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 @@ -46,6 +46,8 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; import static org.apache.hadoop.fs.s3a.Constants.ASSUMED_ROLE_ARN; import static org.apache.hadoop.fs.s3a.Constants.ENABLE_MULTI_DELETE; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBucketOverrides; import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.touchFiles; import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.extractUndeletedPaths; import static org.apache.hadoop.fs.s3a.test.ExtraAssertions.assertFileCount; @@ -106,13 +108,18 @@ public class ITestPartialRenamesDeletes extends AbstractS3ATestBase { /** * Test array for parameterized test runs. + *
    + *
  • Run 0: single deletes
  • + *
  • Run 1: multi deletes
  • + *
+ * * @return a list of parameter tuples. */ @Parameterized.Parameters public static Collection params() { return Arrays.asList(new Object[][]{ - {true}, {false}, + {true}, }); } @@ -174,6 +181,9 @@ public Configuration createAssumedRoleConfig() { private Configuration createAssumedRoleConfig(String roleARN) { Configuration conf = newAssumedRoleConfig(getContract().getConf(), roleARN); + String bucketName = getTestBucketName(conf); + + removeBucketOverrides(bucketName, conf, ENABLE_MULTI_DELETE); conf.setBoolean(ENABLE_MULTI_DELETE, multiDelete); return conf; } @@ -202,6 +212,15 @@ public void testCannotTouchUnderRODir() throws Throwable { () -> ContractTestUtils.touch(roleFS, readonlyChild)); } + @Test + public void testMultiDeleteOptionPropagated() throws Throwable { + describe("Verify the test parameter propagates to the store context"); + StoreContext ctx = roleFS.createStoreContext(); + Assertions.assertThat(ctx.isMultiObjectDeleteEnabled()) + .as(ctx.toString()) + .isEqualTo(multiDelete); + } + /** * Execute a sequence of rename operations with access locked down. */ @@ -255,20 +274,7 @@ public void testRenameSingleFileFailsLeavingSource() throws Throwable { // rename will fail in the delete phase AccessDeniedException deniedException = expectRenameForbidden( readOnlyFile, destDir); - if (multiDelete) { - // look in that exception for a multidelete - Throwable cause = deniedException.getCause(); - if (!(cause instanceof MultiObjectDeleteException)) { - throw new AssertionError("Expected a MultiObjectDeleteException " - + "as the cause ", deniedException); - } - MultiObjectDeleteException mde = (MultiObjectDeleteException) cause; - final List undeleted - = extractUndeletedPaths(mde, fs::keyToQualifiedPath); - Assertions.assertThat(undeleted) - .as("files which could not be deleted") - .containsExactly(readOnlyFile); - } + // and the source file is still there assertIsFile(readOnlyFile); @@ -312,7 +318,7 @@ public void testRenameFilesetFailsLeavingSourceUnchanged() throws Throwable { // create a set of files // this is done in parallel as it is 10x faster on a long-haul test run. int filecount = 10; - touchFiles(fs, readOnlyDir, filecount); + List createdFiles = touchFiles(fs, readOnlyDir, filecount); // are they all there? assertFileCount("files ready to rename", roleFS, readOnlyDir, (long) filecount); @@ -320,8 +326,25 @@ public void testRenameFilesetFailsLeavingSourceUnchanged() throws Throwable { // try to rename the directory LOG.info("Renaming readonly files {} to {}", readOnlyDir, destDir); - AccessDeniedException ex = expectRenameForbidden(readOnlyDir, destDir); - LOG.info("Result of renaming read-only files is AccessDeniedException", ex); + AccessDeniedException deniedException = expectRenameForbidden(readOnlyDir, destDir); + if (multiDelete) { + // look in that exception for a multidelete + Throwable cause = deniedException.getCause(); + if (!(cause instanceof MultiObjectDeleteException)) { + throw new AssertionError("Expected a MultiObjectDeleteException " + + "as the cause ", deniedException); + } + MultiObjectDeleteException mde = (MultiObjectDeleteException) cause; + final List undeleted + = extractUndeletedPaths(mde, fs::keyToQualifiedPath); + Assertions.assertThat(undeleted) + .as("files which could not be deleted") + .hasSize(filecount) + .containsAll(createdFiles) + .containsOnlyElementsOf(createdFiles); + } + LOG.info("Result of renaming read-only files is AccessDeniedException", + deniedException); assertFileCount("files in the source directory", roleFS, readOnlyDir, (long) filecount); From 72cdc87880078b636d7a9189a98be26fa6271f4e Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 24 Apr 2019 23:45:22 +0100 Subject: [PATCH 03/22] HADOOP-15183 evolving the tests to simulate the real failure scenario * Dir to delete contains a mix of r/o and r/w files * asserts that for MPU, only the r/o files are guaranteed to be undeleted. * in -Dscale, thousands of files (Created in parallel threads for max perf) Findings are interesting. For a bulk delete, even with a PAYG DDB table, its the metastore delete which takes O(files) and a slow O(files) at that. Change-Id: I388c24605cd4ac9e07bb43dd16fbfdda1e1cf085 --- .../hadoop/fs/contract/ContractTestUtils.java | 4 +- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 81 ++++--- .../fs/s3a/impl/MultiObjectDeleteSupport.java | 8 +- .../hadoop/fs/s3a/auth/RoleTestUtils.java | 3 + .../s3a/impl/ITestPartialRenamesDeletes.java | 200 +++++++++++++++--- .../s3a/impl/TestPartialDeleteFailures.java | 4 +- 6 files changed, 241 insertions(+), 59 deletions(-) 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 4d975a338cd61..fd1e2f10e1b4c 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 @@ -624,9 +624,10 @@ public static byte[] writeTextFile(FileSystem fs, * @param path path to write * @param overwrite overwrite flag * @param data source dataset. Can be null + * @return the path written to. * @throws IOException on any problem */ - public static void createFile(FileSystem fs, + public static Path createFile(FileSystem fs, Path path, boolean overwrite, byte[] data) throws IOException { @@ -636,6 +637,7 @@ public static void createFile(FileSystem fs, stream.write(data); } stream.close(); + return path; } finally { IOUtils.closeStream(stream); } 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 2121b0be3da24..eef9eb10051e6 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 @@ -88,10 +88,12 @@ import org.apache.hadoop.fs.CreateFlag; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Options; import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport; import org.apache.hadoop.fs.s3a.impl.StoreContext; import org.apache.hadoop.fs.s3a.select.InternalSelectConstants; +import org.apache.hadoop.util.DurationInfo; import org.apache.hadoop.util.LambdaUtils; import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.FileStatus; @@ -1319,6 +1321,21 @@ private boolean innerRename(Path source, Path dest) return true; } + /** + * Expose the superclass rename for ease of testing. + * This is inefficient as it calls getFileStatus on source and dest + * twice, but it always throws exceptions on failures, which is good. + * @param src path to be renamed + * @param dst new path after rename + * @param options rename options. + * @throws IOException failure. + */ + @VisibleForTesting + public void rename(final Path src, final Path dst, + final Options.Rename... options) throws IOException { + super.rename(src, dst, options); + } + /** * Low-level call to get at the object metadata. * @param path path to the object @@ -1529,16 +1546,19 @@ protected S3ListResult listObjects(S3ListRequest request) throws IOException { incrementReadOperations(); incrementStatistic(OBJECT_LIST_REQUESTS); validateListArguments(request); - return invoker.retryUntranslated( - request.toString(), - true, - () -> { - if (useListV1) { - return S3ListResult.v1(s3.listObjects(request.getV1())); - } else { - return S3ListResult.v2(s3.listObjectsV2(request.getV2())); - } - }); + try(DurationInfo ignored = + new DurationInfo(LOG, false, "LIST")) { + return invoker.retryUntranslated( + request.toString(), + true, + () -> { + if (useListV1) { + return S3ListResult.v1(s3.listObjects(request.getV1())); + } else { + return S3ListResult.v2(s3.listObjectsV2(request.getV2())); + } + }); + } } /** @@ -1566,20 +1586,24 @@ protected S3ListResult continueListObjects(S3ListRequest request, S3ListResult prevResult) throws IOException { incrementReadOperations(); validateListArguments(request); - return invoker.retryUntranslated( - request.toString(), - true, - () -> { - incrementStatistic(OBJECT_CONTINUE_LIST_REQUESTS); - if (useListV1) { - return S3ListResult.v1( - s3.listNextBatchOfObjects(prevResult.getV1())); - } else { - request.getV2().setContinuationToken(prevResult.getV2() - .getNextContinuationToken()); - return S3ListResult.v2(s3.listObjectsV2(request.getV2())); - } - }); + + try(DurationInfo ignored = + new DurationInfo(LOG, false, "LIST (continued)")) { + return invoker.retryUntranslated( + request.toString(), + true, + () -> { + incrementStatistic(OBJECT_CONTINUE_LIST_REQUESTS); + if (useListV1) { + return S3ListResult.v1( + s3.listNextBatchOfObjects(prevResult.getV1())); + } else { + request.getV2().setContinuationToken(prevResult.getV2() + .getNextContinuationToken()); + return S3ListResult.v2(s3.listObjectsV2(request.getV2())); + } + }); + } } /** @@ -1617,6 +1641,7 @@ protected void deleteObject(String key) throws AmazonClientException, IOException { blockRootDelete(key); incrementWriteOperations(); + LOG.debug("DELETE {}", key); invoker.retryUntranslated("Delete "+ bucket + ":/" + key, DELETE_CONSIDERED_IDEMPOTENT, ()-> { @@ -1675,7 +1700,8 @@ private void blockRootDelete(String key) throws InvalidRequestException { private void deleteObjects(DeleteObjectsRequest deleteRequest) throws MultiObjectDeleteException, AmazonClientException, IOException { incrementWriteOperations(); - try { + try(DurationInfo ignored = new DurationInfo(LOG, + false, "DELETE %d keys", deleteRequest.getKeys().size())) { invoker.retryUntranslated("delete", DELETE_CONSIDERED_IDEMPOTENT, () -> { @@ -2119,7 +2145,10 @@ private boolean innerDelete(S3AFileStatus status, boolean recursive) } } } - metadataStore.deleteSubtree(f); + try(DurationInfo ignored = + new DurationInfo(LOG, false, "Delete metastore")) { + metadataStore.deleteSubtree(f); + } } else { LOG.debug("delete: Path is a file"); deleteObjectAtPath(f, key, true); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteSupport.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteSupport.java index 8d5949af557a7..7419de71f588f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteSupport.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteSupport.java @@ -113,8 +113,8 @@ public static List extractUndeletedPaths( } /** - * Process a multi object delete exception by removing all undeleted paths - * from the list of paths being deleted. + * Process a {@code MultiObjectDeleteException} by + * removing all undeleted paths from the list of paths being deleted. * The original list is updated, and so becomes the list of successfully * deleted paths. * @param deleteException the delete exception. @@ -122,7 +122,7 @@ public static List extractUndeletedPaths( * This has all undeleted paths removed, leaving only those deleted. * @return the list of undeleted entries */ - public static List splitUndeleted( + public static List removeUndeletedPaths( final MultiObjectDeleteException deleteException, final Collection pathsBeingDeleted, final Function qualifier) { @@ -155,7 +155,7 @@ public Pair, List> splitUndeletedKeys( // Take this is list of paths // extract all undeleted entries contained in the exception and // then removes them from the original list. - List undeleted = splitUndeleted(deleteException, pathsBeingDeleted, + List undeleted = removeUndeletedPaths(deleteException, pathsBeingDeleted, qualifier); return Pair.of(undeleted, pathsBeingDeleted); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/RoleTestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/RoleTestUtils.java index 71acb57a75dba..b7e1edacb22d9 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/RoleTestUtils.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/RoleTestUtils.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.nio.file.AccessDeniedException; import java.util.List; +import java.util.Optional; import java.util.concurrent.Callable; import java.util.function.Function; import java.util.stream.Collectors; @@ -38,7 +39,9 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants; +import org.apache.hadoop.util.LambdaUtils; +import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching; 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 dc62d196e4e2f..e7c13a0048f79 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 @@ -20,14 +20,22 @@ import java.io.IOException; import java.nio.file.AccessDeniedException; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.List; import java.util.Set; import java.util.TreeSet; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; import java.util.stream.Collectors; +import java.util.stream.Stream; import com.amazonaws.services.s3.model.MultiObjectDeleteException; +import com.google.common.base.Charsets; +import com.google.common.util.concurrent.ListeningExecutorService; import org.assertj.core.api.Assertions; import org.junit.Test; import org.junit.runner.RunWith; @@ -36,23 +44,29 @@ import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.impl.WrappedIOException; import org.apache.hadoop.fs.s3a.AbstractS3ATestBase; +import org.apache.hadoop.fs.s3a.Constants; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3AUtils; +import org.apache.hadoop.util.BlockingThreadPoolExecutorService; +import org.apache.hadoop.util.DurationInfo; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertRenameOutcome; +import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; import static org.apache.hadoop.fs.s3a.Constants.ASSUMED_ROLE_ARN; import static org.apache.hadoop.fs.s3a.Constants.ENABLE_MULTI_DELETE; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBucketOverrides; -import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.touchFiles; -import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.extractUndeletedPaths; -import static org.apache.hadoop.fs.s3a.test.ExtraAssertions.assertFileCount; +import static org.apache.hadoop.fs.s3a.Constants.MAXIMUM_CONNECTIONS; +import static org.apache.hadoop.fs.s3a.Constants.MAX_THREADS; import static org.apache.hadoop.fs.s3a.S3ATestUtils.MetricDiff; import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestPropertyBool; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBucketOverrides; import static org.apache.hadoop.fs.s3a.S3ATestUtils.reset; import static org.apache.hadoop.fs.s3a.S3AUtils.applyLocatedFiles; import static org.apache.hadoop.fs.s3a.Statistic.FILES_DELETE_REJECTED; @@ -61,10 +75,17 @@ import static org.apache.hadoop.fs.s3a.auth.RoleModel.Statement; import static org.apache.hadoop.fs.s3a.auth.RoleModel.directory; import static org.apache.hadoop.fs.s3a.auth.RoleModel.statement; -import static org.apache.hadoop.fs.s3a.auth.RolePolicies.*; +import static org.apache.hadoop.fs.s3a.auth.RolePolicies.S3_ALL_BUCKETS; +import static org.apache.hadoop.fs.s3a.auth.RolePolicies.S3_BUCKET_READ_OPERATIONS; +import static org.apache.hadoop.fs.s3a.auth.RolePolicies.S3_PATH_RW_OPERATIONS; +import static org.apache.hadoop.fs.s3a.auth.RolePolicies.STATEMENT_ALLOW_SSE_KMS_RW; +import static org.apache.hadoop.fs.s3a.auth.RolePolicies.STATEMENT_S3GUARD_CLIENT; import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.bindRolePolicyStatements; import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.forbidden; import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.newAssumedRoleConfig; +import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.extractUndeletedPaths; +import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.removeUndeletedPaths; +import static org.apache.hadoop.fs.s3a.test.ExtraAssertions.assertFileCount; import static org.apache.hadoop.fs.s3a.test.ExtraAssertions.extractCause; import static org.apache.hadoop.test.LambdaTestUtils.eval; @@ -76,7 +97,7 @@ * All these test have a unique path for each run, with a roleFS having * full RW access to part of it, and R/O access to a restricted subdirectory */ -@SuppressWarnings({"IOResourceOpenedButNotSafelyClosed", "ThrowableNotThrown"}) +@SuppressWarnings("ThrowableNotThrown") @RunWith(Parameterized.class) public class ITestPartialRenamesDeletes extends AbstractS3ATestBase { @@ -88,6 +109,23 @@ public class ITestPartialRenamesDeletes extends AbstractS3ATestBase { private static final Statement STATEMENT_ALL_BUCKET_READ_ACCESS = statement(true, S3_ALL_BUCKETS, S3_BUCKET_READ_OPERATIONS); + /** Many threads for scale performance: {@value}. */ + public static final int EXECUTOR_THREAD_COUNT = 64; + + /** + * For submitting work. + */ + private static final ListeningExecutorService executor = + BlockingThreadPoolExecutorService.newInstance( + EXECUTOR_THREAD_COUNT, + EXECUTOR_THREAD_COUNT * 2, + 30, TimeUnit.SECONDS, + "test-operations"); + + public static final int FILE_COUNT_NON_SCALED = 10; + + public static final int FILE_COUNT_SCALED = 2000; + /** * A role FS; if non-null it is closed in teardown. */ @@ -106,6 +144,8 @@ public class ITestPartialRenamesDeletes extends AbstractS3ATestBase { private Configuration assumedRoleConfig; + private int filecount; + /** * Test array for parameterized test runs. *
    @@ -148,6 +188,11 @@ public void setup() throws Exception { .addResources(directory(destDir)) ); roleFS = (S3AFileSystem) readOnlyDir.getFileSystem(assumedRoleConfig); + boolean scaleTest = getTestPropertyBool( + getConfiguration(), + KEY_SCALE_TESTS_ENABLED, + DEFAULT_SCALE_TESTS_ENABLED); + filecount = scaleTest ? FILE_COUNT_SCALED : FILE_COUNT_NON_SCALED; } @Override @@ -188,6 +233,22 @@ private Configuration createAssumedRoleConfig(String roleARN) { return conf; } + @Override + protected Configuration createConfiguration() { + Configuration conf = super.createConfiguration(); + String bucketName = getTestBucketName(conf); + + // ramp up the number of connections we can have for maximum PUT + // performance + removeBucketOverrides(bucketName, conf, MAX_THREADS, MAXIMUM_CONNECTIONS); + conf.setInt(MAX_THREADS, EXECUTOR_THREAD_COUNT); + conf.setInt(MAXIMUM_CONNECTIONS, EXECUTOR_THREAD_COUNT * 2); + conf.setBoolean(ENABLE_MULTI_DELETE, multiDelete); + + return conf; + } + + /** * Create a unique path, which includes method name, * multdelete flag and a random UUID. @@ -317,8 +378,7 @@ public void testRenameFilesetFailsLeavingSourceUnchanged() throws Throwable { // create a set of files // this is done in parallel as it is 10x faster on a long-haul test run. - int filecount = 10; - List createdFiles = touchFiles(fs, readOnlyDir, filecount); + List createdFiles = createFiles(fs, readOnlyDir, filecount); // are they all there? assertFileCount("files ready to rename", roleFS, readOnlyDir, (long) filecount); @@ -329,25 +389,20 @@ public void testRenameFilesetFailsLeavingSourceUnchanged() throws Throwable { AccessDeniedException deniedException = expectRenameForbidden(readOnlyDir, destDir); if (multiDelete) { // look in that exception for a multidelete - Throwable cause = deniedException.getCause(); - if (!(cause instanceof MultiObjectDeleteException)) { - throw new AssertionError("Expected a MultiObjectDeleteException " - + "as the cause ", deniedException); - } - MultiObjectDeleteException mde = (MultiObjectDeleteException) cause; + MultiObjectDeleteException mde = extractCause( + MultiObjectDeleteException.class, deniedException); final List undeleted = extractUndeletedPaths(mde, fs::keyToQualifiedPath); Assertions.assertThat(undeleted) .as("files which could not be deleted") .hasSize(filecount) .containsAll(createdFiles) - .containsOnlyElementsOf(createdFiles); + .containsExactlyInAnyOrderElementsOf(createdFiles); } LOG.info("Result of renaming read-only files is AccessDeniedException", deniedException); assertFileCount("files in the source directory", roleFS, readOnlyDir, (long) filecount); - } /** @@ -362,20 +417,28 @@ public void testPartialDelete() throws Throwable { // the full FS S3AFileSystem fs = getFileSystem(); - int range = 10; - List readOnlyFiles = touchFiles(fs, readOnlyDir, range); + List readOnlyFiles = createFiles(fs, readOnlyDir, filecount); + List deletableFiles = createFiles(fs, destDir, filecount); + + // as a safety check, verify that one of the deletable files can be deleted + Path head = deletableFiles.remove(0); + assertTrue("delete " + head + " failed", + roleFS.delete(head, false)); + List allFiles = Stream.concat( + readOnlyFiles.stream(), + deletableFiles.stream()) + .collect(Collectors.toList()); // this set can be deleted by the role FS MetricDiff rejectionCount = new MetricDiff(roleFS, FILES_DELETE_REJECTED); MetricDiff deleteVerbCount = new MetricDiff(roleFS, OBJECT_DELETE_REQUESTS); - describe("Trying to delete read only directory with files %s", - ls(readOnlyDir)); + describe("Trying to delete read only directory"); AccessDeniedException ex = expectDeleteForbidden(readOnlyDir); if (multiDelete) { // multi-delete status checks extractCause(MultiObjectDeleteException.class, ex); - rejectionCount.assertDiffEquals("Wrong rejection count", range); + rejectionCount.assertDiffEquals("Wrong rejection count", filecount); deleteVerbCount.assertDiffEquals("Wrong delete count", 1); reset(rejectionCount, deleteVerbCount); } @@ -387,15 +450,25 @@ public void testPartialDelete() throws Throwable { if (multiDelete) { // multi-delete status checks extractCause(MultiObjectDeleteException.class, ex); - rejectionCount.assertDiffEquals("Wrong rejection count", range); deleteVerbCount.assertDiffEquals("Wrong delete count", 1); - reset(rejectionCount, deleteVerbCount); + MultiObjectDeleteException mde = extractCause( + MultiObjectDeleteException.class, ex); + final List undeleted + = removeUndeletedPaths(mde, allFiles, fs::keyToQualifiedPath); + Assertions.assertThat(undeleted) + .as("files which could not be deleted") + .containsExactlyInAnyOrderElementsOf(readOnlyFiles); + Assertions.assertThat(allFiles) + .as("files which were deleted") + .containsExactlyInAnyOrderElementsOf(deletableFiles); + rejectionCount.assertDiffEquals("Wrong rejection count", filecount); } + reset(rejectionCount, deleteVerbCount); // build the set of all paths under the directory tree through // a directory listing (i.e. not getFileStatus()). // small risk of observed inconsistency here on unguarded stores. - final Set roFListing = filesUnderPath(readOnlyDir, true); + final Set roFListing = listFilesUnderPath(readOnlyDir, true); String directoryList = roFListing .stream() @@ -458,7 +531,7 @@ private void pathMustNotExist(Path p) { * @return an unordered set of the paths. * @throws IOException failure */ - private Set filesUnderPath(Path path, boolean recursive) throws IOException { + private Set listFilesUnderPath(Path path, boolean recursive) throws IOException { Set files = new TreeSet<>(); applyLocatedFiles(getFileSystem().listFiles(path, recursive), (status) -> { @@ -466,4 +539,79 @@ private Set filesUnderPath(Path path, boolean recursive) throws IOExceptio }); return files; } + + /** + * Submit something async. + * @param call call to invoke + * @param type + * @return the future to wait for + */ + private static CompletableFuture submit( + final Callable call) { + return CompletableFuture.supplyAsync( + new CallableSupplier(call), executor); + } + + private static class CallableSupplier implements Supplier { + + final Callable call; + + CallableSupplier(final Callable call) { + this.call = call; + } + + @Override + public Object get() { + try { + return call.call(); + } catch (RuntimeException e) { + throw e; + } catch (IOException e) { + throw new WrappedIOException(e); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + } + + /** + * Write the text to a file asynchronously. Logs the operation too + * @param fs filesystem + * @param path path + * @return future to the patch created. + */ + private static CompletableFuture put(FileSystem fs, + Path path, String text) { + return submit(() -> { + try (DurationInfo ignore = + new DurationInfo(LOG, "Creating %s", path)) { + createFile(fs, path, true, text.getBytes(Charsets.UTF_8)); + return path; + } + }); + } + + /** + * Parallel-touch a set of files in the destination directory. + * @param fs filesystem + * @param destDir destination + * @param range range 1..range inclusive of files to create. + * @return the list of paths created. + */ + @SuppressWarnings("unchecked") + public static List createFiles(final FileSystem fs, + final Path destDir, + final int range) throws IOException { + CompletableFuture[] futures = new CompletableFuture[range]; + List paths = new ArrayList<>(range); + for (int i = 0; i < range; i++) { + String name = "file-" + i; + Path p = new Path(destDir, name); + paths.add(p); + futures[i] = put(fs, p, name); + } + CompletableFuture.allOf(futures).join(); + return paths; + } + } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java index 27db030f139d2..d16e1d75d972c 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java @@ -54,7 +54,7 @@ import org.apache.hadoop.util.BlockingThreadPoolExecutorService; import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.ACCESS_DENIED; -import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.splitUndeleted; +import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.removeUndeletedPaths; import static org.junit.Assert.assertEquals; /** @@ -81,7 +81,7 @@ public void testDeleteExtraction() { List rejected = pathList("a/b"); MultiObjectDeleteException ex = createDeleteException(ACCESS_DENIED, rejected); - List undeleted = splitUndeleted(ex, src, + List undeleted = removeUndeletedPaths(ex, src, TestPartialDeleteFailures::qualify); assertEquals(rejected, undeleted); } From 9162aa3974d8a264aaa185b5505ff652eafce30a Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 25 Apr 2019 13:31:47 +0100 Subject: [PATCH 04/22] HADOOP-15183: parallel submission of delete operations to DDB. This includes some changes in executor construction and the expansion of the defaults from the -too small- default values to something more realistic. Note: big production deployments against a single store still benefit from a very large thread pool and connection pool. The core-default comments cover that. These changes bring the time to delete 1000+ files to tens of seconds from minutes. TODO: 1. unwrap exceptions raised in the DDB operation and wrapped to IOEs 2. page the DDB deletions to some size (100?) to avoid going OOM on a wide/deep tree simply by the number of files to delete. That'll be slightly less efficient. Change-Id: I5e998be6cd4e86b486c744f7f8864ac7945244f4 --- .../src/main/resources/core-default.xml | 31 ++++++-- .../org/apache/hadoop/fs/s3a/Constants.java | 28 +++++-- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 9 ++- .../org/apache/hadoop/fs/s3a/S3AUtils.java | 4 +- .../hadoop/fs/s3a/impl/CallableSupplier.java | 73 +++++++++++++++++++ .../hadoop/fs/s3a/impl/StoreContext.java | 52 +++++++++++-- .../fs/s3a/s3guard/DynamoDBMetadataStore.java | 36 ++++++++- .../s3a/impl/ITestPartialRenamesDeletes.java | 68 +++++------------ .../s3a/impl/TestPartialDeleteFailures.java | 2 + .../hadoop/fs/s3a/test/ExtraAssertions.java | 13 +++- 10 files changed, 241 insertions(+), 75 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CallableSupplier.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 933130125f54a..e5e71c59ae7d9 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 @@ -1213,8 +1213,12 @@ fs.s3a.connection.maximum - 15 - Controls the maximum number of simultaneous connections to S3. + 48 + Controls the maximum number of simultaneous connections to S3. + This must be bigger than the value of fs.s3a.threads.max so as to stop + threads being blocked waiting for new HTTPS connections. + Why not equal? The AWS SDK transfer manager also uses these connections. + @@ -1312,7 +1316,7 @@ fs.s3a.threads.max - 10 + 64 The total number of threads available in the filesystem for data uploads *or any other queued filesystem operation*. @@ -1326,8 +1330,25 @@ fs.s3a.max.total.tasks - 5 - The number of operations which can be queued for execution + 32 + The number of operations which can be queued for execution. + This is in addition to the number of active threads in fs.s3a.threads.max. + + + + + fs.s3a.executor.capacity + 16 + The maximum number of submitted tasks which is a single + operation (e.g. rename(), delete()) may submit simultaneously for + execution -excluding the IO-heavy block uploads, whose capacity + is set in "fs.s3a.fast.upload.active.blocks" + + All tasks are submitted to the shared thread pool whose size is + set in "fs.s3a.threads.max"; the value of capacity should be less than that + of the thread pool itself, as the goal is to stop a single operation + from overloading that thread pool. + 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 7a687943cfb7b..6bd89b19835aa 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 @@ -139,9 +139,15 @@ private Constants() { public static final String ASSUMED_ROLE_CREDENTIALS_DEFAULT = SimpleAWSCredentialsProvider.NAME; + + // the maximum number of tasks cached if all threads are already uploading + public static final String MAX_TOTAL_TASKS = "fs.s3a.max.total.tasks"; + + public static final int DEFAULT_MAX_TOTAL_TASKS = 5; + // number of simultaneous connections to s3 public static final String MAXIMUM_CONNECTIONS = "fs.s3a.connection.maximum"; - public static final int DEFAULT_MAXIMUM_CONNECTIONS = 15; + public static final int DEFAULT_MAXIMUM_CONNECTIONS = DEFAULT_MAX_TOTAL_TASKS * 2; // connect to s3 over ssl? public static final String SECURE_CONNECTIONS = @@ -200,10 +206,6 @@ private Constants() { public static final String KEEPALIVE_TIME = "fs.s3a.threads.keepalivetime"; public static final int DEFAULT_KEEPALIVE_TIME = 60; - // the maximum number of tasks cached if all threads are already uploading - public static final String MAX_TOTAL_TASKS = "fs.s3a.max.total.tasks"; - public static final int DEFAULT_MAX_TOTAL_TASKS = 5; - // size of each of or multipart pieces in bytes public static final String MULTIPART_SIZE = "fs.s3a.multipart.size"; public static final long DEFAULT_MULTIPART_SIZE = 104857600; // 100 MB @@ -289,6 +291,22 @@ private Constants() { @InterfaceStability.Unstable public static final int DEFAULT_FAST_UPLOAD_ACTIVE_BLOCKS = 4; + /** + * The capacity of executor queues for operations other than block + * upload, where {@link #FAST_UPLOAD_ACTIVE_BLOCKS} is used instead. + * This should be less than {@link #MAX_THREADS} for fair + * submission. + * Value: {@value}. + */ + public static final String EXECUTOR_CAPACITY = "fs.s3a.executor.capacity"; + + /** + * The capacity of executor queues for operations other than block + * upload, where {@link #FAST_UPLOAD_ACTIVE_BLOCKS} is used instead. + * Value: {@value} + */ + public static final int DEFAULT_EXECUTOR_CAPACITY = 10; + // Private | PublicRead | PublicReadWrite | AuthenticatedRead | // LogDeliveryWrite | BucketOwnerRead | BucketOwnerFullControl public static final String CANNED_ACL = "fs.s3a.acl.default"; 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 eef9eb10051e6..cfffb85219c8c 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 @@ -202,6 +202,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, private TransferManager transfers; private ListeningExecutorService boundedThreadPool; private ExecutorService unboundedThreadPool; + private int executorCapacity; private long multiPartThreshold; public static final Logger LOG = LoggerFactory.getLogger(S3AFileSystem.class); private static final Logger PROGRESS = @@ -343,14 +344,15 @@ public void initialize(URI name, Configuration originalConf) maxThreads, maxThreads + totalTasks, keepAliveTime, TimeUnit.SECONDS, - "s3a-transfer-shared"); + "s3a-transfer-shared-" + bucket); unboundedThreadPool = new ThreadPoolExecutor( maxThreads, Integer.MAX_VALUE, keepAliveTime, TimeUnit.SECONDS, new LinkedBlockingQueue(), BlockingThreadPoolExecutorService.newDaemonThreadFactory( - "s3a-transfer-unbounded")); - + "s3a-transfer-unbounded-" + bucket)); + executorCapacity = intOption(conf, + EXECUTOR_CAPACITY, DEFAULT_EXECUTOR_CAPACITY, 1); int listVersion = conf.getInt(LIST_VERSION, DEFAULT_LIST_VERSION); if (listVersion < 1 || listVersion > 2) { LOG.warn("Configured fs.s3a.list.version {} is invalid, forcing " + @@ -2380,6 +2382,7 @@ public StoreContext createStoreContext() { getUsername(), owner, boundedThreadPool, + executorCapacity, invoker, directoryAllocator, getInstrumentation(), 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 6b3505e4361e4..934d798ea4c14 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 @@ -965,7 +965,7 @@ public static String stringify(S3ObjectSummary summary) { * @return the value * @throws IllegalArgumentException if the value is below the minimum */ - static int intOption(Configuration conf, String key, int defVal, int min) { + public static int intOption(Configuration conf, String key, int defVal, int min) { int v = conf.getInt(key, defVal); Preconditions.checkArgument(v >= min, String.format("Value of %s: %d is below the minimum value %d", @@ -983,7 +983,7 @@ static int intOption(Configuration conf, String key, int defVal, int min) { * @return the value * @throws IllegalArgumentException if the value is below the minimum */ - static long longOption(Configuration conf, + public static long longOption(Configuration conf, String key, long defVal, long min) { 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 new file mode 100644 index 0000000000000..acccf53d54eb3 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CallableSupplier.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl; + +import java.io.IOException; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; +import java.util.function.Supplier; + +import org.apache.hadoop.fs.impl.WrappedIOException; + +/** + * A bridge from Callable to Supplier; catching exceptions + * raised by the callable and wrapping them as appropriate. + * @param return type. + */ +public final class CallableSupplier implements Supplier { + + final Callable call; + + CallableSupplier(final Callable call) { + this.call = call; + } + + @Override + public Object get() { + try { + return call.call(); + } catch (RuntimeException e) { + throw e; + } catch (IOException e) { + throw new WrappedIOException(e); + } catch (Exception e) { + throw new WrappedIOException(new IOException(e)); + } + } + + /** + * Submit a callable into a completable future. + * RTEs are rethrown. + * Non RTEs are caught and wrapped; IOExceptions to + * {@link WrappedIOException} instances. + * @param executor executor. + * @param call call to invoke + * @param type + * @return the future to wait for + */ + @SuppressWarnings("unchecked") + public static CompletableFuture submit( + final Executor executor, + final Callable call) { + return CompletableFuture.supplyAsync( + new CallableSupplier(call), executor); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java index ffd2a8a95fe7f..654b282d572de 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java @@ -36,6 +36,7 @@ import org.apache.hadoop.fs.s3a.Statistic; import org.apache.hadoop.fs.s3a.s3guard.MetadataStore; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.util.SemaphoredDelegatingExecutor; /** * This class provides the core context of the S3A filesystem to subsidiary @@ -77,7 +78,12 @@ public class StoreContext { /** * Bounded thread pool for async operations. */ - private final ListeningExecutorService boundedThreadPool; + private final ListeningExecutorService executor; + + /** + * Capacity of new executors created. + */ + private final int executorCapacity; /** Invoker of operations. */ private final Invoker invoker; @@ -113,13 +119,16 @@ public class StoreContext { /** Function to take a key and return a path. */ private final Function keyToPathQualifier; + /** + * Instantiate. + */ public StoreContext(final URI fsURI, final String bucket, final Configuration configuration, final String username, final UserGroupInformation owner, - final ListeningExecutorService boundedThreadPool, - final Invoker invoker, + final ListeningExecutorService executor, + final int executorCapacity, final Invoker invoker, final LocalDirAllocator directoryAllocator, final S3AInstrumentation instrumentation, final S3AStorageStatistics storageStatistics, @@ -136,7 +145,8 @@ public StoreContext(final URI fsURI, this.configuration = configuration; this.username = username; this.owner = owner; - this.boundedThreadPool = boundedThreadPool; + this.executor = executor; + this.executorCapacity = executorCapacity; this.invoker = invoker; this.directoryAllocator = directoryAllocator; this.instrumentation = instrumentation; @@ -176,8 +186,8 @@ public Optional getBucketLocation() { return bucketLocation; } - public ListeningExecutorService getBoundedThreadPool() { - return boundedThreadPool; + public ListeningExecutorService getExecutor() { + return executor; } public Invoker getInvoker() { @@ -228,7 +238,6 @@ public S3AStorageStatistics getStorageStatistics() { return storageStatistics; } - /** * Increment a statistic by 1. * This increments both the instrumentation and storage statistics. @@ -267,5 +276,34 @@ public void incrementGauge(Statistic statistic, long count) { instrumentation.incrementGauge(statistic, count); } + /** + * Create a new executor service with a given capacity. + * This executor submits works to the {@link #executor}, using a + * {@link SemaphoredDelegatingExecutor} to limit the number + * of requests coming in from a specific client. + * + * Because this delegates to an existing thread pool, the cost of + * creating a new instance here is low. + * As the throttling is per instance, separate instances + * should be created for each operation which wishes to execute work in + * parallel without saturating the base executor. + * This is important if either the duration of each operation is long + * or the submission rate of work is high. + * @param capacity maximum capacity of this executor. + * @return an executor for submitting work. + */ + public ListeningExecutorService createThrottledExecutor(int capacity) { + return new SemaphoredDelegatingExecutor(executor, + capacity, true); + } + + /** + * Create a new executor with the capacity defined in + * {@link #executorCapacity} + * @return a new executor for exclusive use by the caller. + */ + public ListeningExecutorService createThrottledExecutor() { + return createThrottledExecutor(executorCapacity); + } } 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 769d3d4c4c376..3c95912b211a6 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 @@ -35,6 +35,7 @@ import java.util.Objects; import java.util.Set; import java.util.TreeMap; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; @@ -72,6 +73,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; +import com.google.common.util.concurrent.ListeningExecutorService; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -95,15 +97,18 @@ import org.apache.hadoop.fs.s3a.auth.RoleModel; import org.apache.hadoop.fs.s3a.auth.RolePolicies; import org.apache.hadoop.fs.s3a.auth.delegation.AWSPolicyProvider; +import org.apache.hadoop.fs.s3a.impl.StoreContext; import org.apache.hadoop.io.retry.RetryPolicies; import org.apache.hadoop.io.retry.RetryPolicy; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.util.BlockingThreadPoolExecutorService; import org.apache.hadoop.util.ReflectionUtils; import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.S3AUtils.*; import static org.apache.hadoop.fs.s3a.auth.RolePolicies.allowAllDynamoDBOperations; import static org.apache.hadoop.fs.s3a.auth.RolePolicies.allowS3GuardClientOperations; +import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit; import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.*; import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.*; @@ -296,6 +301,8 @@ public class DynamoDBMetadataStore implements MetadataStore, */ private AtomicInteger throttleEventCount = new AtomicInteger(0); + private ListeningExecutorService executor; + /** * A utility function to create DynamoDB instance. * @param conf the file system configuration @@ -386,8 +393,10 @@ public void initialize(FileSystem fs) throws IOException { void bindToOwnerFilesystem(final S3AFileSystem fs) { owner = fs; conf = owner.getConf(); - instrumentation = owner.getInstrumentation().getS3GuardInstrumentation(); - username = owner.getUsername(); + StoreContext context = owner.createStoreContext(); + instrumentation = context.getInstrumentation().getS3GuardInstrumentation(); + username = context.getUsername(); + executor = context.createThrottledExecutor(); } /** @@ -432,6 +441,18 @@ public void initialize(Configuration config) throws IOException { dynamoDB = createDynamoDB(conf, region, null, credentials); username = UserGroupInformation.getCurrentUser().getShortUserName(); + // without an executor from the owner FS, create one using + // the executor capacity for work. + int executorCapacity = intOption(conf, + EXECUTOR_CAPACITY, DEFAULT_EXECUTOR_CAPACITY, 1); + executor = BlockingThreadPoolExecutorService.newInstance( + executorCapacity, + executorCapacity * 2, + longOption(conf, KEEPALIVE_TIME, + DEFAULT_KEEPALIVE_TIME, 0), + TimeUnit.SECONDS, + "s3a-ddb-" + tableName); + initDataAccessRetries(conf); initTable(); @@ -525,10 +546,19 @@ public void deleteSubtree(Path path) throws IOException { return; } + // bulk execute. This needs to be paged better. + List> futures = new ArrayList<>(); for (DescendantsIterator desc = new DescendantsIterator(this, meta); desc.hasNext();) { - innerDelete(desc.next().getPath(), true); + final Path pathToDelete = desc.next().getPath(); + futures.add(submit(executor, () -> { + innerDelete(pathToDelete, true); + return null; + })); } + // await completion + CompletableFuture.allOf(futures.toArray( + new CompletableFuture[futures.size()])).join(); } /** 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 e7c13a0048f79..8e8848b2d7041 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 @@ -26,10 +26,8 @@ import java.util.List; import java.util.Set; import java.util.TreeSet; -import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; -import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -47,9 +45,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.ContractTestUtils; -import org.apache.hadoop.fs.impl.WrappedIOException; import org.apache.hadoop.fs.s3a.AbstractS3ATestBase; -import org.apache.hadoop.fs.s3a.Constants; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3AUtils; import org.apache.hadoop.util.BlockingThreadPoolExecutorService; @@ -83,6 +79,7 @@ import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.bindRolePolicyStatements; import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.forbidden; import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.newAssumedRoleConfig; +import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit; import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.extractUndeletedPaths; import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.removeUndeletedPaths; import static org.apache.hadoop.fs.s3a.test.ExtraAssertions.assertFileCount; @@ -124,7 +121,7 @@ public class ITestPartialRenamesDeletes extends AbstractS3ATestBase { public static final int FILE_COUNT_NON_SCALED = 10; - public static final int FILE_COUNT_SCALED = 2000; + public static final int FILE_COUNT_SCALED = 1000; /** * A role FS; if non-null it is closed in teardown. @@ -146,6 +143,8 @@ public class ITestPartialRenamesDeletes extends AbstractS3ATestBase { private int filecount; + private boolean scaleTest; + /** * Test array for parameterized test runs. *
      @@ -188,11 +187,16 @@ public void setup() throws Exception { .addResources(directory(destDir)) ); roleFS = (S3AFileSystem) readOnlyDir.getFileSystem(assumedRoleConfig); - boolean scaleTest = getTestPropertyBool( + scaleTest = multiDelete && getTestPropertyBool( getConfiguration(), KEY_SCALE_TESTS_ENABLED, DEFAULT_SCALE_TESTS_ENABLED); - filecount = scaleTest ? FILE_COUNT_SCALED : FILE_COUNT_NON_SCALED; + // switch to the big set of files iff this is a multidelete run + // with -Dscale set. + // without that the DELETE calls become a key part of the bottleneck + filecount = scaleTest + ? FILE_COUNT_SCALED + : FILE_COUNT_NON_SCALED; } @Override @@ -442,8 +446,10 @@ public void testPartialDelete() throws Throwable { deleteVerbCount.assertDiffEquals("Wrong delete count", 1); reset(rejectionCount, deleteVerbCount); } - // all the files are still there - readOnlyFiles.forEach(this::pathMustExist); + // all the files are still there? (avoid in scale test due to cost) + if (!scaleTest) { + readOnlyFiles.forEach(this::pathMustExist); + } describe("Trying to delete upper-level directory"); ex = expectDeleteForbidden(basePath); @@ -540,40 +546,6 @@ private Set listFilesUnderPath(Path path, boolean recursive) throws IOExce return files; } - /** - * Submit something async. - * @param call call to invoke - * @param type - * @return the future to wait for - */ - private static CompletableFuture submit( - final Callable call) { - return CompletableFuture.supplyAsync( - new CallableSupplier(call), executor); - } - - private static class CallableSupplier implements Supplier { - - final Callable call; - - CallableSupplier(final Callable call) { - this.call = call; - } - - @Override - public Object get() { - try { - return call.call(); - } catch (RuntimeException e) { - throw e; - } catch (IOException e) { - throw new WrappedIOException(e); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - } - /** * Write the text to a file asynchronously. Logs the operation too * @param fs filesystem @@ -582,7 +554,7 @@ public Object get() { */ private static CompletableFuture put(FileSystem fs, Path path, String text) { - return submit(() -> { + return submit(executor, () -> { try (DurationInfo ignore = new DurationInfo(LOG, "Creating %s", path)) { createFile(fs, path, true, text.getBytes(Charsets.UTF_8)); @@ -598,19 +570,19 @@ private static CompletableFuture put(FileSystem fs, * @param range range 1..range inclusive of files to create. * @return the list of paths created. */ - @SuppressWarnings("unchecked") public static List createFiles(final FileSystem fs, final Path destDir, final int range) throws IOException { - CompletableFuture[] futures = new CompletableFuture[range]; + List> futures = new ArrayList<>(range); List paths = new ArrayList<>(range); for (int i = 0; i < range; i++) { String name = "file-" + i; Path p = new Path(destDir, name); paths.add(p); - futures[i] = put(fs, p, name); + futures.add(put(fs, p, name)); } - CompletableFuture.allOf(futures).join(); + CompletableFuture.allOf(futures.toArray( + new CompletableFuture[futures.size()])).join(); return paths; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java index d16e1d75d972c..203d255c46619 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java @@ -42,6 +42,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.Constants; import org.apache.hadoop.fs.s3a.Invoker; import org.apache.hadoop.fs.s3a.S3AInputPolicy; import org.apache.hadoop.fs.s3a.S3AInstrumentation; @@ -201,6 +202,7 @@ StoreContext craateMockStoreContext(boolean multiDelete, 4, 10, TimeUnit.SECONDS, "s3a-transfer-shared"), + Constants.DEFAULT_EXECUTOR_CAPACITY, new Invoker(RetryPolicies.TRY_ONCE_THEN_FAIL, Invoker.LOG_EVENT), null, new S3AInstrumentation(name), diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/ExtraAssertions.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/ExtraAssertions.java index f1a4c5554594f..7f42ebef78752 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/ExtraAssertions.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/ExtraAssertions.java @@ -24,11 +24,14 @@ import java.util.stream.Collectors; import org.junit.Assert; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.util.DurationInfo; import static org.apache.hadoop.fs.s3a.S3AUtils.applyLocatedFiles; import static org.junit.Assert.assertTrue; @@ -39,6 +42,9 @@ @InterfaceAudience.Private public class ExtraAssertions { + private static final Logger LOG = LoggerFactory.getLogger( + ExtraAssertions.class); + /** * Assert that the number of files in a destination matches that expected. * @param text text to use in the message @@ -51,8 +57,11 @@ public static void assertFileCount(String text, FileSystem fs, Path path, long expected) throws IOException { List files = new ArrayList<>(); - applyLocatedFiles(fs.listFiles(path, true), - (status) -> files.add(status.getPath().toString())); + try (DurationInfo ignored = new DurationInfo(LOG, false, + "Counting files in %s", path)) { + applyLocatedFiles(fs.listFiles(path, true), + (status) -> files.add(status.getPath().toString())); + } long actual = files.size(); if (actual != expected) { String ls = files.stream().collect(Collectors.joining("\n")); From 81f87cda7ea2514f1f27345b9379c980ff590b62 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 25 Apr 2019 15:35:53 +0100 Subject: [PATCH 05/22] HADOOP-15183 peformance boost of prune on on-demand DDB tables: * disable sleep if interval set to 0. There's no need to throttle it any more * track the # of parent directories across batches to avoid a prune of 1000 files to trigger 40 attempts at changing the parent state. * ITestPartialRenamesDeletes to attempt a prune of the path in teardown This is all to address the fact that once you have a scale test createing a deleting many thousands of files in unique paths, in every test run, you accrue many tombstone markers fast, which prune then goes after. Change-Id: I02f7dffa94785a01cd2a2452cf2ad67d4534f32b --- .../fs/s3a/s3guard/DynamoDBMetadataStore.java | 23 +++++- .../s3a/impl/ITestPartialRenamesDeletes.java | 79 +++++++++++++++---- .../s3guard/AbstractS3GuardToolTestBase.java | 10 +++ 3 files changed, 91 insertions(+), 21 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 3c95912b211a6..202008ded64a4 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 @@ -919,6 +919,13 @@ public void put(Collection metas) throws IOException { @Retries.OnceRaw private void innerPut(Collection metas) throws IOException { + if (metas.isEmpty()) { + // this seems to appear in the logs, so log the full stack to + // identify it. + LOG.debug("Ignoring empty list of entries to put", + new Exception("source")); + return; + } Item[] items = pathMetadataToItem(completeAncestry(metas)); LOG.debug("Saving batch of {} items to table {}, region {}", items.length, tableName, region); @@ -1086,25 +1093,31 @@ public void prune(long modTime, String keyPrefix) throws IOException { S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_DEFAULT, TimeUnit.MILLISECONDS); Set parentPathSet = new HashSet<>(); + Set clearedParentPathSet = new HashSet<>(); for (Item item : expiredFiles(modTime, keyPrefix)) { DDBPathMetadata md = PathMetadataDynamoDBTranslation .itemToPathMetadata(item, username); Path path = md.getFileStatus().getPath(); deletionBatch.add(path); - // add parent path of what we remove + // add parent path of what we remove if it has not + // already been processed Path parentPath = path.getParent(); - if (parentPath != null) { + if (parentPath != null && !clearedParentPathSet.contains(parentPath)) { parentPathSet.add(parentPath); } itemCount++; if (deletionBatch.size() == S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT) { - Thread.sleep(delay); + if (delay > 0) { + Thread.sleep(delay); + } processBatchWriteRequest(pathToKey(deletionBatch), null); // set authoritative false for each pruned dir listing removeAuthoritativeDirFlag(parentPathSet); + // already cleared parent paths. + clearedParentPathSet.addAll(parentPathSet); parentPathSet.clear(); deletionBatch.clear(); @@ -1151,7 +1164,9 @@ private void removeAuthoritativeDirFlag(Set pathSet) try { LOG.debug("innerPut on metas: {}", metas); - innerPut(metas); + if (!metas.isEmpty()) { + innerPut(metas); + } } catch (IOException e) { String msg = String.format("IOException while setting false " + "authoritative directory flag on: %s.", metas); 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 8e8848b2d7041..9ef8897663798 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 @@ -48,6 +48,7 @@ import org.apache.hadoop.fs.s3a.AbstractS3ATestBase; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3AUtils; +import org.apache.hadoop.fs.s3a.s3guard.MetadataStore; import org.apache.hadoop.util.BlockingThreadPoolExecutorService; import org.apache.hadoop.util.DurationInfo; @@ -58,6 +59,8 @@ import static org.apache.hadoop.fs.s3a.Constants.ENABLE_MULTI_DELETE; import static org.apache.hadoop.fs.s3a.Constants.MAXIMUM_CONNECTIONS; import static org.apache.hadoop.fs.s3a.Constants.MAX_THREADS; +import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_DEFAULT; +import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY; import static org.apache.hadoop.fs.s3a.S3ATestUtils.MetricDiff; import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume; import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName; @@ -121,7 +124,16 @@ public class ITestPartialRenamesDeletes extends AbstractS3ATestBase { public static final int FILE_COUNT_NON_SCALED = 10; - public static final int FILE_COUNT_SCALED = 1000; + /** + * The number of files for a scaled test. This is still + * less than half the amount which can be fitted into a delete + * request, so that even with this many R/W and R/O files, + * both can fit in the same request. + * Then, when a partial delete occurs, we can make assertions + * knowing that all R/W files should have been deleted and all + * R/O files rejected. + */ + public static final int FILE_COUNT_SCALED = 400; /** * A role FS; if non-null it is closed in teardown. @@ -205,6 +217,17 @@ public void teardown() throws Exception { super.teardown(); } + @Override + protected void deleteTestDirInTeardown() throws IOException { + super.deleteTestDirInTeardown(); + Path path = getContract().getTestPath(); + try { + prune(path); + } catch (IOException e) { + LOG.warn("When pruning the test directory {}", path, e); + } + } + private void assumeRoleTests() { assume("No ARN for role tests", !getAssumedRoleARN().isEmpty()); } @@ -248,7 +271,9 @@ protected Configuration createConfiguration() { conf.setInt(MAX_THREADS, EXECUTOR_THREAD_COUNT); conf.setInt(MAXIMUM_CONNECTIONS, EXECUTOR_THREAD_COUNT * 2); conf.setBoolean(ENABLE_MULTI_DELETE, multiDelete); - + // turn off prune delays, so as to stop scale tests creating + // so much cruft that future CLI prune commands take forever + conf.setInt(S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY, 0); return conf; } @@ -487,13 +512,15 @@ public void testPartialDelete() throws Throwable { } private AccessDeniedException expectDeleteForbidden(Path path) throws Exception { - return forbidden("Expected an error deleting " + path, - "", - () -> { - boolean r = roleFS.delete(path, true); - return " delete=" + r + " " + ls(path.getParent()); - } - ); + try(DurationInfo ignored = + new DurationInfo(LOG, true, "delete %s", path)) { + return forbidden("Expected an error deleting " + path, + "", + () -> { + boolean r = roleFS.delete(path, true); + return " delete=" + r + " " + ls(path.getParent()); + }); + } } /** @@ -505,13 +532,16 @@ private AccessDeniedException expectDeleteForbidden(Path path) throws Exception */ private AccessDeniedException expectRenameForbidden(Path src, Path dest) throws Exception { - return forbidden( - "Renaming " + src + " to " + dest, - "", - () -> { - roleFS.rename(src, dest); - return ContractTestUtils.ls(getFileSystem(), src.getParent()); - }); + try(DurationInfo ignored = + new DurationInfo(LOG, true, "rename")) { + return forbidden( + "Renaming " + src + " to " + dest, + "", + () -> { + roleFS.rename(src, dest); + return ContractTestUtils.ls(getFileSystem(), src.getParent()); + }); + } } /** @@ -530,6 +560,21 @@ private void pathMustNotExist(Path p) { eval(() -> assertPathDoesNotExist("Path should not exist", p)); } + /** + * Prune the store for everything under the test path. + * @param path path. + * @throws IOException on failure. + */ + private void prune(Path path) throws IOException { + S3AFileSystem fs = getFileSystem(); + if (fs.hasMetadataStore()) { + MetadataStore store = fs.getMetadataStore(); + try(DurationInfo ignored = new DurationInfo(LOG, true, "prune %s", path)) { + store.prune(System.currentTimeMillis(), fs.pathToKey(path)); + } + } + } + /** * List all files under a path. * @param path path to list @@ -547,7 +592,7 @@ private Set listFilesUnderPath(Path path, boolean recursive) throws IOExce } /** - * Write the text to a file asynchronously. Logs the operation too + * Write the text to a file asynchronously. Logs the operation duration. * @param fs filesystem * @param path path * @return future to the patch created. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java index ad4691a6d962b..a3ad91b8c33b7 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java @@ -81,6 +81,16 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase { private MetadataStore ms; private S3AFileSystem rawFs; + /** + * The test timeout is increased in case previous tests have created + * many tombstone markers which now need to be purged. + * @return the test timeout. + */ + @Override + protected int getTestTimeoutMillis() { + return SCALE_TEST_TIMEOUT_SECONDS * 1000; + } + protected static void expectResult(int expected, String message, S3GuardTool tool, From a7f2425669e32b59143b80184b42eb05de31aef4 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 25 Apr 2019 18:19:28 +0100 Subject: [PATCH 06/22] HADOOP-15183: improve parallel execution of metastore deletes we move to batched execution of (parallel) delete calls, so that a failure fails fast and and there's no need to build up structures of millions of completable futures Change-Id: I5e58310fcf7f545ce8904b0d2a80a02cb2681cf4 --- .../hadoop/fs/impl/FutureIOSupport.java | 48 ++++++++++++++++--- .../hadoop/fs/s3a/impl/CallableSupplier.java | 40 +++++++++++++++- .../fs/s3a/s3guard/DynamoDBMetadataStore.java | 17 +++++-- .../site/markdown/tools/hadoop-aws/s3guard.md | 12 +++++ .../s3a/impl/ITestPartialRenamesDeletes.java | 42 +++++++++++----- .../s3a/s3guard/ITestS3GuardToolDynamoDB.java | 5 +- 6 files changed, 140 insertions(+), 24 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureIOSupport.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureIOSupport.java index 9d5f2bf4b6ed1..72f5f91ac84b5 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureIOSupport.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureIOSupport.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.io.InterruptedIOException; import java.util.Map; +import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; @@ -108,20 +109,55 @@ public static T awaitFuture(final Future future, */ public static T raiseInnerCause(final ExecutionException e) throws IOException { + throw unwrapInnerException(e); + } + + /** + * Extract the cause of a completion failure and rethrow it if an IOE + * or RTE + * @param e exception. + * @param type of return value. + * @return nothing, ever. + * @throws IOException either the inner IOException, or a wrapper around + * any non-Runtime-Exception + * @throws RuntimeException if that is the inner cause. + */ + public static T raiseInnerCause(final CompletionException e) + throws IOException { + throw unwrapInnerException(e); + } + + /** + * From the inner cause of an execution exception, extract the inner cause. + * If it is an RTE: throw immediately. + * If it is an IOE: Return. + * If it is a WrappedIOException: Unwrap and return + * Else: create a new IOException. + * + * Recursively handles wrapped Execution and Completion Exceptions in + * case something very complicated has happened. + * @param e exception. + * @return an IOException extracted or built from the cause. + * @throws RuntimeException if that is the inner cause. + */ + private static IOException unwrapInnerException(final Throwable e) { Throwable cause = e.getCause(); if (cause instanceof IOException) { - throw (IOException) cause; + return (IOException) cause; } else if (cause instanceof WrappedIOException){ - throw ((WrappedIOException) cause).getCause(); + return ((WrappedIOException) cause).getCause(); + } else if (cause instanceof CompletionException){ + return unwrapInnerException(cause); + } else if (cause instanceof ExecutionException){ + return unwrapInnerException(cause); } else if (cause instanceof RuntimeException){ throw (RuntimeException) cause; } else if (cause != null) { // other type: wrap with a new IOE - throw new IOException(cause); + return new IOException(cause); } else { - // this only happens if somebody deliberately raises - // an ExecutionException - throw new IOException(e); + // this only happens if there was no cause. + return new IOException(e); } } 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 acccf53d54eb3..68b7ae6be351e 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 @@ -19,12 +19,21 @@ package org.apache.hadoop.fs.s3a.impl; import java.io.IOException; +import java.util.List; import java.util.concurrent.Callable; +import java.util.concurrent.CancellationException; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; import java.util.concurrent.Executor; import java.util.function.Supplier; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import org.apache.hadoop.fs.impl.WrappedIOException; +import org.apache.hadoop.util.DurationInfo; + +import static org.apache.hadoop.fs.impl.FutureIOSupport.raiseInnerCause; /** * A bridge from Callable to Supplier; catching exceptions @@ -33,7 +42,10 @@ */ public final class CallableSupplier implements Supplier { - final Callable call; + private static final Logger LOG = + LoggerFactory.getLogger(CallableSupplier.class); + + private final Callable call; CallableSupplier(final Callable call) { this.call = call; @@ -70,4 +82,30 @@ public static CompletableFuture submit( new CallableSupplier(call), executor); } + /** + * Wait for a list of futures to complete + * @param futures list of futures. + * @throws IOException if one of the called futures raised an IOE. + * @throws RuntimeException if one of the futures raised one. + */ + public static void waitForCompletion( + final List> futures) + throws IOException { + if (futures.isEmpty()) { + return; + } + // await completion + CompletableFuture all = CompletableFuture.allOf( + futures.toArray( + new CompletableFuture[futures.size()])); + try(DurationInfo ignore = + new DurationInfo(LOG, false, "Waiting for task completion")) { + all.join(); + } catch (CancellationException e) { + throw new IOException(e); + } catch (CompletionException e) { + raiseInnerCause(e); + } + } + } 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 202008ded64a4..1c918096f252a 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 @@ -84,6 +84,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.impl.WrappedIOException; import org.apache.hadoop.fs.s3a.AWSClientIOException; import org.apache.hadoop.fs.s3a.AWSCredentialProviderList; import org.apache.hadoop.fs.s3a.AWSServiceThrottledException; @@ -109,6 +110,7 @@ import static org.apache.hadoop.fs.s3a.auth.RolePolicies.allowAllDynamoDBOperations; import static org.apache.hadoop.fs.s3a.auth.RolePolicies.allowS3GuardClientOperations; 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.s3guard.PathMetadataDynamoDBTranslation.*; import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.*; @@ -252,6 +254,13 @@ public class DynamoDBMetadataStore implements MetadataStore, private static ValueMap deleteTrackingValueMap = new ValueMap().withBoolean(":false", false); + /** + * The maximum number of oustanding operations to submit at a time + * in any operation whch submits work through the executors. + * Value: {@value}. + */ + private static final int S3GUARD_DDB_SUBMITTED_TASK_LIMIT = 50; + private AmazonDynamoDB amazonDynamoDB; private DynamoDB dynamoDB; private AWSCredentialProviderList credentials; @@ -555,10 +564,12 @@ public void deleteSubtree(Path path) throws IOException { innerDelete(pathToDelete, true); return null; })); + if (futures.size() > S3GUARD_DDB_SUBMITTED_TASK_LIMIT) { + // first batch done; block for completion. + waitForCompletion(futures); + } } - // await completion - CompletableFuture.allOf(futures.toArray( - new CompletableFuture[futures.size()])).join(); + waitForCompletion(futures); } /** diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md index bb09d576dcf95..2f0c5fa5500d6 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md @@ -1215,6 +1215,18 @@ sync. See [Fail on Error](#fail-on-error) for more detail. +### Error `Attempt to change a resource which is still in use: Table is being deleted` + +``` +com.amazonaws.services.dynamodbv2.model.ResourceInUseException: + Attempt to change a resource which is still in use: Table is being deleted: s + 3guard.test.testDynamoDBInitDestroy351245027 + (Service: AmazonDynamoDBv2; Status Code: 400; Error Code: ResourceInUseException;) +``` + +You have attempted to call `hadoop s3guard destroy` on a table which is already +being destroyed. + ## Other Topics For details on how to test S3Guard, see [Testing S3Guard](./testing.html#s3guard) 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 9ef8897663798..cbcada8161fd9 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 @@ -59,7 +59,6 @@ import static org.apache.hadoop.fs.s3a.Constants.ENABLE_MULTI_DELETE; import static org.apache.hadoop.fs.s3a.Constants.MAXIMUM_CONNECTIONS; import static org.apache.hadoop.fs.s3a.Constants.MAX_THREADS; -import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_DEFAULT; import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY; import static org.apache.hadoop.fs.s3a.S3ATestUtils.MetricDiff; import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume; @@ -83,6 +82,7 @@ import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.forbidden; import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.newAssumedRoleConfig; import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit; +import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletion; import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.extractUndeletedPaths; import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.removeUndeletedPaths; import static org.apache.hadoop.fs.s3a.test.ExtraAssertions.assertFileCount; @@ -96,6 +96,17 @@ * * All these test have a unique path for each run, with a roleFS having * full RW access to part of it, and R/O access to a restricted subdirectory + * + * The tests are parameterized to single/multi delete, which control which + * of the two delete mechanisms are used. + * In multi delete, in a scale test run, a significantly larger set of files + * is created and then deleted. + * This isn't done in the single delete as it is much slower and it is not + * the situation we are trying to create. + * + * This test manages to create lots of load on the s3guard prune command + * when that is tested; too many tombstone files for the test to complete. + * An attempt is made in teardown to prune the test files. */ @SuppressWarnings("ThrowableNotThrown") @RunWith(Parameterized.class) @@ -190,6 +201,8 @@ public void setup() throws Exception { S3AFileSystem fs = getFileSystem(); fs.delete(basePath, true); fs.mkdirs(destDir); + + // create the baseline assumed role assumedRoleConfig = createAssumedRoleConfig(); bindRolePolicyStatements(assumedRoleConfig, STATEMENT_S3GUARD_CLIENT, @@ -198,14 +211,16 @@ public void setup() throws Exception { .addActions(S3_PATH_RW_OPERATIONS) .addResources(directory(destDir)) ); + // the role configured to that set of restrictions roleFS = (S3AFileSystem) readOnlyDir.getFileSystem(assumedRoleConfig); + + // switch to the big set of files iff this is a multidelete run + // with -Dscale set. + // without that the DELETE calls become a key part of the bottleneck scaleTest = multiDelete && getTestPropertyBool( getConfiguration(), KEY_SCALE_TESTS_ENABLED, DEFAULT_SCALE_TESTS_ENABLED); - // switch to the big set of files iff this is a multidelete run - // with -Dscale set. - // without that the DELETE calls become a key part of the bottleneck filecount = scaleTest ? FILE_COUNT_SCALED : FILE_COUNT_NON_SCALED; @@ -277,7 +292,6 @@ protected Configuration createConfiguration() { return conf; } - /** * Create a unique path, which includes method name, * multdelete flag and a random UUID. @@ -620,15 +634,17 @@ public static List createFiles(final FileSystem fs, final int range) throws IOException { List> futures = new ArrayList<>(range); List paths = new ArrayList<>(range); - for (int i = 0; i < range; i++) { - String name = "file-" + i; - Path p = new Path(destDir, name); - paths.add(p); - futures.add(put(fs, p, name)); + try(DurationInfo ignore = + new DurationInfo(LOG, "Creating %d files", range)) { + for (int i = 0; i < range; i++) { + String name = "file-" + i; + Path p = new Path(destDir, name); + paths.add(p); + futures.add(put(fs, p, name)); + } + waitForCompletion(futures); + return paths; } - CompletableFuture.allOf(futures.toArray( - new CompletableFuture[futures.size()])).join(); - return paths; } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java index 98c1e998ed0aa..98936954e4dae 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java @@ -29,6 +29,7 @@ import com.amazonaws.services.dynamodbv2.document.DynamoDB; import com.amazonaws.services.dynamodbv2.document.Table; import com.amazonaws.services.dynamodbv2.model.ListTagsOfResourceRequest; +import com.amazonaws.services.dynamodbv2.model.ResourceInUseException; import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException; import com.amazonaws.services.dynamodbv2.model.Tag; import org.junit.Assert; @@ -270,7 +271,9 @@ public void testDynamoDBInitDestroyCycle() throws Throwable { try { table.delete(); table.waitForDelete(); - } catch (ResourceNotFoundException e) { /* Ignore */ } + } catch (ResourceNotFoundException | ResourceInUseException e) { + /* Ignore */ + } } } } From 733dc4e64d7244ed8650144be3b701564264556e Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 25 Apr 2019 21:48:59 +0100 Subject: [PATCH 07/22] HADOOP-15183. Minor code cleanup before going more aggressively into failures-during-copy Change-Id: Ida9e7424bd635fd6cc246ce65ab8d96c6fb8a958 --- .../org/apache/hadoop/fs/s3a/S3AUtils.java | 4 +- .../hadoop/fs/s3a/impl/CallableSupplier.java | 5 +- .../fs/s3a/impl/MultiObjectDeleteSupport.java | 81 ++++++++++--------- .../s3a/impl/ITestPartialRenamesDeletes.java | 24 +++--- 4 files changed, 59 insertions(+), 55 deletions(-) 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 934d798ea4c14..afc3431758413 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 @@ -87,7 +87,7 @@ import static org.apache.commons.lang3.StringUtils.isEmpty; import static org.apache.hadoop.fs.s3a.Constants.*; -import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.translateMultiObjectDeleteException; +import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.translateDeleteException; /** * Utility methods for S3A code. @@ -288,7 +288,7 @@ public static IOException translateException(@Nullable String operation, case 200: if (exception instanceof MultiObjectDeleteException) { // failure during a bulk delete - return translateMultiObjectDeleteException(message, + return translateDeleteException(message, (MultiObjectDeleteException) exception); } // other 200: FALL THROUGH 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 68b7ae6be351e..bcc8f6ebce044 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 @@ -83,7 +83,7 @@ public static CompletableFuture submit( } /** - * Wait for a list of futures to complete + * Wait for a list of futures to complete. * @param futures list of futures. * @throws IOException if one of the called futures raised an IOE. * @throws RuntimeException if one of the futures raised one. @@ -96,8 +96,7 @@ public static void waitForCompletion( } // await completion CompletableFuture all = CompletableFuture.allOf( - futures.toArray( - new CompletableFuture[futures.size()])); + futures.toArray(new CompletableFuture[0])); try(DurationInfo ignore = new DurationInfo(LOG, false, "Waiting for task completion")) { all.join(); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteSupport.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteSupport.java index 7419de71f588f..3772a78df1d54 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteSupport.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteSupport.java @@ -50,6 +50,10 @@ public final class MultiObjectDeleteSupport { private final StoreContext context; + /** + * Initiate with a store context. + * @param context store context. + */ public MultiObjectDeleteSupport(final StoreContext context) { this.context = context; } @@ -61,7 +65,7 @@ public MultiObjectDeleteSupport(final StoreContext context) { public static final String ACCESS_DENIED = "AccessDenied"; /** - * A {@code }MultiObjectDeleteException} is raised if one or more + * A {@code MultiObjectDeleteException} is raised if one or more * paths listed in a bulk DELETE operation failed. * The top-level exception is therefore just "something wasn't deleted", * but doesn't include the what or the why. @@ -72,7 +76,7 @@ public MultiObjectDeleteSupport(final StoreContext context) { * @param deleteException the delete exception. to translate * @return an IOE with more detail. */ - public static IOException translateMultiObjectDeleteException( + public static IOException translateDeleteException( final String message, final MultiObjectDeleteException deleteException) { final StringBuilder result = new StringBuilder( @@ -96,41 +100,6 @@ public static IOException translateMultiObjectDeleteException( } } - /** - * Build a list of undeleted paths from a {@code MultiObjectDeleteException}. - * Outside of unit tests, the qualifier function should be - * {@link S3AFileSystem#keyToQualifiedPath(String)}. - * @param deleteException the delete exception. - * @param qualifierFn function to qualify paths - * @return the possibly empty list of paths. - */ - public static List extractUndeletedPaths( - final MultiObjectDeleteException deleteException, - final Function qualifierFn) { - return deleteException.getErrors().stream() - .map((e) -> qualifierFn.apply(e.getKey())) - .collect(Collectors.toList()); - } - - /** - * Process a {@code MultiObjectDeleteException} by - * removing all undeleted paths from the list of paths being deleted. - * The original list is updated, and so becomes the list of successfully - * deleted paths. - * @param deleteException the delete exception. - * @param pathsBeingDeleted list of paths which were being deleted. - * This has all undeleted paths removed, leaving only those deleted. - * @return the list of undeleted entries - */ - public static List removeUndeletedPaths( - final MultiObjectDeleteException deleteException, - final Collection pathsBeingDeleted, - final Function qualifier) { - List undeleted = extractUndeletedPaths(deleteException, qualifier); - pathsBeingDeleted.removeAll(undeleted); - return undeleted; - } - /** * Process a multi object delete exception by building two paths from * the delete request: one of all deleted files, one of all undeleted values. @@ -163,10 +132,9 @@ public Pair, List> splitUndeletedKeys( /** * Process a delete failure by removing from the metastore all entries * which where deleted, as inferred from the delete failures exception - * and the original list of files to delete declares to have been delted. + * and the original list of files to delete declares to have been deleted. * @param deleteException the delete exception. * @param keysToDelete collection of keys which had been requested. - * @param qualifierFn qualifier to convert keys to paths * @return a tuple of (undeleted, deleted, failures) */ public Triple, List, List>> @@ -198,4 +166,39 @@ public Pair, List> splitUndeletedKeys( return Triple.of(undeleted, deleted, failures); } + /** + * Build a list of undeleted paths from a {@code MultiObjectDeleteException}. + * Outside of unit tests, the qualifier function should be + * {@link S3AFileSystem#keyToQualifiedPath(String)}. + * @param deleteException the delete exception. + * @param qualifierFn function to qualify paths + * @return the possibly empty list of paths. + */ + public static List extractUndeletedPaths( + final MultiObjectDeleteException deleteException, + final Function qualifierFn) { + return deleteException.getErrors().stream() + .map((e) -> qualifierFn.apply(e.getKey())) + .collect(Collectors.toList()); + } + + /** + * Process a {@code MultiObjectDeleteException} by + * removing all undeleted paths from the list of paths being deleted. + * The original list is updated, and so becomes the list of successfully + * deleted paths. + * @param deleteException the delete exception. + * @param pathsBeingDeleted list of paths which were being deleted. + * This has all undeleted paths removed, leaving only those deleted. + * @return the list of undeleted entries + */ + public static List removeUndeletedPaths( + final MultiObjectDeleteException deleteException, + final Collection pathsBeingDeleted, + final Function qualifier) { + List undeleted = extractUndeletedPaths(deleteException, qualifier); + pathsBeingDeleted.removeAll(undeleted); + return undeleted; + } + } 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 cbcada8161fd9..3980cf0814bcb 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 @@ -376,8 +376,7 @@ public void testRenameSingleFileFailsLeavingSource() throws Throwable { roleFS.delete(destDir, true); roleFS.mkdirs(destDir); // rename will fail in the delete phase - AccessDeniedException deniedException = expectRenameForbidden( - readOnlyFile, destDir); + expectRenameForbidden(readOnlyFile, destDir); // and the source file is still there @@ -513,14 +512,13 @@ public void testPartialDelete() throws Throwable { // build the set of all paths under the directory tree through // a directory listing (i.e. not getFileStatus()). // small risk of observed inconsistency here on unguarded stores. - final Set roFListing = listFilesUnderPath(readOnlyDir, true); + final Set readOnlyListing = listFilesUnderPath(readOnlyDir, true); - String directoryList = roFListing - .stream() + String directoryList = readOnlyListing.stream() .map(Path::toString) .collect(Collectors.joining(", ", "[", "]")); - Assertions.assertThat(roFListing) + Assertions.assertThat(readOnlyListing) .as("ReadOnly directory " + directoryList) .containsAll(readOnlyFiles); } @@ -583,7 +581,8 @@ private void prune(Path path) throws IOException { S3AFileSystem fs = getFileSystem(); if (fs.hasMetadataStore()) { MetadataStore store = fs.getMetadataStore(); - try(DurationInfo ignored = new DurationInfo(LOG, true, "prune %s", path)) { + try(DurationInfo ignored = + new DurationInfo(LOG, true, "prune %s", path)) { store.prune(System.currentTimeMillis(), fs.pathToKey(path)); } } @@ -598,10 +597,13 @@ private void prune(Path path) throws IOException { */ private Set listFilesUnderPath(Path path, boolean recursive) throws IOException { Set files = new TreeSet<>(); - applyLocatedFiles(getFileSystem().listFiles(path, recursive), - (status) -> { - files.add(status.getPath()); - }); + try (DurationInfo ignore = + new DurationInfo(LOG, "ls -R %s", path)) { + applyLocatedFiles(getFileSystem().listFiles(path, recursive), + (status) -> { + files.add(status.getPath()); + }); + } return files; } From 3fa888200377b9b8d076988e591751e67c479533 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 26 Apr 2019 14:49:42 +0100 Subject: [PATCH 08/22] HADOOP-15183: fix up findbugs warnings (and failure to create directory allocator) by providing a specific allocator callback. This is the purer way anyway: users of the store context should get the functionality offered by the store, rather than access to the implementation details. Note; I could have createed template FunctionWithIOE and BiFunctionWithIOE interfaces in fs.impl. Semi tempted, but trying to ramp back ambitions Change-Id: I59fb2bf66218be6318c83fc1c919e004118533ab --- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 4 +-- .../hadoop/fs/s3a/impl/StoreContext.java | 35 ++++++++++++------- .../s3a/impl/TestPartialDeleteFailures.java | 4 +-- 3 files changed, 27 insertions(+), 16 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 cfffb85219c8c..0c864111581ba 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 @@ -2384,7 +2384,6 @@ public StoreContext createStoreContext() { boundedThreadPool, executorCapacity, invoker, - directoryAllocator, getInstrumentation(), getStorageStatistics(), getInputPolicy(), @@ -2394,7 +2393,8 @@ public StoreContext createStoreContext() { this::keyToQualifiedPath, bucketLocation, useListV1, - false); + false, + (prefix, len) -> createTmpFileForWrite(prefix, len, getConf())); } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java index 654b282d572de..e16801a4f738c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java @@ -18,6 +18,8 @@ package org.apache.hadoop.fs.s3a.impl; +import java.io.File; +import java.io.IOException; import java.net.URI; import java.util.Optional; import java.util.function.Function; @@ -27,7 +29,6 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.LocalDirAllocator; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.Invoker; import org.apache.hadoop.fs.s3a.S3AInputPolicy; @@ -88,8 +89,6 @@ public class StoreContext { /** Invoker of operations. */ private final Invoker invoker; - private final LocalDirAllocator directoryAllocator; - /* Instrumentation and statistics. */ private final S3AInstrumentation instrumentation; private final S3AStorageStatistics storageStatistics; @@ -109,7 +108,6 @@ public class StoreContext { /** Is the store versioned? */ private final boolean versioned; - /** * To allow this context to be passed down to the metastore, this field * wll be null until initialized. @@ -119,6 +117,9 @@ public class StoreContext { /** Function to take a key and return a path. */ private final Function keyToPathQualifier; + /** Factory for temporary files. */ + private final TempFileFactory tempFileFactory; + /** * Instantiate. */ @@ -128,8 +129,8 @@ public StoreContext(final URI fsURI, final String username, final UserGroupInformation owner, final ListeningExecutorService executor, - final int executorCapacity, final Invoker invoker, - final LocalDirAllocator directoryAllocator, + final int executorCapacity, + final Invoker invoker, final S3AInstrumentation instrumentation, final S3AStorageStatistics storageStatistics, final S3AInputPolicy inputPolicy, @@ -139,7 +140,8 @@ public StoreContext(final URI fsURI, final Function keyToPathQualifier, final String bucketLocation, final boolean useListV1, - final boolean versioned) { + final boolean versioned, + final TempFileFactory tempFileFactory) { this.fsURI = fsURI; this.bucket = bucket; this.configuration = configuration; @@ -148,7 +150,6 @@ public StoreContext(final URI fsURI, this.executor = executor; this.executorCapacity = executorCapacity; this.invoker = invoker; - this.directoryAllocator = directoryAllocator; this.instrumentation = instrumentation; this.storageStatistics = storageStatistics; this.inputPolicy = inputPolicy; @@ -159,6 +160,7 @@ public StoreContext(final URI fsURI, this.bucketLocation = Optional.ofNullable(bucketLocation); this.useListV1 = useListV1; this.versioned = versioned; + this.tempFileFactory = tempFileFactory; } @Override @@ -194,10 +196,6 @@ public Invoker getInvoker() { return invoker; } - public LocalDirAllocator getDirectoryAllocator() { - return directoryAllocator; - } - public S3AInstrumentation getInstrumentation() { return instrumentation; } @@ -306,4 +304,17 @@ public ListeningExecutorService createThrottledExecutor() { return createThrottledExecutor(executorCapacity); } + public File createTempFile(String pathStr, long size) throws IOException { + return tempFileFactory.createTempFile(pathStr, size); + } + + /** + * The interface for temporary files. + * The standard Java 8 BiFunction cannot be used as it doesn't raise an + * IOE. + */ + @FunctionalInterface + public interface TempFileFactory { + File createTempFile(String pathStr, long size) throws IOException; + } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java index 203d255c46619..fb5296a03b130 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java @@ -204,7 +204,6 @@ StoreContext craateMockStoreContext(boolean multiDelete, "s3a-transfer-shared"), Constants.DEFAULT_EXECUTOR_CAPACITY, new Invoker(RetryPolicies.TRY_ONCE_THEN_FAIL, Invoker.LOG_EVENT), - null, new S3AInstrumentation(name), new S3AStorageStatistics(), S3AInputPolicy.Normal, @@ -215,7 +214,8 @@ StoreContext craateMockStoreContext(boolean multiDelete, TestPartialDeleteFailures::qualify, "us-west", false, - false); + false, + null); } /** From 7362257f5fee1daae01206d0b1049b845329e416 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 26 Apr 2019 14:51:34 +0100 Subject: [PATCH 09/22] HADOOP-15183: ITestMagiCommitMRJob is failing with __magic still being there in parallel auth test runs. Change the failure to incude a listing so as to work out what is happening. I don't think this is related to the rest of the work, so have made this patch isolated Change-Id: I0ca0a1e315ee1d963457249a2d147fbbd2b98901 --- .../fs/s3a/commit/magic/ITestMagicCommitMRJob.java | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitMRJob.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitMRJob.java index a9b9c2cbe1e1d..f3c4174577235 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitMRJob.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitMRJob.java @@ -18,17 +18,20 @@ package org.apache.hadoop.fs.s3a.commit.magic; +import java.io.FileNotFoundException; import java.io.IOException; import org.junit.AfterClass; import org.junit.BeforeClass; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.s3a.commit.AbstractITCommitMRJob; import org.apache.hadoop.fs.s3a.commit.files.SuccessData; import org.apache.hadoop.mapred.JobConf; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** * Full integration test for the Magic Committer. @@ -93,6 +96,13 @@ protected void applyCustomConfigOptions(JobConf conf) { @Override protected void customPostExecutionValidation(Path destPath, SuccessData successData) throws Exception { - assertPathDoesNotExist("No cleanup", new Path(destPath, MAGIC)); + Path magicDir = new Path(destPath, MAGIC); + + // if an FNFE isn't raised on getFileStatus, list out the directory + // contents + intercept(FileNotFoundException.class, () -> { + getFileSystem().getFileStatus(magicDir); + return ContractTestUtils.ls(getFileSystem(), magicDir); + }); } } From ea55c7641ce5e37dc208f4321dc755357004acc3 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 26 Apr 2019 15:46:29 +0100 Subject: [PATCH 10/22] HADOOP-15183 exploring making the copy parallelized; need to be clearer about copy and update logic first, and want to merge in the versioning stuff from Cerner first Change-Id: If167d7b1d89d0a0873708ee9e471a7b68489f040 --- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 44 ++++++++++--------- .../hadoop/fs/s3a/impl/CallableSupplier.java | 17 +++++-- .../fs/s3a/impl/MultiObjectDeleteSupport.java | 5 ++- .../hadoop/fs/s3a/impl/PathToBucketKeys.java | 22 ++++++++++ .../hadoop/fs/s3a/impl/StoreContext.java | 15 ++++--- 5 files changed, 73 insertions(+), 30 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/PathToBucketKeys.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 0c864111581ba..ea0a404a7965e 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 @@ -88,7 +88,6 @@ import org.apache.hadoop.fs.CreateFlag; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.Options; import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport; import org.apache.hadoop.fs.s3a.impl.StoreContext; @@ -146,6 +145,8 @@ 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.CallableSupplier.submit; +import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletion; import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; /** @@ -1268,21 +1269,28 @@ private boolean innerRename(Path source, Path dest) while (iterator.hasNext()) { LocatedFileStatus status = iterator.next(); long length = status.getLen(); - String key = pathToKey(status.getPath()); - if (status.isDirectory() && !key.endsWith("/")) { - key += "/"; - } + String k = pathToKey(status.getPath()); + String key = (status.isDirectory() && !k.endsWith("/")) + ? k + "/" + : k; keysToDelete .add(new DeleteObjectsRequest.KeyVersion(key)); String newDstKey = dstKey + key.substring(srcKey.length()); - copyFile(key, newDstKey, length); + Path childSrc = keyToQualifiedPath(key); + Path childDst = keyToQualifiedPath(newDstKey); + + // set up for async operation but run in sync mode initially. + // we will need to parallelize updates to metastore + // for that. + CompletableFuture copy = submit(boundedThreadPool, + () -> copySourceAndUpdateMetastore(status, key, newDstKey, + childDst); + waitForCompletion(copy); if (hasMetadataStore()) { // with a metadata store, the object entries need to be updated, // including, potentially, the ancestors - Path childSrc = keyToQualifiedPath(key); - Path childDst = keyToQualifiedPath(newDstKey); if (objectRepresentsDirectory(key, length)) { S3Guard.addMoveDir(metadataStore, srcPaths, dstMetas, childSrc, childDst, username); @@ -1323,19 +1331,13 @@ private boolean innerRename(Path source, Path dest) return true; } - /** - * Expose the superclass rename for ease of testing. - * This is inefficient as it calls getFileStatus on source and dest - * twice, but it always throws exceptions on failures, which is good. - * @param src path to be renamed - * @param dst new path after rename - * @param options rename options. - * @throws IOException failure. - */ - @VisibleForTesting - public void rename(final Path src, final Path dst, - final Options.Rename... options) throws IOException { - super.rename(src, dst, options); + private Path copySourceAndUpdateMetastore( + final LocatedFileStatus sourceStatus, + final String srcKey, + final String destKey, + final Path destPath) throws IOException { + copyFile(srcKey, destKey, sourceStatus.getLen()); + return 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 bcc8f6ebce044..761b4f587d3d8 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 @@ -95,11 +95,22 @@ public static void waitForCompletion( return; } // await completion - CompletableFuture all = CompletableFuture.allOf( - futures.toArray(new CompletableFuture[0])); + waitForCompletion(CompletableFuture.allOf( + futures.toArray(new CompletableFuture[0]))); + } + + /** + * Wait for a single of future to complete, extracting IOEs afterwards. + * @param future future to wait for. + * @throws IOException if one of the called futures raised an IOE. + * @throws RuntimeException if one of the futures raised one. + */ + public static void waitForCompletion( + final CompletableFuture future) + throws IOException { try(DurationInfo ignore = new DurationInfo(LOG, false, "Waiting for task completion")) { - all.join(); + future.join(); } catch (CancellationException e) { throw new IOException(e); } catch (CompletionException e) { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteSupport.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteSupport.java index 3772a78df1d54..8f2cd8d11f245 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteSupport.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteSupport.java @@ -28,6 +28,7 @@ import com.amazonaws.services.s3.model.DeleteObjectsRequest; import com.amazonaws.services.s3.model.MultiObjectDeleteException; +import com.google.common.annotations.VisibleForTesting; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -174,6 +175,7 @@ public Pair, List> splitUndeletedKeys( * @param qualifierFn function to qualify paths * @return the possibly empty list of paths. */ + @VisibleForTesting public static List extractUndeletedPaths( final MultiObjectDeleteException deleteException, final Function qualifierFn) { @@ -192,7 +194,8 @@ public static List extractUndeletedPaths( * This has all undeleted paths removed, leaving only those deleted. * @return the list of undeleted entries */ - public static List removeUndeletedPaths( + @VisibleForTesting + static List removeUndeletedPaths( final MultiObjectDeleteException deleteException, final Collection pathsBeingDeleted, final Function qualifier) { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/PathToBucketKeys.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/PathToBucketKeys.java new file mode 100644 index 0000000000000..a444400a6def3 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/PathToBucketKeys.java @@ -0,0 +1,22 @@ +/* + * 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; + +public class PathToBucketKeys { +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java index e16801a4f738c..b0397a3f13828 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java @@ -44,6 +44,10 @@ * components, without exposing the entire parent class. * This is eliminate explicit recursive coupling. * + * Where methods on the FS are to be invoked, they are all passed in + * via functional interfaces, so test setups can pass in mock callbacks + * instead. + * * Warning: this really is private and unstable. Do not use * outside the org.apache.hadoop.fs.s3a package. */ @@ -51,22 +55,19 @@ @InterfaceStability.Unstable public class StoreContext { - /* - * Foundational fields. - */ /** Filesystem URI. */ private final URI fsURI; /** Bucket name */ private final String bucket; - /* FS configuration after all per-bucket overrides applied. */ + /** FS configuration after all per-bucket overrides applied. */ private final Configuration configuration; /** Username. */ private final String username; - /** Principal who created the FS*/ + /** Principal who created the FS. */ private final UserGroupInformation owner; /** @@ -304,6 +305,10 @@ public ListeningExecutorService createThrottledExecutor() { return createThrottledExecutor(executorCapacity); } + public UserGroupInformation getOwner() { + return owner; + } + public File createTempFile(String pathStr, long size) throws IOException { return tempFileFactory.createTempFile(pathStr, size); } From ad5bb48c56608577a3b3c9baa4e0b8e6d2df98dd Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 1 May 2019 21:02:20 +0100 Subject: [PATCH 11/22] HADOOP-15183 S3Guard and rename failures. Pulls out the code to manage state during an S3A rename into a RenameOperation class, which each metastore gets to return on a call to initiateRenameOperations(); The current implementation of this is "just" the design originally in the S3AFileSystem.innerRename(): built up a list of destination metadata entries to create Change-Id: Ibb88f6af0a7f88cc528700942059aede86a72dea TODO: actually do either incremental updates of the store or write current state on a failure --- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 283 +++++++++++------- .../fs/s3a/impl/MultiObjectDeleteSupport.java | 49 ++- .../s3guard/DelayedUpdateRenameOperation.java | 132 ++++++++ .../fs/s3a/s3guard/DynamoDBMetadataStore.java | 8 +- .../fs/s3a/s3guard/LocalMetadataStore.java | 8 + .../hadoop/fs/s3a/s3guard/MetadataStore.java | 16 + .../fs/s3a/s3guard/NullMetadataStore.java | 39 +++ .../fs/s3a/s3guard/RenameOperation.java | 154 ++++++++++ .../fs/s3a/ITestS3AFailureHandling.java | 6 +- .../s3a/impl/ITestPartialRenamesDeletes.java | 21 +- .../s3a/impl/TestPartialDeleteFailures.java | 8 + 11 files changed, 600 insertions(+), 124 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameOperation.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RenameOperation.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 ea0a404a7965e..26cd1a9dfaf79 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 @@ -32,7 +32,6 @@ import java.util.Collections; import java.util.Date; import java.util.EnumSet; -import java.util.HashSet; import java.util.List; import java.util.Locale; import java.util.Map; @@ -82,6 +81,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.commons.lang3.tuple.Triple; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; @@ -91,6 +92,7 @@ import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport; import org.apache.hadoop.fs.s3a.impl.StoreContext; +import org.apache.hadoop.fs.s3a.s3guard.RenameOperation; import org.apache.hadoop.fs.s3a.select.InternalSelectConstants; import org.apache.hadoop.util.DurationInfo; import org.apache.hadoop.util.LambdaUtils; @@ -1215,113 +1217,139 @@ private boolean innerRename(Path source, Path dest) } } - // If we have a MetadataStore, track deletions/creations. - Collection srcPaths = null; - List dstMetas = null; - if (hasMetadataStore()) { - srcPaths = new HashSet<>(); // srcPaths need fast look up before put - dstMetas = new ArrayList<>(); - } - // TODO S3Guard HADOOP-13761: retries when source paths are not visible yet - // TODO S3Guard: performance: mark destination dirs as authoritative - - // Ok! Time to start - if (srcStatus.isFile()) { - LOG.debug("rename: renaming file {} to {}", src, dst); - long length = srcStatus.getLen(); - if (dstStatus != null && dstStatus.isDirectory()) { - String newDstKey = maybeAddTrailingSlash(dstKey); - String filename = - srcKey.substring(pathToKey(src.getParent()).length()+1); - newDstKey = newDstKey + filename; - copyFile(srcKey, newDstKey, length); - S3Guard.addMoveFile(metadataStore, srcPaths, dstMetas, src, - keyToQualifiedPath(newDstKey), length, getDefaultBlockSize(dst), - username); + // Validation completed: time to begin the operation. + // The store-specific rename operation is used to keep the store + // to date with the in-progress operation. + // for the null store, these are all no-ops. + final RenameOperation renameOperation = + metadataStore.initiateRenameOperation( + createStoreContext(), + src, dest); + try { + if (srcStatus.isFile()) { + LOG.debug("rename: renaming file {} to {}", src, dst); + long length = srcStatus.getLen(); + if (dstStatus != null && dstStatus.isDirectory()) { + String newDstKey = maybeAddTrailingSlash(dstKey); + String filename = + srcKey.substring(pathToKey(src.getParent()).length()+1); + newDstKey = newDstKey + filename; + copyFile(srcKey, newDstKey, length); + Path destPath = keyToQualifiedPath(newDstKey); + renameOperation.fileCopied( + srcStatus.getPath(), + srcStatus, + destPath, + getDefaultBlockSize(destPath), + false); + + } else { + copyFile(srcKey, dstKey, srcStatus.getLen()); + renameOperation.fileCopied(srcStatus.getPath(), + srcStatus, + dst, + getDefaultBlockSize(dst), + false); + } + innerDelete(srcStatus, false); } else { - copyFile(srcKey, dstKey, srcStatus.getLen()); - S3Guard.addMoveFile(metadataStore, srcPaths, dstMetas, src, dst, - length, getDefaultBlockSize(dst), username); - } - innerDelete(srcStatus, false); - } else { - LOG.debug("rename: renaming directory {} to {}", src, dst); + LOG.debug("rename: renaming directory {} to {}", src, dst); - // This is a directory to directory copy - dstKey = maybeAddTrailingSlash(dstKey); - srcKey = maybeAddTrailingSlash(srcKey); + // This is a directory to directory copy + dstKey = maybeAddTrailingSlash(dstKey); + srcKey = maybeAddTrailingSlash(srcKey); - //Verify dest is not a child of the source directory - if (dstKey.startsWith(srcKey)) { - throw new RenameFailedException(srcKey, dstKey, - "cannot rename a directory to a subdirectory of itself "); - } + //Verify dest is not a child of the source directory + if (dstKey.startsWith(srcKey)) { + throw new RenameFailedException(srcKey, dstKey, + "cannot rename a directory to a subdirectory of itself "); + } - List keysToDelete = new ArrayList<>(); - if (dstStatus != null && dstStatus.isEmptyDirectory() == Tristate.TRUE) { - // delete unnecessary fake directory. - keysToDelete.add(new DeleteObjectsRequest.KeyVersion(dstKey)); - } + List keysToDelete = new ArrayList<>(); + if (dstStatus != null && dstStatus.isEmptyDirectory() == Tristate.TRUE) { + // delete unnecessary fake directory. + keysToDelete.add(new DeleteObjectsRequest.KeyVersion(dstKey)); + } + + Path parentPath = keyToQualifiedPath(srcKey); + RemoteIterator iterator = listFilesAndEmptyDirectories( + parentPath, true); + while (iterator.hasNext()) { + LocatedFileStatus status = iterator.next(); + long length = status.getLen(); + String k = pathToKey(status.getPath()); + String key = (status.isDirectory() && !k.endsWith("/")) + ? k + "/" + : k; + keysToDelete + .add(new DeleteObjectsRequest.KeyVersion(key)); + String newDstKey = + dstKey + key.substring(srcKey.length()); + Path childSrc = keyToQualifiedPath(key); + Path childDst = keyToQualifiedPath(newDstKey); + + // set up for async operation but run in sync mode initially. + // we will need to parallelize updates to metastore + // for that. + CompletableFuture copy = submit(boundedThreadPool, () -> + copySourceAndUpdateMetastore(status, key, newDstKey, childDst)); + waitForCompletion(copy); - Path parentPath = keyToQualifiedPath(srcKey); - RemoteIterator iterator = listFilesAndEmptyDirectories( - parentPath, true); - while (iterator.hasNext()) { - LocatedFileStatus status = iterator.next(); - long length = status.getLen(); - String k = pathToKey(status.getPath()); - String key = (status.isDirectory() && !k.endsWith("/")) - ? k + "/" - : k; - keysToDelete - .add(new DeleteObjectsRequest.KeyVersion(key)); - String newDstKey = - dstKey + key.substring(srcKey.length()); - Path childSrc = keyToQualifiedPath(key); - Path childDst = keyToQualifiedPath(newDstKey); - - // set up for async operation but run in sync mode initially. - // we will need to parallelize updates to metastore - // for that. - CompletableFuture copy = submit(boundedThreadPool, - () -> copySourceAndUpdateMetastore(status, key, newDstKey, - childDst); - waitForCompletion(copy); - - if (hasMetadataStore()) { - // with a metadata store, the object entries need to be updated, - // including, potentially, the ancestors if (objectRepresentsDirectory(key, length)) { - S3Guard.addMoveDir(metadataStore, srcPaths, dstMetas, childSrc, - childDst, username); + renameOperation.directoryMarkerCopied(status, childDst, true); } else { - S3Guard.addMoveFile(metadataStore, srcPaths, dstMetas, childSrc, - childDst, length, getDefaultBlockSize(childDst), username); + renameOperation.fileCopied( + childSrc, + status, + childDst, + getDefaultBlockSize(childDst), + true); } - // Ancestor directories may not be listed, so we explicitly add them - S3Guard.addMoveAncestors(metadataStore, srcPaths, dstMetas, - keyToQualifiedPath(srcKey), childSrc, childDst, username); - } - if (keysToDelete.size() == MAX_ENTRIES_TO_DELETE) { - removeKeys(keysToDelete, true, false); + if (keysToDelete.size() == MAX_ENTRIES_TO_DELETE) { + List undeletedObjects = new ArrayList<>(); + try { + // remove the keys + // this does NOT update the metastore + removeKeys(keysToDelete, false, undeletedObjects); + // and clear the list. + } catch (AmazonClientException | IOException e) { + // failed, notify the rename operation. + // removeKeys will have already purged the metastore of + // all keys it has known to delete; this is just a final + // bit of housekeeping. + renameOperation.deleteFailed(keysToDelete, undeletedObjects); + // rethrow + throw e; + } + renameOperation.sourceObjectsDeleted(keysToDelete); + keysToDelete.clear(); + } } + // end of iteration -the final delete. + // again, this does not update the metastore. + removeKeys(keysToDelete, false); + renameOperation.sourceObjectsDeleted(keysToDelete); + + // We moved all the children, now move the top-level dir + // Empty directory should have been added as the object summary + renameOperation.noteSourceDirectoryMoved(); + } - removeKeys(keysToDelete, false, false); - - // We moved all the children, now move the top-level dir - // Empty directory should have been added as the object summary - if (hasMetadataStore() - && srcPaths != null - && !srcPaths.contains(src)) { - LOG.debug("To move the non-empty top-level dir src={} and dst={}", - src, dst); - S3Guard.addMoveDir(metadataStore, srcPaths, dstMetas, src, dst, - username); + } catch (IOException | AmazonClientException ex) { + // rename failed. + // update the store state to reflect this + try { + IOException ioe = renameOperation.renameFailed(ex); + throw ioe; + } catch (IOException e) { + } } - metadataStore.move(srcPaths, dstMetas); + // At this point the rename has completed. + // Tell the metastore this fact and let it complete its changes + renameOperation.complete(); if (!src.getParent().equals(dst.getParent())) { LOG.debug("source & dest parents are different; fix up dir markers"); @@ -1943,11 +1971,10 @@ public void incrementPutProgressStatistics(String key, long bytes) { /** * Delete a list of keys on a s3-backend. + * This does not update the metastore. * Retry policy: retry untranslated; delete considered idempotent. * @param keysToDelete collection of keys to delete on the s3-backend. * if empty, no request is made of the object store. - * @param clearKeys clears the keysToDelete-list after processing the list - * when set to true * @param deleteFakeDir indicates whether this is for deleting fake dirs * @throws InvalidRequestException if the request was rejected due to * a mistaken attempt to delete the root directory. @@ -1959,7 +1986,7 @@ public void incrementPutProgressStatistics(String key, long bytes) { */ @Retries.RetryRaw private void removeKeysS3(List keysToDelete, - boolean clearKeys, boolean deleteFakeDir) + boolean deleteFakeDir) throws MultiObjectDeleteException, AmazonClientException, IOException { if (keysToDelete.isEmpty()) { @@ -1989,9 +2016,6 @@ private void removeKeysS3(List keysToDelete, throw ex; } noteDeleted(keysToDelete.size(), deleteFakeDir); - if (clearKeys) { - keysToDelete.clear(); - } } /** @@ -2008,14 +2032,13 @@ private void noteDeleted(final int count, final boolean deleteFakeDir) { } /** - * Invoke {@link #removeKeysS3(List, boolean, boolean)} with handling of + * Invoke {@link #removeKeysS3(List, boolean)} with handling of * {@code MultiObjectDeleteException} in which S3Guard is updated with all * deleted entries, before the exception is rethrown. * + * If an exception is not raised. the metastore is not updated. * @param keysToDelete collection of keys to delete on the s3-backend. * if empty, no request is made of the object store. - * @param clearKeys clears the keysToDelete-list after processing the list - * when set to true * @param deleteFakeDir indicates whether this is for deleting fake dirs * @throws InvalidRequestException if the request was rejected due to * a mistaken attempt to delete the root directory. @@ -2028,18 +2051,57 @@ private void noteDeleted(final int count, final boolean deleteFakeDir) { @Retries.RetryMixed void removeKeys( final List keysToDelete, - final boolean clearKeys, final boolean deleteFakeDir) throws MultiObjectDeleteException, AmazonClientException, IOException { + removeKeys(keysToDelete, deleteFakeDir, new ArrayList<>()); + } + + /** + * Invoke {@link #removeKeysS3(List, boolean)} with handling of + * {@code MultiObjectDeleteException} in which S3Guard is updated with all + * deleted entries, before the exception is rethrown. + * + * @param keysToDelete collection of keys to delete on the s3-backend. + * if empty, no request is made of the object store. + * @param deleteFakeDir indicates whether this is for deleting fake dirs + * @param undeletedObjectsOnFailure List which will be built up of all + * files that were not deleted. This happens even as an exception + * is raised. + * @throws InvalidRequestException if the request was rejected due to + * a mistaken attempt to delete the root directory. + * @throws MultiObjectDeleteException one or more of the keys could not + * be deleted in a multiple object delete operation. + * @throws AmazonClientException amazon-layer failure. + * @throws IOException other IO Exception. + */ + @VisibleForTesting + @Retries.RetryMixed + void removeKeys( + final List keysToDelete, + final boolean deleteFakeDir, + final List undeletedObjectsOnFailure) + throws MultiObjectDeleteException, AmazonClientException, + IOException { + undeletedObjectsOnFailure.clear(); try { - removeKeysS3(keysToDelete, clearKeys, deleteFakeDir); + removeKeysS3(keysToDelete, deleteFakeDir); } catch (MultiObjectDeleteException ex) { LOG.debug("Partial delete failure"); // what to do if an IOE was raised? Given an exception was being // raised anyway, and the failures are logged, do nothing. - new MultiObjectDeleteSupport(createStoreContext()) - .processDeleteFailure(ex, keysToDelete); + Triple, List, List>> results = + new MultiObjectDeleteSupport( + createStoreContext()) + .processDeleteFailure(ex, keysToDelete); + undeletedObjectsOnFailure.addAll(results.getMiddle()); + throw ex; + } catch (AmazonClientException | IOException ex) { + List paths = new MultiObjectDeleteSupport( + createStoreContext()) + .processDeleteFailureGenericException(ex, keysToDelete); + // other failures. Assume nothing was deleted + undeletedObjectsOnFailure.addAll(paths); throw ex; } } @@ -2137,14 +2199,15 @@ private boolean innerDelete(S3AFileStatus status, boolean recursive) LOG.debug("Got object to delete {}", summary.getKey()); if (keys.size() == MAX_ENTRIES_TO_DELETE) { - removeKeys(keys, true, false); + removeKeys(keys, false); + keys.clear(); } } if (objects.isTruncated()) { objects = continueListObjects(request, objects); } else { - removeKeys(keys, false, false); + removeKeys(keys, false); break; } } @@ -3221,7 +3284,7 @@ private void deleteUnnecessaryFakeDirectories(Path path) { path = path.getParent(); } try { - removeKeys(keysToRemove, false, true); + removeKeys(keysToRemove, true); } catch(AmazonClientException | IOException e) { instrumentation.errorIgnored(); if (LOG.isDebugEnabled()) { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteSupport.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteSupport.java index 8f2cd8d11f245..e6d2417c21aac 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteSupport.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteSupport.java @@ -117,19 +117,43 @@ public Pair, List> splitUndeletedKeys( keysToDelete.size(), deleteException.getErrors().size(), deleteException.getDeletedObjects().size()); - Function qualifier = context.getKeyToPathQualifier(); // convert the collection of keys being deleted into paths - final List pathsBeingDeleted = keysToDelete.stream() - .map((keyVersion) -> qualifier.apply(keyVersion.getKey())) - .collect(Collectors.toList()); + final List pathsBeingDeleted = keysToPaths(keysToDelete); // Take this is list of paths // extract all undeleted entries contained in the exception and // then removes them from the original list. List undeleted = removeUndeletedPaths(deleteException, pathsBeingDeleted, - qualifier); + context.getKeyToPathQualifier()); return Pair.of(undeleted, pathsBeingDeleted); } + /** + * Given a list of delete requests, convert them all to paths. + * @param keysToDelete list of keys for the delete operation. + * @return the paths. + */ + public List keysToPaths( + final Collection keysToDelete) { + Function qualifier + = context.getKeyToPathQualifier(); + return convertToPaths(keysToDelete, qualifier); + } + + /** + * Given a list of delete requests, convert them all to paths. + * @param keysToDelete list of keys for the delete operation. + * @param qualifier path qualifier + * @return the paths. + */ + public static List convertToPaths( + final Collection keysToDelete, + final Function qualifier) { + return keysToDelete.stream() + .map((keyVersion) -> + qualifier.apply(keyVersion.getKey())) + .collect(Collectors.toList()); + } + /** * Process a delete failure by removing from the metastore all entries * which where deleted, as inferred from the delete failures exception @@ -145,8 +169,8 @@ public Pair, List> splitUndeletedKeys( final MetadataStore metadataStore = checkNotNull(context.getMetadataStore(), "context metadatastore"); final List> failures = new ArrayList<>(); - final Pair, List> outcome = splitUndeletedKeys( - deleteException, keysToDelete); + final Pair, List> outcome = + splitUndeletedKeys(deleteException, keysToDelete); List deleted = outcome.getRight(); List undeleted = outcome.getLeft(); // delete the paths but recover @@ -204,4 +228,15 @@ static List removeUndeletedPaths( return undeleted; } + /** + * A delete operation failed. + * Currently just returns the list of all paths. + * @param ex exception. + * @param keysToDelete the keys which were being deleted. + * @return all paths which were not deleted. + */ + public List processDeleteFailureGenericException(Exception ex, + final List keysToDelete) { + return keysToPaths(keysToDelete); + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameOperation.java new file mode 100644 index 0000000000000..2cd1f99f90684 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameOperation.java @@ -0,0 +1,132 @@ +/* + * 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.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; + +import com.amazonaws.services.s3.model.DeleteObjectsRequest; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.impl.StoreContext; + +/** + * This is the rename updating strategy originally used: + * a collection of source paths and a list of destinations are created, + * then updated at the end (possibly slow) + */ +public class DelayedUpdateRenameOperation extends RenameOperation { + + private final StoreContext storeContext; + + private final MetadataStore metadataStore; + private final Collection srcPaths = new HashSet<>(); + + private final List dstMetas = new ArrayList<>(); + + private final List deletedKeys = new ArrayList<>(); + + public DelayedUpdateRenameOperation( + final StoreContext storeContext, + final MetadataStore metadataStore, + final Path sourceRoot, + final Path dest) { + super(sourceRoot, dest, storeContext.getUsername()); + this.storeContext = storeContext; + this.metadataStore = metadataStore; + } + + @Override + public void fileCopied( + final Path childSource, + final FileStatus sourceStatus, + final Path destPath, + final long blockSize, + final boolean addAncestors + ) throws IOException { + S3Guard.addMoveFile(metadataStore, + srcPaths, + dstMetas, + childSource, + destPath, + sourceStatus.getLen(), + blockSize, + getOwner()); + // Ancestor directories may not be listed, so we explicitly add them + if (addAncestors) { + S3Guard.addMoveAncestors(metadataStore, + srcPaths, + dstMetas, + getSourceRoot(), + sourceStatus.getPath(), + destPath, + getOwner()); + } + } + + @Override + public void directoryMarkerCopied(final FileStatus sourceStatus, + final Path destPath, + final boolean addAncestors) throws IOException { + S3Guard.addMoveDir(metadataStore, srcPaths, dstMetas, + sourceStatus.getPath(), + destPath, getOwner()); + // Ancestor directories may not be listed, so we explicitly add them + if (addAncestors) { + S3Guard.addMoveAncestors(metadataStore, + srcPaths, + dstMetas, + getSourceRoot(), + sourceStatus.getPath(), + destPath, + getOwner()); + } + } + + @Override + public void noteSourceDirectoryMoved() throws IOException { + if (!srcPaths.contains(getSourceRoot())) { + S3Guard.addMoveDir(metadataStore, srcPaths, dstMetas, + getSourceRoot(), + getDest(), getOwner()); + } + } + + @Override + public void sourceObjectsDeleted( + final List keys) throws IOException { + // convert to paths. + deletedKeys.addAll(keys); + } + + @Override + public void complete() throws IOException { + metadataStore.move(srcPaths, dstMetas); + } + + @Override + public IOException renameFailed(final Exception ex) throws IOException { + super.renameFailed(ex); + return null; + } +} 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 1c918096f252a..296712b90be57 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 @@ -84,7 +84,6 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.impl.WrappedIOException; import org.apache.hadoop.fs.s3a.AWSClientIOException; import org.apache.hadoop.fs.s3a.AWSCredentialProviderList; import org.apache.hadoop.fs.s3a.AWSServiceThrottledException; @@ -1763,6 +1762,13 @@ public Invoker getInvoker() { return invoker; } + @Override + public RenameOperation initiateRenameOperation(final StoreContext storeContext, + final Path source, + final Path dest) throws IOException { + return new DelayedUpdateRenameOperation(storeContext, this, source, dest); + } + /** * Take an {@code IllegalArgumentException} raised by a DDB operation * and if it contains an inner SDK exception, unwrap it. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java index b8f9635dcd283..e92ddfef8f87c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java @@ -29,6 +29,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.Tristate; +import org.apache.hadoop.fs.s3a.impl.StoreContext; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -530,4 +532,10 @@ DirListingMetadata getDirListingMeta(Path p){ } } + @Override + public RenameOperation initiateRenameOperation(final StoreContext storeContext, + final Path source, + final Path dest) throws IOException { + return new DelayedUpdateRenameOperation(storeContext, this, source, dest); + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java index 746fd82950b27..2644790aaf29b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java @@ -30,6 +30,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.Retries.RetryTranslated; +import org.apache.hadoop.fs.s3a.impl.StoreContext; /** * {@code MetadataStore} defines the set of operations that any metadata store @@ -252,4 +253,19 @@ void prune(long modTime, String keyPrefix) * @throws IOException if there is an error */ void updateParameters(Map parameters) throws IOException; + + /** + * Start a rename operation. + * + * @param storeContext store context. + * @param source source path + * @param dest destination path. + * @return the rename operation to update + * @throws IOException Failure. + */ + RenameOperation initiateRenameOperation( + StoreContext storeContext, + Path source, + Path dest) + throws IOException; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java index 04704e7ea73d7..46b525e1f1432 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java @@ -19,8 +19,10 @@ package org.apache.hadoop.fs.s3a.s3guard; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.impl.StoreContext; import java.io.IOException; import java.util.Collection; @@ -120,4 +122,41 @@ public Map getDiagnostics() throws IOException { public void updateParameters(Map parameters) throws IOException { } + + @Override + public RenameOperation initiateRenameOperation(final StoreContext storeContext, + final Path source, + final Path dest) + throws IOException { + return new NullRenameOperation(source, dest, storeContext.getUsername()); + } + + private static class NullRenameOperation extends RenameOperation { + + public NullRenameOperation(final Path source, + final Path dest, + final String owner) { + super(source, dest, owner); + } + + @Override + public void fileCopied(final Path childSource, + final FileStatus sourceStatus, + final Path destPath, + final long blockSize, + final boolean addAncestors) throws IOException { + + } + + @Override + public void directoryMarkerCopied(final FileStatus sourceStatus, + final Path destPath, + final boolean addAncestors) throws IOException { + } + + @Override + public void complete() throws IOException { + + } + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RenameOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RenameOperation.java new file mode 100644 index 0000000000000..c9ad56e36e17b --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RenameOperation.java @@ -0,0 +1,154 @@ +/* + * 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.Closeable; +import java.io.IOException; +import java.util.List; + +import com.amazonaws.services.s3.model.DeleteObjectsRequest; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; + +/** + * The base class for a rename operation. + */ +public abstract class RenameOperation implements Closeable { + + + /** source path. */ + private final Path sourceRoot; + + /** destination path. */ + private final Path dest; + + private final String owner; + + /** + * constructor. + * @param sourceRoot source path. + * @param dest destination path. + * @param owner + */ + + public RenameOperation( + final Path sourceRoot, + final Path dest, + final String owner) { + this.sourceRoot = sourceRoot; + this.dest = dest; + this.owner = owner; + } + + public Path getSourceRoot() { + return sourceRoot; + } + + public Path getDest() { + return dest; + } + + public String getOwner() { + return owner; + } + + @Override + public void close() throws IOException { + + } + + /** + * A file has been copied. + * + * @param childSource + * @param sourceStatus status of source. + * @param destPath destination path. + * @param blockSize block size. + * @param addAncestors should ancestors be added? + * @throws IOException failure. + */ + public abstract void fileCopied( + final Path childSource, FileStatus sourceStatus, + Path destPath, + long blockSize, + boolean addAncestors) throws IOException; + + /** + * A directory marker has been copied. + * @param sourceStatus status of source. + * @param destPath destination path. + * @param addAncestors should ancestors be added? + * @throws IOException failure. + */ + public abstract void directoryMarkerCopied( + FileStatus sourceStatus, + Path destPath, + boolean addAncestors) throws IOException; + + /** + * The delete failed. + * By the time this is called, the metastore will already have + * been updated with the results of any partial delete failure, + * such that all files known to have been deleted will have been + * removed. + * @param undeletedObjects list of objects which were not deleted. + */ + public void deleteFailed( + final List keysToDelete, + final List undeletedObjects) { + + } + + /** + * Top level directory move. + * @throws IOException on failure + */ + public void noteSourceDirectoryMoved() throws IOException { + + } + + /** + * Note that source objects have been deleted. + * The metastore will already have been updated. + * @param keys keys of objects deleted. + */ + public void sourceObjectsDeleted( + final List keys) throws IOException { + } + + + /** + * Complete the operation. + * @throws IOException failure. + */ + public abstract void complete() throws IOException ; + + /** + * Rename has failed. + * The metastore now needs to be updated with its current state + * even though the operation is incomplete. + * @throws IOException failure. + */ + public IOException renameFailed(Exception ex) throws IOException { + + + return null; + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java index 660b62cdc015b..f708012570e2c 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java @@ -77,7 +77,7 @@ public void testMultiObjectDeleteNoFile() throws Throwable { private void removeKeys(S3AFileSystem fileSystem, String... keys) throws IOException { - fileSystem.removeKeys(buildDeleteRequest(keys), false, false); + fileSystem.removeKeys(buildDeleteRequest(keys), false); } private List buildDeleteRequest( @@ -125,7 +125,7 @@ public void testMultiObjectDeleteNoPermissions() throws Throwable { }); MultiObjectDeleteException ex = intercept( MultiObjectDeleteException.class, - () -> fs.removeKeys(keys, false, false)); + () -> fs.removeKeys(keys, false)); final List undeleted = extractUndeletedPaths(ex, fs::keyToQualifiedPath); @@ -160,7 +160,7 @@ public void testMultiObjectDeleteMissingEntriesSucceeds() throws Throwable { S3AFileSystem fs = getFileSystem(); List keys = keysToDelete( Lists.newArrayList(new Path(base, "1"), new Path(base, "2"))); - fs.removeKeys(keys, false, false); + fs.removeKeys(keys, false); } private String join(final Iterable iterable) { 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 3980cf0814bcb..bbd3747d6e9f2 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 @@ -52,6 +52,7 @@ import org.apache.hadoop.util.BlockingThreadPoolExecutorService; import org.apache.hadoop.util.DurationInfo; +import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathsDoNotExist; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertRenameOutcome; import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; @@ -177,7 +178,7 @@ public class ITestPartialRenamesDeletes extends AbstractS3ATestBase { * * @return a list of parameter tuples. */ - @Parameterized.Parameters + @Parameterized.Parameters(name = "bulk delete={0}") public static Collection params() { return Arrays.asList(new Object[][]{ {false}, @@ -445,6 +446,14 @@ public void testRenameFilesetFailsLeavingSourceUnchanged() throws Throwable { deniedException); assertFileCount("files in the source directory", roleFS, readOnlyDir, (long) filecount); + // now lets look at the destination. + // even with S3Guard on, we expect the destination to match that of our + // the remote state. + // the test will exist + assertIsDirectory(destDir); + assertFileCount("files in the source directory", roleFS, + destDir, (long) filecount); + } /** @@ -550,8 +559,14 @@ private AccessDeniedException expectRenameForbidden(Path src, Path dest) "Renaming " + src + " to " + dest, "", () -> { - roleFS.rename(src, dest); - return ContractTestUtils.ls(getFileSystem(), src.getParent()); + boolean result = roleFS.rename(src, dest); + LOG.error("Rename should have been forbidden but returned {}", + result); + LOG.error("Source directory:\n{}", + ContractTestUtils.ls(getFileSystem(), src.getParent())); + LOG.error("Destination directory:\n{}", + ContractTestUtils.ls(getFileSystem(), src.getParent())); + return "Rename unexpectedly returned " + result; }); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java index fb5296a03b130..68f321f9b7886 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java @@ -50,6 +50,7 @@ import org.apache.hadoop.fs.s3a.s3guard.DirListingMetadata; import org.apache.hadoop.fs.s3a.s3guard.MetadataStore; import org.apache.hadoop.fs.s3a.s3guard.PathMetadata; +import org.apache.hadoop.fs.s3a.s3guard.RenameOperation; import org.apache.hadoop.io.retry.RetryPolicies; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.BlockingThreadPoolExecutorService; @@ -323,6 +324,13 @@ public List getDeleted() { public List getCreated() { return created; } + + @Override + public RenameOperation initiateRenameOperation(final StoreContext storeContext, + final Path source, + final Path dest) throws IOException { + throw new UnsupportedOperationException("unsupported"); + } } } From db463aa5dee208217ec2d1ab9badd6dfd251ab58 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 2 May 2019 23:38:05 +0100 Subject: [PATCH 12/22] HADOOP-15183: copies are parallelised This schedules copies through the executor up to ten at a time (random choice), doing a join() to wait for completion after a batch of ten, before issuing a builk delete, and after an exception is thrown. That's the trouble with the parallel runs see: if one fails then you still have to wait for the active operation to reach some state before escalating. I've moved the updating of the operation state into the submitted closures. This still doesn't handle making the store consistent, but it reflects the structure I want the rename to work, now I'm going to add a new operation implementation which does incremental work. `ITestPartialRenamesDeletes.testRenameFilesetFailsLeavingSourceUnchanged` still fails. Added the empty test cases for the conditions I now want to create, moving beyond delete() failures into failures of the copy process (Write, read) to cover them too. Once the current test works. Copy code changes + some DurationInfo uses @ debug to log how long things take + propagates storage class from source. + getObjectMetadata now retries using the s3guard invoker if non-null, standard invoker otherwise. This is to handle failures and the case that the object to rename isn't there yet. Thoughts * maybe: issuing individual DELETE calls in that operation if single delete is enabled. This will eliminate the big paused delete phases. But do I need to care about this? It's only needed for working with non-AWS S3 impls, and for them I'd expect copy operations to be O(1) so while DELETE is slow, copy will be so fast that it'll be unimportant except in test cases with thousands of small files. * maybe: when we know files are 0 bytes long, rather than copy just do a 0 byte PUT. (but this loses the headers &c) * we may want to increase the time to wait in the S3Guard invoker to stabilise, making a separate option, and add 412 as retryable. * the page size for a multidelete is fixed at the AWS limit of 1000. This means we aren't currently testing the paged operation. Proposed: fs.s3a.internal. property to make this smaller for testing only (not in Constants, not in docs, except in testing.md) * the existing huge file test does a file rename. It should be changed to do a dir rename. Change-Id: Icfec827b001ec4546491b459d7ccc6f0f1a45706 --- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 220 ++++++++++++------ .../fs/s3a/impl/FunctionsRaisingIOE.java | 52 +++++ .../s3guard/DelayedUpdateRenameOperation.java | 30 ++- .../fs/s3a/s3guard/DynamoDBMetadataStore.java | 2 +- .../fs/s3a/s3guard/LocalMetadataStore.java | 2 +- .../hadoop/fs/s3a/s3guard/MetadataStore.java | 3 + .../fs/s3a/s3guard/NullMetadataStore.java | 8 +- .../fs/s3a/s3guard/RenameOperation.java | 89 +++++-- .../s3a/impl/ITestPartialRenamesDeletes.java | 38 ++- .../s3a/impl/TestPartialDeleteFailures.java | 3 +- 10 files changed, 326 insertions(+), 121 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/FunctionsRaisingIOE.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 26cd1a9dfaf79..431ffc0357b30 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 @@ -90,6 +90,7 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; +import org.apache.hadoop.fs.s3a.impl.FunctionsRaisingIOE; import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport; import org.apache.hadoop.fs.s3a.impl.StoreContext; import org.apache.hadoop.fs.s3a.s3guard.RenameOperation; @@ -1224,34 +1225,44 @@ private boolean innerRename(Path source, Path dest) final RenameOperation renameOperation = metadataStore.initiateRenameOperation( createStoreContext(), - src, dest); + src, srcStatus, dest); + int renameParallelLimit = 10; + final List> activeCopies = + new ArrayList<>(renameParallelLimit); + // aggregate operation to wait for the copies to complete then reset + // the list. + final FunctionsRaisingIOE.FunctionRaisingIOE + completeActiveCopies = (String reason) -> { + LOG.debug("Waiting for {} active copies to complete during {}", + activeCopies.size(), reason); + waitForCompletion(activeCopies); + activeCopies.clear(); + return null; + }; + try { if (srcStatus.isFile()) { - LOG.debug("rename: renaming file {} to {}", src, dst); - long length = srcStatus.getLen(); + Path copyDestinationPath = dst; + String copyDestinationKey = dstKey; if (dstStatus != null && dstStatus.isDirectory()) { + // destination is a directory: build the final destination underneath String newDstKey = maybeAddTrailingSlash(dstKey); String filename = srcKey.substring(pathToKey(src.getParent()).length()+1); newDstKey = newDstKey + filename; - copyFile(srcKey, newDstKey, length); - Path destPath = keyToQualifiedPath(newDstKey); - renameOperation.fileCopied( - srcStatus.getPath(), - srcStatus, - destPath, - getDefaultBlockSize(destPath), - false); - - } else { - copyFile(srcKey, dstKey, srcStatus.getLen()); - renameOperation.fileCopied(srcStatus.getPath(), - srcStatus, - dst, - getDefaultBlockSize(dst), - false); + copyDestinationKey = newDstKey; + copyDestinationPath = keyToQualifiedPath(newDstKey); } - innerDelete(srcStatus, false); + // destination either does not exist or is a file to overwrite. + LOG.debug("rename: renaming file {} to {}", src, copyDestinationPath); + copySourceAndUpdateRenameOperation(renameOperation, + src, srcKey, srcStatus, + copyDestinationPath, copyDestinationKey, + false); + // delete the source + deleteObjectAtPath(src, srcKey, true); + // TODO: renameOperation.sourceObjectsDeleted(keysToDelete); + } else { LOG.debug("rename: renaming directory {} to {}", src, dst); @@ -1276,7 +1287,6 @@ private boolean innerRename(Path source, Path dest) parentPath, true); while (iterator.hasNext()) { LocatedFileStatus status = iterator.next(); - long length = status.getLen(); String k = pathToKey(status.getPath()); String key = (status.isDirectory() && !k.endsWith("/")) ? k + "/" @@ -1285,71 +1295,60 @@ private boolean innerRename(Path source, Path dest) .add(new DeleteObjectsRequest.KeyVersion(key)); String newDstKey = dstKey + key.substring(srcKey.length()); - Path childSrc = keyToQualifiedPath(key); - Path childDst = keyToQualifiedPath(newDstKey); + Path childSourcePath = keyToQualifiedPath(key); + Path childDestPath = keyToQualifiedPath(newDstKey); // set up for async operation but run in sync mode initially. // we will need to parallelize updates to metastore // for that. CompletableFuture copy = submit(boundedThreadPool, () -> - copySourceAndUpdateMetastore(status, key, newDstKey, childDst)); - waitForCompletion(copy); - - if (objectRepresentsDirectory(key, length)) { - renameOperation.directoryMarkerCopied(status, childDst, true); - } else { - renameOperation.fileCopied( - childSrc, - status, - childDst, - getDefaultBlockSize(childDst), - true); + copySourceAndUpdateRenameOperation(renameOperation, + childSourcePath, key, status, + childDestPath, newDstKey, + true)); + activeCopies.add(copy); + if (activeCopies.size() == renameParallelLimit) { + LOG.debug("Waiting for active copies to complete"); + waitForCompletion(activeCopies); + activeCopies.clear(); } - if (keysToDelete.size() == MAX_ENTRIES_TO_DELETE) { - List undeletedObjects = new ArrayList<>(); - try { - // remove the keys - // this does NOT update the metastore - removeKeys(keysToDelete, false, undeletedObjects); - // and clear the list. - } catch (AmazonClientException | IOException e) { - // failed, notify the rename operation. - // removeKeys will have already purged the metastore of - // all keys it has known to delete; this is just a final - // bit of housekeeping. - renameOperation.deleteFailed(keysToDelete, undeletedObjects); - // rethrow - throw e; - } - renameOperation.sourceObjectsDeleted(keysToDelete); + // time to queue a delete. + // first wait for the copies in progress to finish + // (deleting a file mid-copy would not be good) + completeActiveCopies.apply("before a delete"); + removeSourceObjects(renameOperation, keysToDelete); keysToDelete.clear(); } } + // end of iteration await final set of copies + completeActiveCopies.apply("final copies"); + // end of iteration -the final delete. - // again, this does not update the metastore. - removeKeys(keysToDelete, false); - renameOperation.sourceObjectsDeleted(keysToDelete); + // This will notify the renameOperation that these objects + // have been deleted. + removeSourceObjects(renameOperation, keysToDelete); // We moved all the children, now move the top-level dir // Empty directory should have been added as the object summary renameOperation.noteSourceDirectoryMoved(); - } - } catch (IOException | AmazonClientException ex) { + } catch (AmazonClientException | IOException ex) { // rename failed. - // update the store state to reflect this + // block for all ongoing copies to complete try { - IOException ioe = renameOperation.renameFailed(ex); - throw ioe; + completeActiveCopies.apply("failure handling"); } catch (IOException e) { - + LOG.warn("While completing all active copies", e); } + + // update the store state to reflect this + throw renameOperation.renameFailed(ex); } - // At this point the rename has completed. + // At this point the rename has completed in the S3 store. // Tell the metastore this fact and let it complete its changes - renameOperation.complete(); + renameOperation.completeRename(); if (!src.getParent().equals(dst.getParent())) { LOG.debug("source & dest parents are different; fix up dir markers"); @@ -1359,12 +1358,71 @@ private boolean innerRename(Path source, Path dest) return true; } - private Path copySourceAndUpdateMetastore( - final LocatedFileStatus sourceStatus, + /** + * Remove source objects + * @param renameOperation operation to update. + * @param keysToDelete list of keys to delete + * @throws IOException failure + */ + @Retries.RetryMixed + private void removeSourceObjects(final RenameOperation renameOperation, + final List keysToDelete) + throws IOException { + List undeletedObjects = new ArrayList<>(); + try { + // remove the keys + // this does will update the metastore on a failure, but on + // a successful operation leaves the store as is. + removeKeys(keysToDelete, false, undeletedObjects); + // and clear the list. + } catch (AmazonClientException | IOException e) { + // failed, notify the rename operation. + // removeKeys will have already purged the metastore of + // all keys it has known to delete; this is just a final + // bit of housekeeping and a chance to tune exception + // reporting + throw renameOperation.deleteFailed(e, keysToDelete, undeletedObjects); + } + renameOperation.sourceObjectsDeleted(keysToDelete); + } + + /** + * This invoked to copy a file or directory marker then update the + * rename operation on success. + * It may be called in its own thread. + * @param renameOperation operation to update + * @param sourcePath source path of the copy; may have a trailing / on it. + * @param srcKey source key + * @param sourceStatus status of the source object + * @param destPath destination as a qualified path. + * @param destKey destination key + * @param addAncestors should ancestors be added to the metastore? + * @return the destination path. + * @throws IOException failure + */ + @Retries.RetryTranslated + private Path copySourceAndUpdateRenameOperation( + final RenameOperation renameOperation, + final Path sourcePath, final String srcKey, + final FileStatus sourceStatus, + final Path destPath, final String destKey, - final Path destPath) throws IOException { - copyFile(srcKey, destKey, sourceStatus.getLen()); + final boolean addAncestors) throws IOException { + copyFile(srcKey, destKey, sourceStatus, sourceStatus.getLen()); + if (objectRepresentsDirectory(srcKey, sourceStatus.getLen())) { + renameOperation.directoryMarkerCopied( + sourceStatus, + destPath, + addAncestors); + } else { + renameOperation.fileCopied( + sourcePath, + sourceStatus, + destPath, + getDefaultBlockSize(destPath), + addAncestors); + } return destPath; } @@ -1691,9 +1749,9 @@ protected void deleteObject(String key) * @param key key of entry * @param isFile is the path a file (used for instrumentation only) * @throws AmazonClientException problems working with S3 - * @throws IOException IO failure + * @throws IOException IO failure in the metastore */ - @Retries.RetryRaw + @Retries.RetryMixed void deleteObjectAtPath(Path f, String key, boolean isFile) throws AmazonClientException, IOException { if (isFile) { @@ -2084,7 +2142,7 @@ void removeKeys( throws MultiObjectDeleteException, AmazonClientException, IOException { undeletedObjectsOnFailure.clear(); - try { + try(DurationInfo ignored = new DurationInfo(LOG, false, "Deleting")) { removeKeysS3(keysToDelete, deleteFakeDir); } catch (MultiObjectDeleteException ex) { LOG.debug("Partial delete failure"); @@ -2199,6 +2257,7 @@ private boolean innerDelete(S3AFileStatus status, boolean recursive) LOG.debug("Got object to delete {}", summary.getKey()); if (keys.size() == MAX_ENTRIES_TO_DELETE) { + // delete a single page of keys removeKeys(keys, false); keys.clear(); } @@ -2207,6 +2266,7 @@ private boolean innerDelete(S3AFileStatus status, boolean recursive) if (objects.isTruncated()) { objects = continueListObjects(request, objects); } else { + // there is no more data: delete the final set of entries. removeKeys(keys, false); break; } @@ -3088,13 +3148,16 @@ public List listAWSPolicyRules( * Callers must implement. * @param srcKey source object path * @param dstKey destination object path + * @param sourceStatus * @param size object size - * @throws AmazonClientException on failures inside the AWS SDK * @throws InterruptedIOException the operation was interrupted * @throws IOException Other IO problems */ - @Retries.RetryMixed - private void copyFile(String srcKey, String dstKey, long size) + @Retries.OnceTranslated + private void copyFile(String srcKey, + String dstKey, + final FileStatus sourceStatus, + long size) throws IOException, InterruptedIOException { LOG.debug("copyFile {} -> {} ", srcKey, dstKey); @@ -3108,9 +3171,16 @@ private void copyFile(String srcKey, String dstKey, long size) } }; + // get the object header, handling the possibility that a + // newly created file is not yet present. + final Invoker inv = s3guardInvoker != null ? s3guardInvoker : invoker; + final ObjectMetadata srcom = inv.retry("HEAD", srcKey, true, + () -> getObjectMetadata(srcKey)); + + // there is no retry logic here on the expectation that the transfer + // manager is doing the work once("copyFile(" + srcKey + ", " + dstKey + ")", srcKey, () -> { - ObjectMetadata srcom = getObjectMetadata(srcKey); ObjectMetadata dstom = cloneObjectMetadata(srcom); setOptionalObjectMetadata(dstom); CopyObjectRequest copyObjectRequest = @@ -3118,6 +3188,8 @@ private void copyFile(String srcKey, String dstKey, long size) setOptionalCopyObjectRequestParameters(copyObjectRequest); copyObjectRequest.setCannedAccessControlList(cannedACL); copyObjectRequest.setNewObjectMetadata(dstom); + Optional.ofNullable(srcom.getStorageClass()) + .ifPresent(copyObjectRequest::setStorageClass); Copy copy = transfers.copy(copyObjectRequest); copy.addProgressListener(progressListener); try { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/FunctionsRaisingIOE.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/FunctionsRaisingIOE.java new file mode 100644 index 0000000000000..3755a64013fa8 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/FunctionsRaisingIOE.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl; + +import java.io.IOException; + +/** + * Function of arity 1 which may raise an IOException. + * @param + * @param

      + */ + + +public final class FunctionsRaisingIOE { + + private FunctionsRaisingIOE() { + } + + /** + * Function of arity 1 which may raise an IOException. + * @param + * @param

      + */ + @FunctionalInterface + public interface FunctionRaisingIOE { + + T apply(P p) throws IOException; + } + + @FunctionalInterface + public interface CallableRaisingIOE { + + T apply() throws IOException; + + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameOperation.java index 2cd1f99f90684..dc9b50d2e7506 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameOperation.java @@ -24,6 +24,7 @@ import java.util.HashSet; import java.util.List; +import com.amazonaws.SdkBaseException; import com.amazonaws.services.s3.model.DeleteObjectsRequest; import org.apache.hadoop.fs.FileStatus; @@ -33,7 +34,7 @@ /** * This is the rename updating strategy originally used: * a collection of source paths and a list of destinations are created, - * then updated at the end (possibly slow) + * then updated at the end (possibly slow). */ public class DelayedUpdateRenameOperation extends RenameOperation { @@ -57,8 +58,8 @@ public DelayedUpdateRenameOperation( } @Override - public void fileCopied( - final Path childSource, + public synchronized void fileCopied( + final Path sourcePath, final FileStatus sourceStatus, final Path destPath, final long blockSize, @@ -67,7 +68,7 @@ public void fileCopied( S3Guard.addMoveFile(metadataStore, srcPaths, dstMetas, - childSource, + sourcePath, destPath, sourceStatus.getLen(), blockSize, @@ -85,7 +86,7 @@ public void fileCopied( } @Override - public void directoryMarkerCopied(final FileStatus sourceStatus, + public synchronized void directoryMarkerCopied(final FileStatus sourceStatus, final Path destPath, final boolean addAncestors) throws IOException { S3Guard.addMoveDir(metadataStore, srcPaths, dstMetas, @@ -104,7 +105,7 @@ public void directoryMarkerCopied(final FileStatus sourceStatus, } @Override - public void noteSourceDirectoryMoved() throws IOException { + public synchronized void noteSourceDirectoryMoved() throws IOException { if (!srcPaths.contains(getSourceRoot())) { S3Guard.addMoveDir(metadataStore, srcPaths, dstMetas, getSourceRoot(), @@ -113,20 +114,27 @@ public void noteSourceDirectoryMoved() throws IOException { } @Override - public void sourceObjectsDeleted( + public synchronized void sourceObjectsDeleted( final List keys) throws IOException { // convert to paths. deletedKeys.addAll(keys); } @Override - public void complete() throws IOException { + public void completeRename() throws IOException { metadataStore.move(srcPaths, dstMetas); + super.completeRename(); } @Override - public IOException renameFailed(final Exception ex) throws IOException { - super.renameFailed(ex); - return null; + public IOException renameFailed(final Exception ex) { + LOG.warn("Rename has failed; updating S3Guard"); + try { + metadataStore.move(srcPaths, dstMetas); + } catch (IOException | SdkBaseException e) { + LOG.warn("Ignoring error raised in AWS SDK ", e); + } + + return super.renameFailed(ex); } } 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 296712b90be57..dceafbe069118 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 @@ -1765,7 +1765,7 @@ public Invoker getInvoker() { @Override public RenameOperation initiateRenameOperation(final StoreContext storeContext, final Path source, - final Path dest) throws IOException { + final FileStatus srcStatus, final Path dest) throws IOException { return new DelayedUpdateRenameOperation(storeContext, this, source, dest); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java index e92ddfef8f87c..d7b70cf418853 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java @@ -535,7 +535,7 @@ DirListingMetadata getDirListingMeta(Path p){ @Override public RenameOperation initiateRenameOperation(final StoreContext storeContext, final Path source, - final Path dest) throws IOException { + final FileStatus srcStatus, final Path dest) throws IOException { return new DelayedUpdateRenameOperation(storeContext, this, source, dest); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java index 2644790aaf29b..8eb63ad37ed6a 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java @@ -27,6 +27,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.Retries.RetryTranslated; @@ -259,6 +260,7 @@ void prune(long modTime, String keyPrefix) * * @param storeContext store context. * @param source source path + * @param srcStatus * @param dest destination path. * @return the rename operation to update * @throws IOException Failure. @@ -266,6 +268,7 @@ void prune(long modTime, String keyPrefix) RenameOperation initiateRenameOperation( StoreContext storeContext, Path source, + FileStatus srcStatus, Path dest) throws IOException; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java index 46b525e1f1432..4e10b1e07b6c5 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java @@ -126,14 +126,14 @@ public void updateParameters(Map parameters) @Override public RenameOperation initiateRenameOperation(final StoreContext storeContext, final Path source, - final Path dest) + final FileStatus srcStatus, final Path dest) throws IOException { return new NullRenameOperation(source, dest, storeContext.getUsername()); } private static class NullRenameOperation extends RenameOperation { - public NullRenameOperation(final Path source, + private NullRenameOperation(final Path source, final Path dest, final String owner) { super(source, dest, owner); @@ -154,9 +154,5 @@ public void directoryMarkerCopied(final FileStatus sourceStatus, final boolean addAncestors) throws IOException { } - @Override - public void complete() throws IOException { - - } } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RenameOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RenameOperation.java index c9ad56e36e17b..9cef97e80c515 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RenameOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RenameOperation.java @@ -18,20 +18,31 @@ package org.apache.hadoop.fs.s3a.s3guard; -import java.io.Closeable; import java.io.IOException; import java.util.List; +import com.amazonaws.SdkBaseException; import com.amazonaws.services.s3.model.DeleteObjectsRequest; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.impl.StoreContext; +import org.apache.hadoop.util.DurationInfo; + +import static org.apache.hadoop.fs.s3a.S3AUtils.translateException; /** - * The base class for a rename operation. + * A class which manages updating the metastore with the rename process + * as initiated in the S3AFilesystem rename. + * Subclasses must provide an implementation and return it in + * {@link MetadataStore#initiateRenameOperation(StoreContext, Path, FileStatus, Path)}. */ -public abstract class RenameOperation implements Closeable { +public abstract class RenameOperation { + public static final Logger LOG = LoggerFactory.getLogger( + RenameOperation.class); /** source path. */ private final Path sourceRoot; @@ -39,22 +50,27 @@ public abstract class RenameOperation implements Closeable { /** destination path. */ private final Path dest; + /** owner of the filesystem. */ private final String owner; + private final DurationInfo durationInfo; + /** * constructor. * @param sourceRoot source path. * @param dest destination path. - * @param owner + * @param owner owner of the filesystem. */ - public RenameOperation( + protected RenameOperation( final Path sourceRoot, final Path dest, final String owner) { this.sourceRoot = sourceRoot; this.dest = dest; this.owner = owner; + durationInfo = new DurationInfo(LOG, false, + "rename(%s, %s)", sourceRoot, dest); } public Path getSourceRoot() { @@ -69,15 +85,11 @@ public String getOwner() { return owner; } - @Override - public void close() throws IOException { - - } - /** * A file has been copied. * - * @param childSource + * @param childSource source of the file. This may actually be different + * from the path of the sourceStatus. * @param sourceStatus status of source. * @param destPath destination path. * @param blockSize block size. @@ -85,7 +97,8 @@ public void close() throws IOException { * @throws IOException failure. */ public abstract void fileCopied( - final Path childSource, FileStatus sourceStatus, + Path childSource, + FileStatus sourceStatus, Path destPath, long blockSize, boolean addAncestors) throws IOException; @@ -108,11 +121,14 @@ public abstract void directoryMarkerCopied( * been updated with the results of any partial delete failure, * such that all files known to have been deleted will have been * removed. + * @param e * @param undeletedObjects list of objects which were not deleted. */ - public void deleteFailed( + public IOException deleteFailed( + final Exception e, final List keysToDelete, final List undeletedObjects) { + return convertToIOException(e); } @@ -133,22 +149,57 @@ public void sourceObjectsDeleted( final List keys) throws IOException { } - /** * Complete the operation. * @throws IOException failure. */ - public abstract void complete() throws IOException ; + public void completeRename() throws IOException { + noteRenameFinished(); + } + + /** + * Note that the rename has finished by closing the duration info; + * this will log the duration of the operation at debug. + */ + protected void noteRenameFinished() { + durationInfo.close(); + } /** * Rename has failed. * The metastore now needs to be updated with its current state * even though the operation is incomplete. - * @throws IOException failure. + * Implementations MUST NOT throw exceptions here, as this is going to + * be invoked in an exception handler. + * catch and log or catch and return/wrap. + * + * The base implementation returns the IOE passed in and translates + * any AWS exception into an IOE. + * @param ex the exception which caused the failure. + * This is either an IOException or and AWS exception + * @return an IOException to throw in an exception. */ - public IOException renameFailed(Exception ex) throws IOException { - + public IOException renameFailed(Exception ex) { + noteRenameFinished(); + return convertToIOException(ex); + } - return null; + /** + * Convert a passed in exception (expected to be an IOE or AWS exception + * into an IOException. + * @param ex exception caught + * @return the exception to throw in the failure handler. + */ + protected IOException convertToIOException(final Exception ex) { + if (ex instanceof IOException) { + return (IOException) ex; + } else if (ex instanceof SdkBaseException) { + return translateException("rename " + sourceRoot + " to " + dest, + sourceRoot.toString(), + (SdkBaseException) ex); + } else { + // should never happen, but for strictness + return new IOException(ex); + } } } 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 bbd3747d6e9f2..39cacf969773a 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 @@ -52,7 +52,6 @@ import org.apache.hadoop.util.BlockingThreadPoolExecutorService; import org.apache.hadoop.util.DurationInfo; -import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathsDoNotExist; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertRenameOutcome; import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; @@ -363,7 +362,7 @@ public void testRenameParentPathNotWriteable() throws Throwable { } @Test - public void testRenameSingleFileFailsLeavingSource() throws Throwable { + public void testRenameSingleFileFailsInDelete() throws Throwable { describe("rename with source read only; multi=%s", multiDelete); Path readOnlyFile = readonlyChild; @@ -379,11 +378,10 @@ public void testRenameSingleFileFailsLeavingSource() throws Throwable { // rename will fail in the delete phase expectRenameForbidden(readOnlyFile, destDir); - // and the source file is still there assertIsFile(readOnlyFile); - // but so is the copied version, because there's no attempt + // and so is the copied version, because there's no attempt // at rollback, or preflight checking on the delete permissions Path renamedFile = new Path(destDir, readOnlyFile.getName()); @@ -410,7 +408,7 @@ public void testRenameSingleFileFailsLeavingSource() throws Throwable { * */ @Test - public void testRenameFilesetFailsLeavingSourceUnchanged() throws Throwable { + public void testRenameDirFailsInDelete() throws Throwable { describe("rename with source read only; multi=%s", multiDelete); // the full FS @@ -450,6 +448,7 @@ public void testRenameFilesetFailsLeavingSourceUnchanged() throws Throwable { // even with S3Guard on, we expect the destination to match that of our // the remote state. // the test will exist + assertPathExists("Destination directory of rename", destDir); assertIsDirectory(destDir); assertFileCount("files in the source directory", roleFS, destDir, (long) filecount); @@ -459,11 +458,11 @@ public void testRenameFilesetFailsLeavingSourceUnchanged() throws Throwable { /** * Have a directory with full R/W permissions, but then remove * write access underneath, and try to delete it. - * */ @Test - public void testPartialDelete() throws Throwable { - describe("delete with part of the child tree read only; multidelete"); + public void testPartialDirDelete() throws Throwable { + describe("delete with part of the child tree read only;" + + " multidelete=%s", multiDelete); // the full FS S3AFileSystem fs = getFileSystem(); @@ -532,6 +531,29 @@ public void testPartialDelete() throws Throwable { .containsAll(readOnlyFiles); } + @Test + public void testCopyDirFailsInNoWrite() throws Throwable { + describe("Try to copy to a write-only destination"); + } + + @Test + public void testCopyFileFailsNoWrite() throws Throwable { + describe("Try to copy to a write-only destination"); + } + + @Test + public void testCopyFileFailsOnSourceRead() throws Throwable { + describe("The source file isn't readable, so the COPY fails"); + + } + + @Test + public void testCopyDirFailsOnSourceRead() throws Throwable { + describe("The source file isn't readable, so the COPY fails"); + + } + + private AccessDeniedException expectDeleteForbidden(Path path) throws Exception { try(DurationInfo ignored = new DurationInfo(LOG, true, "delete %s", path)) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java index 68f321f9b7886..1b98030508604 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java @@ -40,6 +40,7 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.commons.lang3.tuple.Triple; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.Constants; @@ -328,7 +329,7 @@ public List getCreated() { @Override public RenameOperation initiateRenameOperation(final StoreContext storeContext, final Path source, - final Path dest) throws IOException { + final FileStatus srcStatus, final Path dest) throws IOException { throw new UnsupportedOperationException("unsupported"); } } From 4bda57d11c1d2dbbe81b692ca139c959c47bfef6 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 3 May 2019 18:14:59 +0100 Subject: [PATCH 13/22] HADOOP-15183: rename operation failure handling works in the test cases * Add explicit notion of a StoreOperation, similar to Gang-of-four Command pattern; RenameOperation => RenameTracker and a subclass of this * DelayedUpdateRenameTracker will update metastore with destfiles on a falure, while leaving source alone. * now creating a directory tree for each rename test, so that we can verify that it works for >1 entry deep. Size of tree depends on -Dscale flag. The delete cleanup tries to work out from the metastore if a parent dir is empty or not; needs some isolation for its own test. New tests all happy. This still only updates the store on the completion of a successful rename, so the window of inconsistency (and failure ) is O(files* data); moving to do it after every file is copied will (a) reduce peak load and (b) shrink the window significantly Change-Id: I33c82856107f5c9c685c72eedd5309d6489a56dd --- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 77 +++-- .../fs/s3a/impl/FunctionsRaisingIOE.java | 12 +- .../hadoop/fs/s3a/impl/StoreOperation.java | 42 +++ ...n.java => DelayedUpdateRenameTracker.java} | 82 +++-- .../fs/s3a/s3guard/DynamoDBMetadataStore.java | 10 +- .../fs/s3a/s3guard/LocalMetadataStore.java | 4 +- .../hadoop/fs/s3a/s3guard/MetadataStore.java | 2 +- .../fs/s3a/s3guard/NullMetadataStore.java | 21 +- ...enameOperation.java => RenameTracker.java} | 41 +-- .../s3a/impl/ITestPartialRenamesDeletes.java | 282 ++++++++++++------ .../s3a/impl/TestPartialDeleteFailures.java | 4 +- .../hadoop/fs/s3a/test/ExtraAssertions.java | 10 +- 12 files changed, 399 insertions(+), 188 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreOperation.java rename hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/{DelayedUpdateRenameOperation.java => DelayedUpdateRenameTracker.java} (57%) rename hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/{RenameOperation.java => RenameTracker.java} (88%) 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 431ffc0357b30..a95176ec242b8 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 @@ -44,6 +44,7 @@ import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.BiFunction; import javax.annotation.Nullable; import com.amazonaws.AmazonClientException; @@ -77,6 +78,7 @@ import com.amazonaws.event.ProgressListener; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; import com.google.common.util.concurrent.ListeningExecutorService; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -93,7 +95,7 @@ import org.apache.hadoop.fs.s3a.impl.FunctionsRaisingIOE; import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport; import org.apache.hadoop.fs.s3a.impl.StoreContext; -import org.apache.hadoop.fs.s3a.s3guard.RenameOperation; +import org.apache.hadoop.fs.s3a.s3guard.RenameTracker; import org.apache.hadoop.fs.s3a.select.InternalSelectConstants; import org.apache.hadoop.util.DurationInfo; import org.apache.hadoop.util.LambdaUtils; @@ -1222,7 +1224,7 @@ private boolean innerRename(Path source, Path dest) // The store-specific rename operation is used to keep the store // to date with the in-progress operation. // for the null store, these are all no-ops. - final RenameOperation renameOperation = + final RenameTracker renameTracker = metadataStore.initiateRenameOperation( createStoreContext(), src, srcStatus, dest); @@ -1231,7 +1233,7 @@ private boolean innerRename(Path source, Path dest) new ArrayList<>(renameParallelLimit); // aggregate operation to wait for the copies to complete then reset // the list. - final FunctionsRaisingIOE.FunctionRaisingIOE + final FunctionsRaisingIOE.FunctionRaisingIOE completeActiveCopies = (String reason) -> { LOG.debug("Waiting for {} active copies to complete during {}", activeCopies.size(), reason); @@ -1255,14 +1257,14 @@ private boolean innerRename(Path source, Path dest) } // destination either does not exist or is a file to overwrite. LOG.debug("rename: renaming file {} to {}", src, copyDestinationPath); - copySourceAndUpdateRenameOperation(renameOperation, + copySourceAndUpdateTracker(renameTracker, src, srcKey, srcStatus, copyDestinationPath, copyDestinationKey, false); // delete the source deleteObjectAtPath(src, srcKey, true); - // TODO: renameOperation.sourceObjectsDeleted(keysToDelete); - + // and update the tracker + renameTracker.sourceObjectsDeleted(Lists.newArrayList(src)); } else { LOG.debug("rename: renaming directory {} to {}", src, dst); @@ -1276,10 +1278,21 @@ private boolean innerRename(Path source, Path dest) "cannot rename a directory to a subdirectory of itself "); } - List keysToDelete = new ArrayList<>(); + // These are the lists of keys to delete and of their paths, the + // latter being used to update the rename tracker. + final List keysToDelete = new ArrayList<>(); + final List pathsToDelete = new ArrayList<>(); + // the operation to update the lists of keys and paths. + final BiFunction + toDelete = (Path path, String key) -> { + pathsToDelete.add(path); + keysToDelete.add(new DeleteObjectsRequest.KeyVersion(key)); + return null; + }; + if (dstStatus != null && dstStatus.isEmptyDirectory() == Tristate.TRUE) { // delete unnecessary fake directory. - keysToDelete.add(new DeleteObjectsRequest.KeyVersion(dstKey)); + toDelete.apply(dstStatus.getPath(), dstKey); } Path parentPath = keyToQualifiedPath(srcKey); @@ -1291,47 +1304,49 @@ private boolean innerRename(Path source, Path dest) String key = (status.isDirectory() && !k.endsWith("/")) ? k + "/" : k; - keysToDelete - .add(new DeleteObjectsRequest.KeyVersion(key)); String newDstKey = dstKey + key.substring(srcKey.length()); Path childSourcePath = keyToQualifiedPath(key); + + toDelete.apply(childSourcePath, key); + Path childDestPath = keyToQualifiedPath(newDstKey); // set up for async operation but run in sync mode initially. // we will need to parallelize updates to metastore // for that. CompletableFuture copy = submit(boundedThreadPool, () -> - copySourceAndUpdateRenameOperation(renameOperation, + copySourceAndUpdateTracker(renameTracker, childSourcePath, key, status, childDestPath, newDstKey, true)); activeCopies.add(copy); if (activeCopies.size() == renameParallelLimit) { LOG.debug("Waiting for active copies to complete"); - waitForCompletion(activeCopies); - activeCopies.clear(); + completeActiveCopies.apply("Size threshold reached"); } if (keysToDelete.size() == MAX_ENTRIES_TO_DELETE) { // time to queue a delete. // first wait for the copies in progress to finish // (deleting a file mid-copy would not be good) completeActiveCopies.apply("before a delete"); - removeSourceObjects(renameOperation, keysToDelete); + removeSourceObjects(renameTracker, keysToDelete, pathsToDelete); + // now reset the lists. keysToDelete.clear(); + pathsToDelete.clear(); } } // end of iteration await final set of copies completeActiveCopies.apply("final copies"); // end of iteration -the final delete. - // This will notify the renameOperation that these objects + // This will notify the renameTracker that these objects // have been deleted. - removeSourceObjects(renameOperation, keysToDelete); + removeSourceObjects(renameTracker, keysToDelete, pathsToDelete); // We moved all the children, now move the top-level dir // Empty directory should have been added as the object summary - renameOperation.noteSourceDirectoryMoved(); + renameTracker.noteSourceDirectoryMoved(); } } catch (AmazonClientException | IOException ex) { // rename failed. @@ -1343,12 +1358,12 @@ private boolean innerRename(Path source, Path dest) } // update the store state to reflect this - throw renameOperation.renameFailed(ex); + throw renameTracker.renameFailed(ex); } // At this point the rename has completed in the S3 store. // Tell the metastore this fact and let it complete its changes - renameOperation.completeRename(); + renameTracker.completeRename(); if (!src.getParent().equals(dst.getParent())) { LOG.debug("source & dest parents are different; fix up dir markers"); @@ -1359,14 +1374,16 @@ private boolean innerRename(Path source, Path dest) } /** - * Remove source objects - * @param renameOperation operation to update. + * Remove source objects, and update the metastore + * @param renameTracker rename state to update. * @param keysToDelete list of keys to delete + * @param pathsToDelete list of paths matching the keys to delete 1:1. * @throws IOException failure */ @Retries.RetryMixed - private void removeSourceObjects(final RenameOperation renameOperation, - final List keysToDelete) + private void removeSourceObjects(final RenameTracker renameTracker, + final List keysToDelete, + final List pathsToDelete) throws IOException { List undeletedObjects = new ArrayList<>(); try { @@ -1381,16 +1398,16 @@ private void removeSourceObjects(final RenameOperation renameOperation, // all keys it has known to delete; this is just a final // bit of housekeeping and a chance to tune exception // reporting - throw renameOperation.deleteFailed(e, keysToDelete, undeletedObjects); + throw renameTracker.deleteFailed(e, pathsToDelete, undeletedObjects); } - renameOperation.sourceObjectsDeleted(keysToDelete); + renameTracker.sourceObjectsDeleted(pathsToDelete); } /** * This invoked to copy a file or directory marker then update the * rename operation on success. * It may be called in its own thread. - * @param renameOperation operation to update + * @param renameTracker operation to update * @param sourcePath source path of the copy; may have a trailing / on it. * @param srcKey source key * @param sourceStatus status of the source object @@ -1401,8 +1418,8 @@ private void removeSourceObjects(final RenameOperation renameOperation, * @throws IOException failure */ @Retries.RetryTranslated - private Path copySourceAndUpdateRenameOperation( - final RenameOperation renameOperation, + private Path copySourceAndUpdateTracker( + final RenameTracker renameTracker, final Path sourcePath, final String srcKey, final FileStatus sourceStatus, @@ -1411,12 +1428,12 @@ private Path copySourceAndUpdateRenameOperation( final boolean addAncestors) throws IOException { copyFile(srcKey, destKey, sourceStatus, sourceStatus.getLen()); if (objectRepresentsDirectory(srcKey, sourceStatus.getLen())) { - renameOperation.directoryMarkerCopied( + renameTracker.directoryMarkerCopied( sourceStatus, destPath, addAncestors); } else { - renameOperation.fileCopied( + renameTracker.fileCopied( sourcePath, sourceStatus, destPath, diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/FunctionsRaisingIOE.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/FunctionsRaisingIOE.java index 3755a64013fa8..92741aebb8b5b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/FunctionsRaisingIOE.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/FunctionsRaisingIOE.java @@ -34,19 +34,19 @@ private FunctionsRaisingIOE() { /** * Function of arity 1 which may raise an IOException. - * @param - * @param

      + * @param type of arg1 + * @param type of return value. */ @FunctionalInterface - public interface FunctionRaisingIOE { + public interface FunctionRaisingIOE { - T apply(P p) throws IOException; + R apply(T p) throws IOException; } @FunctionalInterface - public interface CallableRaisingIOE { + public interface CallableRaisingIOE { - T apply() throws IOException; + R apply() throws IOException; } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreOperation.java new file mode 100644 index 0000000000000..76d7c7b13b2d6 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreOperation.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl; + +/** + * Base class of operation in the store. + * An operation is something which executes against the context to + * perform a single function. + * It is expected to have a limited lifespan. + */ +public abstract class StoreOperation { + + private final StoreContext storeContext; + + /** + * constructor. + * @param storeContext store context. + */ + public StoreOperation(final StoreContext storeContext) { + this.storeContext = storeContext; + } + + public StoreContext getStoreContext() { + return storeContext; + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameTracker.java similarity index 57% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameOperation.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameTracker.java index dc9b50d2e7506..75968653f99fe 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameTracker.java @@ -21,14 +21,17 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collection; +import java.util.Comparator; import java.util.HashSet; import java.util.List; +import java.util.Set; +import java.util.function.Function; import com.amazonaws.SdkBaseException; -import com.amazonaws.services.s3.model.DeleteObjectsRequest; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.Tristate; import org.apache.hadoop.fs.s3a.impl.StoreContext; /** @@ -36,38 +39,41 @@ * a collection of source paths and a list of destinations are created, * then updated at the end (possibly slow). */ -public class DelayedUpdateRenameOperation extends RenameOperation { +public class DelayedUpdateRenameTracker extends RenameTracker { private final StoreContext storeContext; private final MetadataStore metadataStore; private final Collection srcPaths = new HashSet<>(); - private final List dstMetas = new ArrayList<>(); + private final List destMetas = new ArrayList<>(); - private final List deletedKeys = new ArrayList<>(); + private final List deletedPaths = new ArrayList<>(); - public DelayedUpdateRenameOperation( + public DelayedUpdateRenameTracker( final StoreContext storeContext, final MetadataStore metadataStore, final Path sourceRoot, final Path dest) { - super(sourceRoot, dest, storeContext.getUsername()); + super(storeContext, sourceRoot, dest); this.storeContext = storeContext; this.metadataStore = metadataStore; } + public StoreContext getStoreContext() { + return storeContext; + } + @Override public synchronized void fileCopied( final Path sourcePath, final FileStatus sourceStatus, final Path destPath, final long blockSize, - final boolean addAncestors - ) throws IOException { + final boolean addAncestors) throws IOException { S3Guard.addMoveFile(metadataStore, srcPaths, - dstMetas, + destMetas, sourcePath, destPath, sourceStatus.getLen(), @@ -77,7 +83,7 @@ public synchronized void fileCopied( if (addAncestors) { S3Guard.addMoveAncestors(metadataStore, srcPaths, - dstMetas, + destMetas, getSourceRoot(), sourceStatus.getPath(), destPath, @@ -89,14 +95,14 @@ public synchronized void fileCopied( public synchronized void directoryMarkerCopied(final FileStatus sourceStatus, final Path destPath, final boolean addAncestors) throws IOException { - S3Guard.addMoveDir(metadataStore, srcPaths, dstMetas, + S3Guard.addMoveDir(metadataStore, srcPaths, destMetas, sourceStatus.getPath(), destPath, getOwner()); // Ancestor directories may not be listed, so we explicitly add them if (addAncestors) { S3Guard.addMoveAncestors(metadataStore, srcPaths, - dstMetas, + destMetas, getSourceRoot(), sourceStatus.getPath(), destPath, @@ -107,7 +113,7 @@ public synchronized void directoryMarkerCopied(final FileStatus sourceStatus, @Override public synchronized void noteSourceDirectoryMoved() throws IOException { if (!srcPaths.contains(getSourceRoot())) { - S3Guard.addMoveDir(metadataStore, srcPaths, dstMetas, + S3Guard.addMoveDir(metadataStore, srcPaths, destMetas, getSourceRoot(), getDest(), getOwner()); } @@ -115,26 +121,66 @@ public synchronized void noteSourceDirectoryMoved() throws IOException { @Override public synchronized void sourceObjectsDeleted( - final List keys) throws IOException { + final List paths) throws IOException { // convert to paths. - deletedKeys.addAll(keys); + deletedPaths.addAll(paths); } @Override public void completeRename() throws IOException { - metadataStore.move(srcPaths, dstMetas); + metadataStore.move(srcPaths, destMetas); super.completeRename(); } @Override public IOException renameFailed(final Exception ex) { - LOG.warn("Rename has failed; updating S3Guard"); + LOG.warn("Rename has failed; updating s3guard with destination state"); try { - metadataStore.move(srcPaths, dstMetas); + // the destination paths are updated; the source is left alone. + // either the delete operation didn't begin, or the + metadataStore.move(new ArrayList<>(0), destMetas); + Function qualifier + = getStoreContext().getKeyToPathQualifier(); + for (Path deletedPath : deletedPaths) { + // this is not ideal in that it may leave parent stuff around. + metadataStore.delete(deletedPath); + } + deleteParentPaths(); } catch (IOException | SdkBaseException e) { LOG.warn("Ignoring error raised in AWS SDK ", e); } return super.renameFailed(ex); } + + /** + * Delete all the parent paths we know to be empty (by walking up the tree + * deleting as appropriate). + * @throws IOException failure + */ + private void deleteParentPaths() throws IOException { + Set parentPaths = new HashSet<>(); + for (Path deletedPath : deletedPaths) { + Path parent = deletedPath.getParent(); + if (!parent.equals(getSourceRoot())) { + parentPaths.add(parent); + } + } + // now there's a set of parent paths. We now want to + // get them ordered by depth, so that deeper entries come first + // that way: when we check for a parent path existing we can + // see if it really is empty. + List parents = new ArrayList<>(parentPaths); + parents.sort( + Comparator.comparing( + (Path p) -> p.depth()) + .thenComparing((Path p) -> p.toUri().getPath())); + for (Path parent : parents) { + PathMetadata md = metadataStore.get(parent, true); + if (md != null && md.isEmptyDirectory().equals(Tristate.TRUE)) { + // if were confident that this is empty: delete it. + metadataStore.delete(parent); + } + } + } } 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 dceafbe069118..1df56050cc1f0 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 @@ -554,8 +554,8 @@ public void deleteSubtree(Path path) throws IOException { return; } - // bulk execute. This needs to be paged better. - List> futures = new ArrayList<>(); + // Execute via the bounded threadpool. + final List> futures = new ArrayList<>(); for (DescendantsIterator desc = new DescendantsIterator(this, meta); desc.hasNext();) { final Path pathToDelete = desc.next().getPath(); @@ -566,6 +566,7 @@ public void deleteSubtree(Path path) throws IOException { if (futures.size() > S3GUARD_DDB_SUBMITTED_TASK_LIMIT) { // first batch done; block for completion. waitForCompletion(futures); + futures.clear(); } } waitForCompletion(futures); @@ -574,7 +575,6 @@ public void deleteSubtree(Path path) throws IOException { /** * Get a consistent view of an item. * @param path path to look up in the database - * @param path entry * @return the result * @throws IOException failure */ @@ -1763,10 +1763,10 @@ public Invoker getInvoker() { } @Override - public RenameOperation initiateRenameOperation(final StoreContext storeContext, + public RenameTracker initiateRenameOperation(final StoreContext storeContext, final Path source, final FileStatus srcStatus, final Path dest) throws IOException { - return new DelayedUpdateRenameOperation(storeContext, this, source, dest); + return new DelayedUpdateRenameTracker(storeContext, this, source, dest); } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java index d7b70cf418853..798443b68e715 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java @@ -533,9 +533,9 @@ DirListingMetadata getDirListingMeta(Path p){ } @Override - public RenameOperation initiateRenameOperation(final StoreContext storeContext, + public RenameTracker initiateRenameOperation(final StoreContext storeContext, final Path source, final FileStatus srcStatus, final Path dest) throws IOException { - return new DelayedUpdateRenameOperation(storeContext, this, source, dest); + return new DelayedUpdateRenameTracker(storeContext, this, source, dest); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java index 8eb63ad37ed6a..e6acdb75c2c2f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java @@ -265,7 +265,7 @@ void prune(long modTime, String keyPrefix) * @return the rename operation to update * @throws IOException Failure. */ - RenameOperation initiateRenameOperation( + RenameTracker initiateRenameOperation( StoreContext storeContext, Path source, FileStatus srcStatus, diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java index 4e10b1e07b6c5..a81bfac3aea37 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java @@ -124,19 +124,20 @@ public void updateParameters(Map parameters) } @Override - public RenameOperation initiateRenameOperation(final StoreContext storeContext, + public RenameTracker initiateRenameOperation(final StoreContext storeContext, final Path source, final FileStatus srcStatus, final Path dest) throws IOException { - return new NullRenameOperation(source, dest, storeContext.getUsername()); + return new NullRenameTracker(storeContext, source, dest); } - private static class NullRenameOperation extends RenameOperation { + private static class NullRenameTracker extends RenameTracker { - private NullRenameOperation(final Path source, - final Path dest, - final String owner) { - super(source, dest, owner); + private NullRenameTracker( + final StoreContext storeContext, + final Path source, + final Path dest) { + super(storeContext, source, dest); } @Override @@ -148,11 +149,5 @@ public void fileCopied(final Path childSource, } - @Override - public void directoryMarkerCopied(final FileStatus sourceStatus, - final Path destPath, - final boolean addAncestors) throws IOException { - } - } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RenameOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RenameTracker.java similarity index 88% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RenameOperation.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RenameTracker.java index 9cef97e80c515..75fe0b36ba280 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RenameOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RenameTracker.java @@ -22,13 +22,13 @@ import java.util.List; import com.amazonaws.SdkBaseException; -import com.amazonaws.services.s3.model.DeleteObjectsRequest; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.impl.StoreContext; +import org.apache.hadoop.fs.s3a.impl.StoreOperation; import org.apache.hadoop.util.DurationInfo; import static org.apache.hadoop.fs.s3a.S3AUtils.translateException; @@ -39,10 +39,10 @@ * Subclasses must provide an implementation and return it in * {@link MetadataStore#initiateRenameOperation(StoreContext, Path, FileStatus, Path)}. */ -public abstract class RenameOperation { +public abstract class RenameTracker extends StoreOperation { public static final Logger LOG = LoggerFactory.getLogger( - RenameOperation.class); + RenameTracker.class); /** source path. */ private final Path sourceRoot; @@ -50,25 +50,25 @@ public abstract class RenameOperation { /** destination path. */ private final Path dest; - /** owner of the filesystem. */ - private final String owner; + /** + * Track the duration of this operation. + */ private final DurationInfo durationInfo; /** * constructor. + * @param storeContext store context. * @param sourceRoot source path. * @param dest destination path. - * @param owner owner of the filesystem. */ - - protected RenameOperation( + protected RenameTracker( + final StoreContext storeContext, final Path sourceRoot, - final Path dest, - final String owner) { + final Path dest) { + super(storeContext); this.sourceRoot = sourceRoot; this.dest = dest; - this.owner = owner; durationInfo = new DurationInfo(LOG, false, "rename(%s, %s)", sourceRoot, dest); } @@ -82,7 +82,7 @@ public Path getDest() { } public String getOwner() { - return owner; + return getStoreContext().getUsername(); } /** @@ -110,10 +110,12 @@ public abstract void fileCopied( * @param addAncestors should ancestors be added? * @throws IOException failure. */ - public abstract void directoryMarkerCopied( + public void directoryMarkerCopied( FileStatus sourceStatus, Path destPath, - boolean addAncestors) throws IOException; + boolean addAncestors) throws IOException { + + } /** * The delete failed. @@ -121,15 +123,16 @@ public abstract void directoryMarkerCopied( * been updated with the results of any partial delete failure, * such that all files known to have been deleted will have been * removed. - * @param e + * @param e exception + * @param pathsToDelete paths which were to be deleted. * @param undeletedObjects list of objects which were not deleted. */ public IOException deleteFailed( final Exception e, - final List keysToDelete, + final List pathsToDelete, final List undeletedObjects) { - return convertToIOException(e); + return convertToIOException(e); } /** @@ -143,10 +146,10 @@ public void noteSourceDirectoryMoved() throws IOException { /** * Note that source objects have been deleted. * The metastore will already have been updated. - * @param keys keys of objects deleted. + * @param paths keys of objects deleted. */ public void sourceObjectsDeleted( - final List keys) throws IOException { + final List paths) throws IOException { } /** 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 39cacf969773a..fe4ade3b04598 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 @@ -54,6 +54,7 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.assertRenameOutcome; import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile; +import static org.apache.hadoop.fs.contract.ContractTestUtils.readUTF8; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; import static org.apache.hadoop.fs.s3a.Constants.ASSUMED_ROLE_ARN; import static org.apache.hadoop.fs.s3a.Constants.ENABLE_MULTI_DELETE; @@ -73,11 +74,7 @@ import static org.apache.hadoop.fs.s3a.auth.RoleModel.Statement; import static org.apache.hadoop.fs.s3a.auth.RoleModel.directory; import static org.apache.hadoop.fs.s3a.auth.RoleModel.statement; -import static org.apache.hadoop.fs.s3a.auth.RolePolicies.S3_ALL_BUCKETS; -import static org.apache.hadoop.fs.s3a.auth.RolePolicies.S3_BUCKET_READ_OPERATIONS; -import static org.apache.hadoop.fs.s3a.auth.RolePolicies.S3_PATH_RW_OPERATIONS; -import static org.apache.hadoop.fs.s3a.auth.RolePolicies.STATEMENT_ALLOW_SSE_KMS_RW; -import static org.apache.hadoop.fs.s3a.auth.RolePolicies.STATEMENT_S3GUARD_CLIENT; +import static org.apache.hadoop.fs.s3a.auth.RolePolicies.*; import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.bindRolePolicyStatements; import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.forbidden; import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.newAssumedRoleConfig; @@ -133,7 +130,7 @@ public class ITestPartialRenamesDeletes extends AbstractS3ATestBase { 30, TimeUnit.SECONDS, "test-operations"); - public static final int FILE_COUNT_NON_SCALED = 10; + public static final int FILE_COUNT_NON_SCALED = 2; /** * The number of files for a scaled test. This is still @@ -144,27 +141,39 @@ public class ITestPartialRenamesDeletes extends AbstractS3ATestBase { * knowing that all R/W files should have been deleted and all * R/O files rejected. */ - public static final int FILE_COUNT_SCALED = 400; + public static final int FILE_COUNT_SCALED = 10; + public static final int DIR_COUNT = 2; + public static final int DIR_COUNT_SCALED = 4; + public static final int DEPTH = 2; + public static final int DEPTH_SCALED = 2; + /** * A role FS; if non-null it is closed in teardown. */ private S3AFileSystem roleFS; + /** + * Base path for this test run; it's all + */ private Path basePath; - private Path destDir; + private Path writableDir; - private Path readonlyChild; + private Path readOnlyChild; private Path readOnlyDir; + private Path noReadDir; + /** delete policy: single or multi? */ private final boolean multiDelete; private Configuration assumedRoleConfig; - private int filecount; + private int fileCount; + private int dirCount; + private int dirDepth; private boolean scaleTest; @@ -189,28 +198,41 @@ public ITestPartialRenamesDeletes(final boolean multiDelete) { this.multiDelete = multiDelete; } + /** + * This sets up a unique path for every test run, so as to guarantee isolation + * from previous runs. + * It creates a role policy which has read access to everything except + * the contents of {@link #noReadDir}, and with write access to + * {@link #writableDir}. + * @throws Exception failure + */ @Override public void setup() throws Exception { super.setup(); assumeRoleTests(); basePath = uniquePath(); readOnlyDir = new Path(basePath, "readonlyDir"); - destDir = new Path(basePath, "renameDest"); - readonlyChild = new Path(readOnlyDir, "child"); + writableDir = new Path(basePath, "writableDir"); + readOnlyChild = new Path(readOnlyDir, "child"); + noReadDir = new Path(basePath, "noReadDir"); // the full FS S3AFileSystem fs = getFileSystem(); fs.delete(basePath, true); - fs.mkdirs(destDir); + fs.mkdirs(writableDir); // create the baseline assumed role assumedRoleConfig = createAssumedRoleConfig(); bindRolePolicyStatements(assumedRoleConfig, STATEMENT_S3GUARD_CLIENT, - STATEMENT_ALL_BUCKET_READ_ACCESS, - new Statement(Effects.Allow) + STATEMENT_ALL_BUCKET_READ_ACCESS, // root: r-x + new Statement(Effects.Allow) // dest: rwx .addActions(S3_PATH_RW_OPERATIONS) - .addResources(directory(destDir)) - ); + .addResources(directory(writableDir)), + new Statement(Effects.Deny) // noReadDir: --x + .addActions(S3_ALL_GET) + .addActions(S3_ALL_PUT) + .addActions(S3_ALL_DELETE) + .addResources(directory(noReadDir))); // the role configured to that set of restrictions roleFS = (S3AFileSystem) readOnlyDir.getFileSystem(assumedRoleConfig); @@ -221,9 +243,9 @@ public void setup() throws Exception { getConfiguration(), KEY_SCALE_TESTS_ENABLED, DEFAULT_SCALE_TESTS_ENABLED); - filecount = scaleTest - ? FILE_COUNT_SCALED - : FILE_COUNT_NON_SCALED; + fileCount = scaleTest ? FILE_COUNT_SCALED : FILE_COUNT_NON_SCALED; + dirCount = scaleTest ? DIR_COUNT_SCALED : DIR_COUNT; + dirDepth = scaleTest ? DEPTH_SCALED : DEPTH; } @Override @@ -232,6 +254,12 @@ public void teardown() throws Exception { super.teardown(); } + /** + * Directory cleanup includes pruning everything under the path. + * This ensures that any in the tree from failed tests don't fill up + * the store with many, many, deleted entries. + * @throws IOException failure. + */ @Override protected void deleteTestDirInTeardown() throws IOException { super.deleteTestDirInTeardown(); @@ -294,7 +322,7 @@ protected Configuration createConfiguration() { /** * Create a unique path, which includes method name, - * multdelete flag and a random UUID. + * multidelete flag and a random UUID. * @return a string to use for paths. * @throws IOException path creation failure. */ @@ -311,9 +339,17 @@ private Path uniquePath() throws IOException { */ @Test public void testCannotTouchUnderRODir() throws Throwable { - forbidden("touching the empty child " + readonlyChild, + forbidden("touching the empty child " + readOnlyChild, "", - () -> ContractTestUtils.touch(roleFS, readonlyChild)); + () -> touch(roleFS, readOnlyChild)); + } + @Test + public void testCannotReadUnderNoReadDir() throws Throwable { + Path path = new Path(noReadDir, "unreadable.txt"); + createFile(getFileSystem(), path, true, "readonly".getBytes()); + forbidden("trying to read " + path, + "", + () -> readUTF8(roleFS, path, -1)); } @Test @@ -339,32 +375,32 @@ public void testRenameParentPathNotWriteable() throws Throwable { new Statement(Effects.Allow) .addActions(S3_PATH_RW_OPERATIONS) .addResources(directory(readOnlyDir)) - .addResources(directory(destDir))); + .addResources(directory(writableDir))); roleFS = (S3AFileSystem) readOnlyDir.getFileSystem(conf); S3AFileSystem fs = getFileSystem(); roleFS.getFileStatus(ROOT); fs.mkdirs(readOnlyDir); // you can create an adjacent child - touch(fs, readonlyChild); + touch(fs, readOnlyChild); - fs.delete(destDir, true); + fs.delete(writableDir, true); // as dest doesn't exist, this will map child -> dest - assertRenameOutcome(roleFS, readonlyChild, destDir, true); + assertRenameOutcome(roleFS, readOnlyChild, writableDir, true); - assertIsFile(destDir); + assertIsFile(writableDir); assertIsDirectory(readOnlyDir); - Path renamedDestPath = new Path(readOnlyDir, destDir.getName()); - assertRenameOutcome(roleFS, destDir, readOnlyDir, true); + Path renamedDestPath = new Path(readOnlyDir, writableDir.getName()); + assertRenameOutcome(roleFS, writableDir, readOnlyDir, true); assertIsFile(renamedDestPath); roleFS.delete(readOnlyDir, true); - roleFS.delete(destDir, true); + roleFS.delete(writableDir, true); } @Test public void testRenameSingleFileFailsInDelete() throws Throwable { describe("rename with source read only; multi=%s", multiDelete); - Path readOnlyFile = readonlyChild; + Path readOnlyFile = readOnlyChild; // the full FS S3AFileSystem fs = getFileSystem(); @@ -373,23 +409,23 @@ public void testRenameSingleFileFailsInDelete() throws Throwable { // this file is readable by the roleFS, but cannot be deleted touch(fs, readOnlyFile); - roleFS.delete(destDir, true); - roleFS.mkdirs(destDir); + roleFS.delete(writableDir, true); + roleFS.mkdirs(writableDir); // rename will fail in the delete phase - expectRenameForbidden(readOnlyFile, destDir); + expectRenameForbidden(readOnlyFile, writableDir); // and the source file is still there assertIsFile(readOnlyFile); // and so is the copied version, because there's no attempt // at rollback, or preflight checking on the delete permissions - Path renamedFile = new Path(destDir, readOnlyFile.getName()); + Path renamedFile = new Path(writableDir, readOnlyFile.getName()); assertIsFile(renamedFile); ContractTestUtils.assertDeleted(roleFS, renamedFile, true); assertFileCount("Empty Dest Dir", roleFS, - destDir, 0); + writableDir, 0); } /** @@ -413,21 +449,22 @@ public void testRenameDirFailsInDelete() throws Throwable { // the full FS S3AFileSystem fs = getFileSystem(); - fs.delete(basePath, true); - roleFS.mkdirs(destDir); + roleFS.mkdirs(writableDir); // create a set of files // this is done in parallel as it is 10x faster on a long-haul test run. - List createdFiles = createFiles(fs, readOnlyDir, filecount); + List createdFiles = createFiles(fs, readOnlyDir, dirDepth, fileCount, dirCount); // are they all there? + int expectedFileCount = createdFiles.size(); assertFileCount("files ready to rename", roleFS, - readOnlyDir, (long) filecount); + readOnlyDir, expectedFileCount); // try to rename the directory - LOG.info("Renaming readonly files {} to {}", readOnlyDir, destDir); + LOG.info("Renaming readonly files {} to {}", readOnlyDir, writableDir); - AccessDeniedException deniedException = expectRenameForbidden(readOnlyDir, destDir); + AccessDeniedException deniedException = expectRenameForbidden(readOnlyDir, + writableDir); if (multiDelete) { // look in that exception for a multidelete MultiObjectDeleteException mde = extractCause( @@ -436,23 +473,85 @@ public void testRenameDirFailsInDelete() throws Throwable { = extractUndeletedPaths(mde, fs::keyToQualifiedPath); Assertions.assertThat(undeleted) .as("files which could not be deleted") - .hasSize(filecount) + .hasSize(expectedFileCount) .containsAll(createdFiles) .containsExactlyInAnyOrderElementsOf(createdFiles); } LOG.info("Result of renaming read-only files is AccessDeniedException", deniedException); assertFileCount("files in the source directory", roleFS, - readOnlyDir, (long) filecount); + readOnlyDir, expectedFileCount); // now lets look at the destination. // even with S3Guard on, we expect the destination to match that of our // the remote state. // the test will exist - assertPathExists("Destination directory of rename", destDir); - assertIsDirectory(destDir); + assertPathExists("Destination directory of rename", writableDir); + assertIsDirectory(writableDir); + assertFileCount("files in the dest directory", roleFS, + writableDir, expectedFileCount); + + } + + @Test + public void testRenameFileFailsNoWrite() throws Throwable { + describe("Try to rename to a write-only destination fails with src" + + " & dest unchanged."); + roleFS.mkdirs(writableDir); + S3AFileSystem fs = getFileSystem(); +// List deletableFiles = createFiles(fs, writableDir, filecount); + + Path source = new Path(writableDir, "source"); + touch(fs, source); + fs.mkdirs(readOnlyDir); + Path dest = new Path(readOnlyDir, "dest"); + LOG.info("Renaming files {} to {}", writableDir, dest); + // rename fails but doesn't raise an exception. Good or bad? + expectRenameForbidden(source, dest); + assertIsFile(source); + assertPathDoesNotExist("rename destination", dest); + } + + @Test + public void testCopyDirFailsInNoWrite() throws Throwable { + describe("Try to copy to a write-only destination"); + roleFS.mkdirs(writableDir); + S3AFileSystem fs = getFileSystem(); + List files = createFiles(fs, writableDir, dirDepth, fileCount, + dirCount); + + fs.mkdirs(readOnlyDir); + Path dest = new Path(readOnlyDir, "dest"); + expectRenameForbidden(writableDir, dest); + assertPathDoesNotExist("rename destination", dest); assertFileCount("files in the source directory", roleFS, - destDir, (long) filecount); + writableDir, files.size()); + } + @Test + public void testCopyFileFailsOnSourceRead() throws Throwable { + describe("The source file isn't readable, so the COPY fails"); + Path source = new Path(noReadDir, "source"); + S3AFileSystem fs = getFileSystem(); + touch(fs, source); + fs.mkdirs(writableDir); + Path dest = new Path(writableDir, "dest"); + expectRenameForbidden(source, dest); + assertIsFile(source); + assertPathDoesNotExist("rename destination", dest); + } + + @Test + public void testCopyDirFailsOnSourceRead() throws Throwable { + describe("The source file isn't readable, so the COPY fails"); + Path source = new Path(noReadDir, "source"); + S3AFileSystem fs = getFileSystem(); + List files = createFiles(fs, noReadDir, dirDepth, fileCount, + dirCount); + fs.mkdirs(writableDir); + Path dest = new Path(writableDir, "dest"); + expectRenameForbidden(noReadDir, dest); + assertFileCount("files in the source directory", fs, + noReadDir, files.size()); } /** @@ -467,8 +566,8 @@ public void testPartialDirDelete() throws Throwable { // the full FS S3AFileSystem fs = getFileSystem(); - List readOnlyFiles = createFiles(fs, readOnlyDir, filecount); - List deletableFiles = createFiles(fs, destDir, filecount); + List readOnlyFiles = createFiles(fs, readOnlyDir, dirDepth, fileCount, dirCount); + List deletableFiles = createFiles(fs, writableDir, dirDepth, fileCount, dirCount); // as a safety check, verify that one of the deletable files can be deleted Path head = deletableFiles.remove(0); @@ -488,7 +587,8 @@ public void testPartialDirDelete() throws Throwable { if (multiDelete) { // multi-delete status checks extractCause(MultiObjectDeleteException.class, ex); - rejectionCount.assertDiffEquals("Wrong rejection count", filecount); + rejectionCount.assertDiffEquals("Wrong rejection count", + readOnlyFiles.size()); deleteVerbCount.assertDiffEquals("Wrong delete count", 1); reset(rejectionCount, deleteVerbCount); } @@ -513,7 +613,8 @@ public void testPartialDirDelete() throws Throwable { Assertions.assertThat(allFiles) .as("files which were deleted") .containsExactlyInAnyOrderElementsOf(deletableFiles); - rejectionCount.assertDiffEquals("Wrong rejection count", filecount); + rejectionCount.assertDiffEquals("Wrong rejection count", + readOnlyFiles.size()); } reset(rejectionCount, deleteVerbCount); @@ -531,29 +632,6 @@ public void testPartialDirDelete() throws Throwable { .containsAll(readOnlyFiles); } - @Test - public void testCopyDirFailsInNoWrite() throws Throwable { - describe("Try to copy to a write-only destination"); - } - - @Test - public void testCopyFileFailsNoWrite() throws Throwable { - describe("Try to copy to a write-only destination"); - } - - @Test - public void testCopyFileFailsOnSourceRead() throws Throwable { - describe("The source file isn't readable, so the COPY fails"); - - } - - @Test - public void testCopyDirFailsOnSourceRead() throws Throwable { - describe("The source file isn't readable, so the COPY fails"); - - } - - private AccessDeniedException expectDeleteForbidden(Path path) throws Exception { try(DurationInfo ignored = new DurationInfo(LOG, true, "delete %s", path)) { @@ -567,7 +645,7 @@ private AccessDeniedException expectDeleteForbidden(Path path) throws Exception } /** - * Expect that a rename will fail with an exception. + * Expect that a rename will fail with an exception using the roleFS. * @param src source * @param dest dest * @return the exception raised. @@ -576,7 +654,8 @@ private AccessDeniedException expectDeleteForbidden(Path path) throws Exception private AccessDeniedException expectRenameForbidden(Path src, Path dest) throws Exception { try(DurationInfo ignored = - new DurationInfo(LOG, true, "rename")) { + new DurationInfo(LOG, true, + "rename(%s, %s)", src, dest)) { return forbidden( "Renaming " + src + " to " + dest, "", @@ -665,25 +744,52 @@ private static CompletableFuture put(FileSystem fs, * Parallel-touch a set of files in the destination directory. * @param fs filesystem * @param destDir destination - * @param range range 1..range inclusive of files to create. - * @return the list of paths created. + * @param depth file depth + * @param fileCount number of files to create. + * @param dirCount number of dirs to create at each level + * @return the list of files created. */ public static List createFiles(final FileSystem fs, final Path destDir, - final int range) throws IOException { - List> futures = new ArrayList<>(range); - List paths = new ArrayList<>(range); + final int depth, + final int fileCount, + final int dirCount) throws IOException { + List> futures = new ArrayList<>(fileCount); + List paths = new ArrayList<>(fileCount); + List dirs = new ArrayList<>(fileCount); + buildPaths(paths, dirs, destDir, depth, fileCount, dirCount); try(DurationInfo ignore = - new DurationInfo(LOG, "Creating %d files", range)) { - for (int i = 0; i < range; i++) { - String name = "file-" + i; - Path p = new Path(destDir, name); - paths.add(p); - futures.add(put(fs, p, name)); + new DurationInfo(LOG, "Creating %d files", fileCount)) { + for (Path path : paths) { + futures.add(put(fs, path, path.getName())); } waitForCompletion(futures); return paths; } } + private static void buildPaths( + final List filePaths, + final List dirPaths, + final Path destDir, + final int depth, + final int fileCount, + final int dirCount) { + if (depth<=0) { + return; + } + // create the file paths + for (int i = 0; i < fileCount; i++) { + String name = "file-" + i; + Path p = new Path(destDir, name); + filePaths.add(p); + } + for (int i = 0; i < dirCount; i++) { + String name = "dir-" + i; + Path p = new Path(destDir, name); + dirPaths.add(p); + buildPaths(filePaths, dirPaths, p, depth - 1, fileCount, dirCount); + } + + } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java index 1b98030508604..7c649db8a08a5 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java @@ -51,7 +51,7 @@ import org.apache.hadoop.fs.s3a.s3guard.DirListingMetadata; import org.apache.hadoop.fs.s3a.s3guard.MetadataStore; import org.apache.hadoop.fs.s3a.s3guard.PathMetadata; -import org.apache.hadoop.fs.s3a.s3guard.RenameOperation; +import org.apache.hadoop.fs.s3a.s3guard.RenameTracker; import org.apache.hadoop.io.retry.RetryPolicies; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.BlockingThreadPoolExecutorService; @@ -327,7 +327,7 @@ public List getCreated() { } @Override - public RenameOperation initiateRenameOperation(final StoreContext storeContext, + public RenameTracker initiateRenameOperation(final StoreContext storeContext, final Path source, final FileStatus srcStatus, final Path dest) throws IOException { throw new UnsupportedOperationException("unsupported"); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/ExtraAssertions.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/ExtraAssertions.java index 7f42ebef78752..9b5fa053aacc8 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/ExtraAssertions.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/ExtraAssertions.java @@ -47,14 +47,16 @@ public class ExtraAssertions { /** * Assert that the number of files in a destination matches that expected. - * @param text text to use in the message + * @param message text to use in the message * @param fs filesystem * @param path path to list (recursively) * @param expected expected count * @throws IOException IO problem */ - public static void assertFileCount(String text, FileSystem fs, - Path path, long expected) + public static void assertFileCount(final String message, + final FileSystem fs, + final Path path, + final long expected) throws IOException { List files = new ArrayList<>(); try (DurationInfo ignored = new DurationInfo(LOG, false, @@ -65,7 +67,7 @@ public static void assertFileCount(String text, FileSystem fs, long actual = files.size(); if (actual != expected) { String ls = files.stream().collect(Collectors.joining("\n")); - Assert.fail(text + ": expected " + expected + " files in " + path + Assert.fail(message + ": expected " + expected + " files in " + path + " but got " + actual + "\n" + ls); } } From bfe8b4f9ad4d34d157a0104752adb4ea11a6ace5 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 6 May 2019 19:04:09 +0100 Subject: [PATCH 14/22] HADOOP-15183: progressive rename tracker This adds a rename tracker which adds entries incrementally, deletes source files after a delete. As such, the failed() and completed() operations become no-ops. + not yet tested; thinking of some unit tests to add alongside the live ones + also explicitly sorting the order of addition and deletion of pathmetadata elements such that in a patch delete, higher level entries are added first, while during deletion leaves are deleted first. This is to maintain the invariant "there are never any entries in the tables which do not have parents". Lots of tests on the sorting...if someone could tell me why Comparators.reverse returns something which fails my tests, I would love to know. Outstanding TODO items * switch DDB and local metastores to the new tracker * isolated unit test to simulate some of the operations and verify that all is good Change-Id: Iad76eefe4770699da6d867fa34ab59cebc9cf28c Testing: only ITestPartialRenamesDeletes against S3 ireland --- .../s3guard/DelayedUpdateRenameTracker.java | 30 +-- .../fs/s3a/s3guard/DynamoDBMetadataStore.java | 27 ++- .../fs/s3a/s3guard/PathOrderComparators.java | 107 +++++++++ .../s3a/s3guard/ProgressiveRenameTracker.java | 212 ++++++++++++++++++ .../apache/hadoop/fs/s3a/s3guard/S3Guard.java | 38 ++-- .../s3a/s3guard/TestPathOrderComparators.java | 197 ++++++++++++++++ .../hadoop/fs/s3a/test/ExtraAssertions.java | 3 +- 7 files changed, 583 insertions(+), 31 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathOrderComparators.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ProgressiveRenameTracker.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestPathOrderComparators.java diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameTracker.java index 75968653f99fe..844adab027e0d 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameTracker.java @@ -25,7 +25,6 @@ import java.util.HashSet; import java.util.List; import java.util.Set; -import java.util.function.Function; import com.amazonaws.SdkBaseException; @@ -34,6 +33,10 @@ import org.apache.hadoop.fs.s3a.Tristate; import org.apache.hadoop.fs.s3a.impl.StoreContext; +import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.addMoveAncestors; +import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.addMoveDir; +import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.addMoveFile; + /** * This is the rename updating strategy originally used: * a collection of source paths and a list of destinations are created, @@ -44,7 +47,8 @@ public class DelayedUpdateRenameTracker extends RenameTracker { private final StoreContext storeContext; private final MetadataStore metadataStore; - private final Collection srcPaths = new HashSet<>(); + + private final Collection sourcePaths = new HashSet<>(); private final List destMetas = new ArrayList<>(); @@ -71,8 +75,8 @@ public synchronized void fileCopied( final Path destPath, final long blockSize, final boolean addAncestors) throws IOException { - S3Guard.addMoveFile(metadataStore, - srcPaths, + addMoveFile(metadataStore, + sourcePaths, destMetas, sourcePath, destPath, @@ -81,8 +85,8 @@ public synchronized void fileCopied( getOwner()); // Ancestor directories may not be listed, so we explicitly add them if (addAncestors) { - S3Guard.addMoveAncestors(metadataStore, - srcPaths, + addMoveAncestors(metadataStore, + sourcePaths, destMetas, getSourceRoot(), sourceStatus.getPath(), @@ -95,13 +99,13 @@ public synchronized void fileCopied( public synchronized void directoryMarkerCopied(final FileStatus sourceStatus, final Path destPath, final boolean addAncestors) throws IOException { - S3Guard.addMoveDir(metadataStore, srcPaths, destMetas, + addMoveDir(metadataStore, sourcePaths, destMetas, sourceStatus.getPath(), destPath, getOwner()); // Ancestor directories may not be listed, so we explicitly add them if (addAncestors) { - S3Guard.addMoveAncestors(metadataStore, - srcPaths, + addMoveAncestors(metadataStore, + sourcePaths, destMetas, getSourceRoot(), sourceStatus.getPath(), @@ -112,8 +116,8 @@ public synchronized void directoryMarkerCopied(final FileStatus sourceStatus, @Override public synchronized void noteSourceDirectoryMoved() throws IOException { - if (!srcPaths.contains(getSourceRoot())) { - S3Guard.addMoveDir(metadataStore, srcPaths, destMetas, + if (!sourcePaths.contains(getSourceRoot())) { + addMoveDir(metadataStore, sourcePaths, destMetas, getSourceRoot(), getDest(), getOwner()); } @@ -128,7 +132,7 @@ public synchronized void sourceObjectsDeleted( @Override public void completeRename() throws IOException { - metadataStore.move(srcPaths, destMetas); + metadataStore.move(sourcePaths, destMetas); super.completeRename(); } @@ -139,8 +143,6 @@ public IOException renameFailed(final Exception ex) { // the destination paths are updated; the source is left alone. // either the delete operation didn't begin, or the metadataStore.move(new ArrayList<>(0), destMetas); - Function qualifier - = getStoreContext().getKeyToPathQualifier(); for (Path deletedPath : deletedPaths) { // this is not ideal in that it may leave parent stuff around. metadataStore.delete(deletedPath); 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 1df56050cc1f0..02855469cba97 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 @@ -759,6 +759,22 @@ Collection completeAncestry( return ancestry.values(); } + /** + * {@inheritDoc}. + * + * The DDB implementation sorts all the paths such that new items + * are ordered highest level entry first; deleted items are ordered + * lowest entry first. + * + * This is to ensure that if a client failed partway through the update, + * there will no entries in the table which lack parent entries. + * @param pathsToDelete Collection of all paths that were removed from the + * source directory tree of the move. + * @param pathsToCreate Collection of all PathMetadata for the new paths + * that were created at the destination of the rename + * (). + * @throws IOException + */ @Override @Retries.RetryTranslated public void move(Collection pathsToDelete, @@ -779,14 +795,21 @@ public void move(Collection pathsToDelete, // Following code is to maintain this invariant by putting all ancestor // directories of the paths to create. // ancestor paths that are not explicitly added to paths to create - Collection newItems = new ArrayList<>(); + List newItems = new ArrayList<>(); if (pathsToCreate != null) { newItems.addAll(completeAncestry(pathMetaToDDBPathMeta(pathsToCreate))); } + // sort all the new items topmost first. + newItems.sort(PathOrderComparators.TOPMOST_PM_FIRST); if (pathsToDelete != null) { + + List tombstones = new ArrayList<>(pathsToDelete.size()); for (Path meta : pathsToDelete) { - newItems.add(new DDBPathMetadata(PathMetadata.tombstone(meta))); + tombstones.add(new DDBPathMetadata(PathMetadata.tombstone(meta))); } + // sort all the tombstones lowest first. + tombstones.sort(PathOrderComparators.TOPMOST_PM_LAST); + newItems.addAll(tombstones); } processBatchWriteRequest(null, pathMetadataToItem(newItems)); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathOrderComparators.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathOrderComparators.java new file mode 100644 index 0000000000000..d772a779344f5 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathOrderComparators.java @@ -0,0 +1,107 @@ +/* + * 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.util.Comparator; + +import org.apache.hadoop.fs.Path; + +/** + * A comparator of path ordering where those paths which are higher up + * the tree come first. + * This can be used to ensure the sort order of changes. + * + * Policy + *

        + *
      1. higher up entries come first
      2. + *
      3. Root is topmost
      4. + *
      5. repeated sorts do not change the order
      6. + *
      + */ +class PathOrderComparators { + + + static final Comparator TOPMOST_PATH_FIRST + = new TopmostFirst(); + + static final Comparator TOPMOST_PATH_LAST + = new TopmostLast(); + + static final Comparator TOPMOST_PM_FIRST + = new PathMetadataComparator(TOPMOST_PATH_FIRST); + + static final Comparator TOPMOST_PM_LAST + = new PathMetadataComparator(TOPMOST_PATH_LAST); + + private static class TopmostFirst implements Comparator { + + @Override + public int compare(Path pathL, Path pathR) { + // exist fast on equal values. + if (pathL.equals(pathR)) { + return 0; + } + int depthL = pathL.depth(); + int depthR = pathR.depth(); + if (depthL < depthR) { + // left is higher up than the right. + return -1; + } + if (depthR < depthL) { + // right is higher up than the left + return 1; + } + // and if they are of equal depth, use the "classic" comparator + // of paths. + return pathL.compareTo(pathR); + } + } + + private static final class TopmostLast extends TopmostFirst { + + @Override + public int compare(final Path pathL, final Path pathR) { + int compare = super.compare(pathL, pathR); + if (compare < 0) { + return 1; + } + if (compare > 0) { + return -1; + } + return 0; + } + + } + + private static final class PathMetadataComparator implements + Comparator { + + private final Comparator inner; + + public PathMetadataComparator(final Comparator inner) { + this.inner = inner; + } + + @Override + public int compare(final PathMetadata o1, final PathMetadata o2) { + return inner.compare(o1.getFileStatus().getPath(), + o2.getFileStatus().getPath()); + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ProgressiveRenameTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ProgressiveRenameTracker.java new file mode 100644 index 0000000000000..8c4a4281460bd --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ProgressiveRenameTracker.java @@ -0,0 +1,212 @@ +/* + * 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.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.impl.StoreContext; +import org.apache.hadoop.util.DurationInfo; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; +import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.addMoveAncestors; +import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.addMoveDir; + +/** + * This rename tracker progressively updates the metadata store + * as it proceeds, during the parallelized copy operation. + * + * Algorithm + *
        + *
      1. + * As {@link #fileCopied(Path, FileStatus, Path, long, boolean)} callbacks + * are raised, the metastore is updated. + *
      2. + *
      3. + * Including parent entries, as appropriate. + *
      4. + *
      5. + * All directories which have been created are tracked locally, + * to avoid needing to read the store; this is a thread-safe structure. + *
      6. + *
      7. + * The actual update is performed out of any synchronized block. + *
      8. + *
      9. + * The delete list is also (currently) built up. + *
      10. + *
      + *
      + *
      + * 
      + */ +public class ProgressiveRenameTracker extends RenameTracker { + + private final Object lock = new Object(); + + private final StoreContext storeContext; + + private final MetadataStore metadataStore; + + private final Collection sourcePaths = new HashSet<>(); + + private final List destMetas = new ArrayList<>(); + + public ProgressiveRenameTracker( + final StoreContext storeContext, + final MetadataStore metadataStore, + final Path sourceRoot, + final Path dest) { + super(storeContext, sourceRoot, dest); + this.storeContext = storeContext; + this.metadataStore = metadataStore; + } + + public StoreContext getStoreContext() { + return storeContext; + } + + @Override + public void fileCopied( + final Path sourcePath, + final FileStatus sourceStatus, + final Path destPath, + final long blockSize, + final boolean addAncestors) throws IOException { + + // build the list of entries to add in a synchronized block. + List entriesToAdd; + + synchronized (lock) { + checkArgument(!sourcePaths.contains(sourcePath), + "File being renamed is already processed %s", destPath); + // create the file metadata and update the local structures. + PathMetadata newEntry = checkNotNull( + S3Guard.addMoveFile(metadataStore, + sourcePaths, + destMetas, + sourcePath, + destPath, + sourceStatus.getLen(), + blockSize, + getOwner())); + if (addAncestors) { + // add all new ancestors. The null check is to keep code checks + // happy. + entriesToAdd = checkNotNull( + addMoveAncestors( + metadataStore, + sourcePaths, + destMetas, + getSourceRoot(), + sourcePath, + destPath, + getOwner())); + } else { + // no ancestors, so create an empty list instead. + entriesToAdd = new ArrayList<>(1); + } + // add the final entry + entriesToAdd.add(newEntry); + } + + // outside the lock, the entriesToAdd list has all new files to create. + // ...so update the store. + metadataStore.put(entriesToAdd); + } + + @Override + public void directoryMarkerCopied(final FileStatus sourceStatus, + final Path destPath, + final boolean addAncestors) throws IOException { + List entriesToAdd; + synchronized (lock) { + PathMetadata newEntry = checkNotNull( + addMoveDir(metadataStore, sourcePaths, destMetas, + sourceStatus.getPath(), + destPath, getOwner())); + // Ancestor directories may not be listed, so we explicitly add them + if (addAncestors) { + entriesToAdd = checkNotNull( + addMoveAncestors(metadataStore, + sourcePaths, + destMetas, + getSourceRoot(), + sourceStatus.getPath(), + destPath, + getOwner())); + } else { + // no ancestors, so create an empty list instead. + entriesToAdd = new ArrayList<>(1); + } + // add the final entry + entriesToAdd.add(newEntry); + } + // outside the lock, the entriesToAdd list has all new files to create. + // ...so update the store. + try (DurationInfo ignored = new DurationInfo(LOG, false, + "adding %s metastore entries", entriesToAdd.size())) { + metadataStore.move(null, entriesToAdd); + } + } + + @Override + public synchronized void noteSourceDirectoryMoved() throws IOException { + if (!sourcePaths.contains(getSourceRoot())) { + addMoveDir(metadataStore, sourcePaths, destMetas, + getSourceRoot(), + getDest(), getOwner()); + } + } + + /** + * As source objects are deleted, so is the list of entries. + * @param paths keys of objects deleted. + * @throws IOException failure. + */ + @Override + public void sourceObjectsDeleted( + final List paths) throws IOException { + + // delete the paths from the metastore + try (DurationInfo ignored = new DurationInfo(LOG, false, + "Deleting %s metastore entries", paths.size())) { + metadataStore.move(paths, null); + } + } + + @Override + public void completeRename() throws IOException { + // this should all have happened. + metadataStore.move(sourcePaths, destMetas); + super.completeRename(); + } + + @Override + public IOException renameFailed(final Exception ex) { + LOG.debug("Rename has failed", ex); + return super.renameFailed(ex); + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java index 5728fb10ac2f5..81c371b68c109 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java @@ -23,6 +23,7 @@ import java.net.URI; import java.util.ArrayList; import java.util.Collection; +import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Set; @@ -377,17 +378,18 @@ public static void makeDirsOrdered(MetadataStore ms, List dirs, * @param srcPath source path to store * @param dstPath destination path to store * @param owner file owner to use in created records + * @return the new entry created; null if there is no metastore. */ - public static void addMoveDir(MetadataStore ms, Collection srcPaths, + public static PathMetadata addMoveDir(MetadataStore ms, Collection srcPaths, Collection dstMetas, Path srcPath, Path dstPath, String owner) { if (isNullMetadataStore(ms)) { - return; + return null; } assertQualified(srcPath, dstPath); FileStatus dstStatus = createUploadFileStatus(dstPath, true, 0, 0, owner); - addMoveStatus(srcPaths, dstMetas, srcPath, dstStatus); + return addMoveStatus(srcPaths, dstMetas, srcPath, dstStatus); } /** @@ -402,17 +404,18 @@ public static void addMoveDir(MetadataStore ms, Collection srcPaths, * @param size length of file moved * @param blockSize blocksize to associate with destination file * @param owner file owner to use in created records + * @return the new entry created; null if there is no metastore. */ - public static void addMoveFile(MetadataStore ms, Collection srcPaths, + public static PathMetadata addMoveFile(MetadataStore ms, Collection srcPaths, Collection dstMetas, Path srcPath, Path dstPath, long size, long blockSize, String owner) { if (isNullMetadataStore(ms)) { - return; + return null; } assertQualified(srcPath, dstPath); FileStatus dstStatus = createUploadFileStatus(dstPath, false, size, blockSize, owner); - addMoveStatus(srcPaths, dstMetas, srcPath, dstStatus); + return addMoveStatus(srcPaths, dstMetas, srcPath, dstStatus); } /** @@ -425,27 +428,29 @@ public static void addMoveFile(MetadataStore ms, Collection srcPaths, * * As {@link #addMoveFile} and {@link #addMoveDir}, this method adds resulting * metadata to the supplied lists. It does not store in MetadataStore. - * - * @param ms MetadataStore, no-op if it is NullMetadataStore + * @param ms MetadataStore, no-op if it is NullMetadataStore * @param srcPaths stores the source path here * @param dstMetas stores destination metadata here * @param srcRoot source root up to which (exclusive) should we add ancestors * @param srcPath source path of the child to add ancestors * @param dstPath destination path of the child to add ancestors * @param owner Hadoop user name + * @return the list of ancestors added; null if there is no metastore. + * This list is ordered such that the highest entries come in the list first. */ - public static void addMoveAncestors(MetadataStore ms, + public static List addMoveAncestors(MetadataStore ms, Collection srcPaths, Collection dstMetas, Path srcRoot, Path srcPath, Path dstPath, String owner) { if (isNullMetadataStore(ms)) { - return; + return null; } assertQualified(srcRoot, srcPath, dstPath); + LinkedList ancestors = new LinkedList<>(); if (srcPath.equals(srcRoot)) { LOG.debug("Skip moving ancestors of source root directory {}", srcRoot); - return; + return ancestors; } Path parentSrc = srcPath.getParent(); @@ -455,10 +460,13 @@ public static void addMoveAncestors(MetadataStore ms, && !parentSrc.equals(srcRoot) && !srcPaths.contains(parentSrc)) { LOG.debug("Renaming non-listed parent {} to {}", parentSrc, parentDst); - S3Guard.addMoveDir(ms, srcPaths, dstMetas, parentSrc, parentDst, owner); + PathMetadata d = S3Guard.addMoveDir(ms, srcPaths, dstMetas, + parentSrc, parentDst, owner); + ancestors.addFirst(d); parentSrc = parentSrc.getParent(); parentDst = parentDst.getParent(); } + return ancestors; } public static void addAncestors(MetadataStore metadataStore, @@ -480,12 +488,14 @@ public static void addAncestors(MetadataStore metadataStore, metadataStore.put(newDirs); } - private static void addMoveStatus(Collection srcPaths, + private static PathMetadata addMoveStatus(Collection srcPaths, Collection dstMetas, Path srcPath, FileStatus dstStatus) { srcPaths.add(srcPath); - dstMetas.add(new PathMetadata(dstStatus)); + PathMetadata d = new PathMetadata(dstStatus); + dstMetas.add(d); + return d; } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestPathOrderComparators.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestPathOrderComparators.java new file mode 100644 index 0000000000000..9a3db1ab21a5d --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestPathOrderComparators.java @@ -0,0 +1,197 @@ +/* + * 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.util.Comparator; +import java.util.List; + +import org.junit.Test; + +import org.apache.hadoop.fs.Path; + +import static com.google.common.collect.Lists.newArrayList; +import static org.apache.hadoop.fs.s3a.s3guard.PathOrderComparators.TOPMOST_PATH_FIRST; +import static org.apache.hadoop.fs.s3a.s3guard.PathOrderComparators.TOPMOST_PATH_LAST; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.Assert.assertEquals; + +/** + * Test ordering of paths with the comparator matches requirements. + */ +public class TestPathOrderComparators { + + private static final Path ROOT = new Path("s3a://bucket/"); + + public static final Path DIR_A = new Path(ROOT, "dirA"); + + public static final Path DIR_B = new Path(ROOT, "dirB"); + + public static final Path DIR_A_FILE_1 = new Path(DIR_A, "file1"); + + public static final Path DIR_A_FILE_2 = new Path(DIR_A, "file2"); + + public static final Path DIR_B_FILE_3 = new Path(DIR_B, "file3"); + + public static final Path DIR_B_FILE_4 = new Path(DIR_B, "file4"); + + @Test + public void testRootEqual() throws Throwable { + assertComparesEqual(ROOT, ROOT); + } + + @Test + public void testRootFirst() throws Throwable { + assertComparesTopmost(ROOT, DIR_A_FILE_1); + } + + @Test + public void testDirOrdering() throws Throwable { + assertComparesTopmost(DIR_A, DIR_B); + } + + @Test + public void testFilesEqual() throws Throwable { + assertComparesEqual(DIR_A_FILE_1, DIR_A_FILE_1); + } + + @Test + public void testFilesInSameDir() throws Throwable { + assertComparesTopmost(ROOT, DIR_A_FILE_1); + assertComparesTopmost(DIR_A, DIR_A_FILE_1); + assertComparesTopmost(DIR_A, DIR_A_FILE_2); + assertComparesTopmost(DIR_A_FILE_1, DIR_A_FILE_2); + } + + @Test + public void testReversedFiles() throws Throwable { + assertReverseOrder(DIR_A_FILE_1, ROOT); + assertReverseOrder(DIR_A_FILE_1, DIR_A); + assertReverseOrder(DIR_A_FILE_2, DIR_A); + assertReverseOrder(DIR_A_FILE_2, DIR_A_FILE_1); + } + + @Test + public void testFilesAndDifferentShallowDir() throws Throwable { + assertComparesTopmost(DIR_B, DIR_A_FILE_1); + assertComparesTopmost(DIR_A, DIR_B_FILE_3); + } + + @Test + public void testOrderRoot() throws Throwable { + verifySorted(ROOT); + } + + @Test + public void testOrderRootDirs() throws Throwable { + verifySorted(ROOT, DIR_A, DIR_B); + } + + @Test + public void testOrderRootDirsAndFiles() throws Throwable { + verifySorted(ROOT, DIR_A, DIR_B, DIR_A_FILE_1, DIR_A_FILE_2); + } + + @Test + public void testOrderRootDirsAndAllFiles() throws Throwable { + verifySorted(ROOT, DIR_A, DIR_B, + DIR_A_FILE_1, DIR_A_FILE_2, + DIR_B_FILE_3, DIR_B_FILE_4); + } + + @Test + public void testSortOrderConstant() throws Throwable { + List sort1 = verifySorted(ROOT, DIR_A, DIR_B, + DIR_A_FILE_1, DIR_A_FILE_2, + DIR_B_FILE_3, DIR_B_FILE_4); + List sort2 = newArrayList(sort1); + assertSortsTo(sort2, sort1, true); + } + + @Test + public void testSortReverse() throws Throwable { + List sort1 = newArrayList( + ROOT, + DIR_A, + DIR_B, + DIR_A_FILE_1, + DIR_A_FILE_2, + DIR_B_FILE_3, + DIR_B_FILE_4); + List expected = newArrayList( + DIR_B_FILE_4, + DIR_B_FILE_3, + DIR_A_FILE_2, + DIR_A_FILE_1, + DIR_B, + DIR_A, + ROOT); + assertSortsTo(expected, sort1, false); + } + + + private List verifySorted(Path... paths) { + List original = newArrayList(paths); + List sorted = newArrayList(paths); + assertSortsTo(original, sorted, true); + return sorted; + } + + private void assertSortsTo( + final List original, + final List sorted, + boolean topmost) { + sorted.sort(topmost ? TOPMOST_PATH_FIRST : TOPMOST_PATH_LAST); + assertThat(sorted) + .as("Sorted paths") + .containsExactlyElementsOf(original); + } + + private void assertComparesEqual(Path l, Path r) { + assertOrder(0, l, r); + } + + private void assertComparesTopmost(Path l, Path r) { + assertOrder(-1, l, r); + assertOrder(1, r, l); + } + + private void assertReverseOrder(Path l, Path r) { + assertComparesTo(-1, TOPMOST_PATH_LAST, l, r); + assertComparesTo(1, TOPMOST_PATH_LAST, r, l); + } + + private void assertOrder(int res, + Path l, Path r) { + assertComparesTo(res, TOPMOST_PATH_FIRST, l, r); + } + + private void assertComparesTo(final int expected, + final Comparator comparator, + final Path l, final Path r) { + int actual = comparator.compare(l, r); + if (actual < -1) { + actual = -1; + } + if (actual > 1) { + actual = 1; + } + assertEquals("Comparing " + l + " to " + r, + expected, actual); + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/ExtraAssertions.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/ExtraAssertions.java index 9b5fa053aacc8..78a5ee5d1a3b3 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/ExtraAssertions.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/ExtraAssertions.java @@ -34,6 +34,7 @@ import org.apache.hadoop.util.DurationInfo; import static org.apache.hadoop.fs.s3a.S3AUtils.applyLocatedFiles; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; /** @@ -114,7 +115,7 @@ public static void failUnless(boolean condition, /** * Extract the inner cause of an exception. - * @param expected expected class of the cuse + * @param expected expected class of the cause * @param thrown thrown exception. * @param type of the cause * @return the extracted exception. From a11135f4885ecf25378a53b51202cddd3e81d63f Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 7 May 2019 19:13:33 +0100 Subject: [PATCH 15/22] HADOOP-15183 rename all happy. * Improvements to diagnostics, e.g. rename tracker string value. * Metastore interface marks move() args may be null, fix up all implementations to handle this (DynamoDB already did, but not local) * added another rename tests to ITestS3AContractRename which verifies file contents at the end and that the counters are as expected. This was done to debug a test failure, but is low cost and useful enough to retain. * remove from S3Guard class the returning of new elements....that class now matches trunk again. * Stub test class TestProgressiveRenameTracker for which I want to add some unit tests if I can think of some Tested: yes, local and dynamo and auth/not-auth. TODO: * some unit tests for progressive rename tracker (we can, so let's try, because that way yetus will run them) * reviews by others and respond to their feedback Change-Id: I9cfe5313f93399b70ac9aedaf0edafd10aa1db3e --- .../contract/AbstractContractRenameTest.java | 2 +- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 110 +++++++----- .../hadoop/fs/s3a/impl/StoreContext.java | 2 + .../s3guard/DelayedUpdateRenameTracker.java | 4 +- .../fs/s3a/s3guard/DynamoDBMetadataStore.java | 9 +- .../fs/s3a/s3guard/LocalMetadataStore.java | 20 ++- .../hadoop/fs/s3a/s3guard/MetadataStore.java | 6 +- .../fs/s3a/s3guard/NullMetadataStore.java | 2 +- .../fs/s3a/s3guard/PathOrderComparators.java | 27 ++- .../s3a/s3guard/ProgressiveRenameTracker.java | 156 +++++++++--------- .../hadoop/fs/s3a/s3guard/RenameTracker.java | 18 +- .../apache/hadoop/fs/s3a/s3guard/S3Guard.java | 45 ++--- .../contract/s3a/ITestS3AContractRename.java | 71 +++++++- .../s3a/impl/ITestPartialRenamesDeletes.java | 29 +++- .../s3guard/TestProgressiveRenameTracker.java | 25 +++ 15 files changed, 356 insertions(+), 170 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestProgressiveRenameTracker.java 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 5b76a753de170..2751294beb92c 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 @@ -268,7 +268,7 @@ public void testRenamePopulatesFileAncestors() throws IOException { * @param dst the destination root to move * @param nestedPath the nested path to move */ - private void validateAncestorsMoved(Path src, Path dst, String nestedPath) + protected void validateAncestorsMoved(Path src, Path dst, String nestedPath) throws IOException { assertIsDirectory(dst); assertPathDoesNotExist("src path should not exist", path(src + nestedPath)); 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 a95176ec242b8..c5ee9808242ae 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 @@ -27,6 +27,9 @@ import java.nio.file.AccessDeniedException; import java.text.DateFormat; import java.text.SimpleDateFormat; +import java.time.Instant; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -44,6 +47,7 @@ import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; import java.util.function.BiFunction; import javax.annotation.Nullable; @@ -1109,9 +1113,13 @@ public FSDataOutputStream append(Path f, int bufferSize, * @throws IOException on IO failure * @return true if rename is successful */ + @Retries.RetryTranslated public boolean rename(Path src, Path dst) throws IOException { - try { - return innerRename(src, dst); + try(DurationInfo ignored = new DurationInfo(LOG, false, + "rename(%s, %s", src, dst)) { + long bytesCopied = innerRename(src, dst); + LOG.debug("Copied {} bytes", bytesCopied); + return true; } catch (AmazonClientException e) { throw translateException("rename(" + src +", " + dst + ")", src, e); } catch (RenameFailedException e) { @@ -1137,12 +1145,13 @@ public boolean rename(Path src, Path dst) throws IOException { * rename was not met. This means work didn't happen; it's not something * which is reported upstream to the FileSystem APIs, for which the semantics * of "false" are pretty vague. + * @return the number of bytes copied. * @throws FileNotFoundException there's no source file. * @throws IOException on IO failure. * @throws AmazonClientException on failures inside the AWS SDK */ @Retries.RetryMixed - private boolean innerRename(Path source, Path dest) + private long innerRename(Path source, Path dest) throws RenameFailedException, FileNotFoundException, IOException, AmazonClientException { Path src = qualify(source); @@ -1228,6 +1237,7 @@ private boolean innerRename(Path source, Path dest) metadataStore.initiateRenameOperation( createStoreContext(), src, srcStatus, dest); + final AtomicLong bytesCopied = new AtomicLong(); int renameParallelLimit = 10; final List> activeCopies = new ArrayList<>(renameParallelLimit); @@ -1235,7 +1245,7 @@ private boolean innerRename(Path source, Path dest) // the list. final FunctionsRaisingIOE.FunctionRaisingIOE completeActiveCopies = (String reason) -> { - LOG.debug("Waiting for {} active copies to complete during {}", + LOG.debug("Waiting for {} active copies to complete: {}", activeCopies.size(), reason); waitForCompletion(activeCopies); activeCopies.clear(); @@ -1244,6 +1254,7 @@ private boolean innerRename(Path source, Path dest) try { if (srcStatus.isFile()) { + // the source is a file. Path copyDestinationPath = dst; String copyDestinationKey = dstKey; if (dstStatus != null && dstStatus.isDirectory()) { @@ -1258,9 +1269,13 @@ private boolean innerRename(Path source, Path dest) // destination either does not exist or is a file to overwrite. LOG.debug("rename: renaming file {} to {}", src, copyDestinationPath); copySourceAndUpdateTracker(renameTracker, - src, srcKey, srcStatus, - copyDestinationPath, copyDestinationKey, + src, + srcKey, + srcStatus, + copyDestinationPath, + copyDestinationKey, false); + bytesCopied.addAndGet(srcStatus.getLen()); // delete the source deleteObjectAtPath(src, srcKey, true); // and update the tracker @@ -1283,16 +1298,30 @@ private boolean innerRename(Path source, Path dest) final List keysToDelete = new ArrayList<>(); final List pathsToDelete = new ArrayList<>(); // the operation to update the lists of keys and paths. - final BiFunction - toDelete = (Path path, String key) -> { - pathsToDelete.add(path); - keysToDelete.add(new DeleteObjectsRequest.KeyVersion(key)); - return null; - }; + final BiFunction queueToDelete = + (Path path, String key) -> { + pathsToDelete.add(path); + keysToDelete.add(new DeleteObjectsRequest.KeyVersion(key)); + return null; + }; + // the operation to block waiting for ay active copies to finish + // then delete all queued keys + paths to delete. + final FunctionsRaisingIOE.FunctionRaisingIOE + completeActiveCopiesAndDeleteSources = + (String reason) -> { + completeActiveCopies.apply(reason); + removeSourceObjects(renameTracker, keysToDelete, pathsToDelete); + // now reset the lists. + keysToDelete.clear(); + pathsToDelete.clear(); + return null; + }; if (dstStatus != null && dstStatus.isEmptyDirectory() == Tristate.TRUE) { - // delete unnecessary fake directory. - toDelete.apply(dstStatus.getPath(), dstKey); + // delete unnecessary fake directory at the destination. + // this MUST be done before anything else so that + // rollback code doesn't get confused. + deleteObjectAtPath(dstStatus.getPath(), dstKey, false); } Path parentPath = keyToQualifiedPath(srcKey); @@ -1308,45 +1337,40 @@ private boolean innerRename(Path source, Path dest) dstKey + key.substring(srcKey.length()); Path childSourcePath = keyToQualifiedPath(key); - toDelete.apply(childSourcePath, key); + queueToDelete.apply(childSourcePath, key); Path childDestPath = keyToQualifiedPath(newDstKey); - // set up for async operation but run in sync mode initially. - // we will need to parallelize updates to metastore - // for that. + // queue the copy for execution. CompletableFuture copy = submit(boundedThreadPool, () -> copySourceAndUpdateTracker(renameTracker, - childSourcePath, key, status, - childDestPath, newDstKey, + childSourcePath, + key, + status, + childDestPath, + newDstKey, true)); + bytesCopied.addAndGet(srcStatus.getLen()); activeCopies.add(copy); if (activeCopies.size() == renameParallelLimit) { LOG.debug("Waiting for active copies to complete"); - completeActiveCopies.apply("Size threshold reached"); + completeActiveCopies.apply("batch threshold reached"); } if (keysToDelete.size() == MAX_ENTRIES_TO_DELETE) { - // time to queue a delete. - // first wait for the copies in progress to finish - // (deleting a file mid-copy would not be good) - completeActiveCopies.apply("before a delete"); - removeSourceObjects(renameTracker, keysToDelete, pathsToDelete); - // now reset the lists. - keysToDelete.clear(); - pathsToDelete.clear(); + // finish ongoing copies then delete all queued keys. + completeActiveCopiesAndDeleteSources.apply("paged delete"); } } - // end of iteration await final set of copies - completeActiveCopies.apply("final copies"); + // end of iteration - // end of iteration -the final delete. + // await final set of copies and then delete // This will notify the renameTracker that these objects // have been deleted. - removeSourceObjects(renameTracker, keysToDelete, pathsToDelete); + completeActiveCopiesAndDeleteSources.apply("final copy and delete"); // We moved all the children, now move the top-level dir // Empty directory should have been added as the object summary - renameTracker.noteSourceDirectoryMoved(); + renameTracker.moveSourceDirectory(); } } catch (AmazonClientException | IOException ex) { // rename failed. @@ -1354,6 +1378,11 @@ private boolean innerRename(Path source, Path dest) try { completeActiveCopies.apply("failure handling"); } catch (IOException e) { + // a failure to update the metastore after a rename failure is what + // we'd see on a network problem, expired credentials and other + // unrecoverable errors. + // Downgrading to warn because an exception is already + // about to be thrown. LOG.warn("While completing all active copies", e); } @@ -1370,7 +1399,7 @@ private boolean innerRename(Path source, Path dest) deleteUnnecessaryFakeDirectories(dst.getParent()); maybeCreateFakeParentDirectory(src); } - return true; + return bytesCopied.get(); } /** @@ -1381,7 +1410,8 @@ private boolean innerRename(Path source, Path dest) * @throws IOException failure */ @Retries.RetryMixed - private void removeSourceObjects(final RenameTracker renameTracker, + private void removeSourceObjects( + final RenameTracker renameTracker, final List keysToDelete, final List pathsToDelete) throws IOException { @@ -2166,8 +2196,7 @@ void removeKeys( // what to do if an IOE was raised? Given an exception was being // raised anyway, and the failures are logged, do nothing. Triple, List, List>> results = - new MultiObjectDeleteSupport( - createStoreContext()) + new MultiObjectDeleteSupport(createStoreContext()) .processDeleteFailure(ex, keysToDelete); undeletedObjectsOnFailure.addAll(results.getMiddle()); throw ex; @@ -2650,8 +2679,11 @@ S3AFileStatus innerGetFileStatus(final Path f, Set tombstones = Collections.emptySet(); if (pm != null) { if (pm.isDeleted()) { + OffsetDateTime deletedAt = OffsetDateTime.ofInstant( + Instant.ofEpochMilli(pm.getFileStatus().getModificationTime()), + ZoneOffset.UTC); throw new FileNotFoundException("Path " + f + " is recorded as " + - "deleted by S3Guard"); + "deleted by S3Guard at " + deletedAt.toString()); } // if ms is not authoritative, check S3 if there's any recent diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java index b0397a3f13828..3d823828c7729 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java @@ -123,6 +123,8 @@ public class StoreContext { /** * Instantiate. + * No attempt to use a builder here as outside tests + * this should only be created in the S3AFileSystem. */ public StoreContext(final URI fsURI, final String bucket, diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameTracker.java index 844adab027e0d..b4e5e2f5f9cf8 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameTracker.java @@ -59,7 +59,7 @@ public DelayedUpdateRenameTracker( final MetadataStore metadataStore, final Path sourceRoot, final Path dest) { - super(storeContext, sourceRoot, dest); + super("DelayedUpdateRenameTracker", storeContext, sourceRoot, dest); this.storeContext = storeContext; this.metadataStore = metadataStore; } @@ -115,7 +115,7 @@ public synchronized void directoryMarkerCopied(final FileStatus sourceStatus, } @Override - public synchronized void noteSourceDirectoryMoved() throws IOException { + public synchronized void moveSourceDirectory() throws IOException { if (!sourcePaths.contains(getSourceRoot())) { addMoveDir(metadataStore, sourcePaths, destMetas, getSourceRoot(), 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 02855469cba97..a36347d66b26f 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 @@ -18,6 +18,7 @@ package org.apache.hadoop.fs.s3a.s3guard; +import javax.annotation.Nullable; import java.io.FileNotFoundException; import java.io.IOException; import java.io.InterruptedIOException; @@ -777,8 +778,9 @@ Collection completeAncestry( */ @Override @Retries.RetryTranslated - public void move(Collection pathsToDelete, - Collection pathsToCreate) throws IOException { + public void move( + @Nullable Collection pathsToDelete, + @Nullable Collection pathsToCreate) throws IOException { if (pathsToDelete == null && pathsToCreate == null) { return; } @@ -802,7 +804,6 @@ public void move(Collection pathsToDelete, // sort all the new items topmost first. newItems.sort(PathOrderComparators.TOPMOST_PM_FIRST); if (pathsToDelete != null) { - List tombstones = new ArrayList<>(pathsToDelete.size()); for (Path meta : pathsToDelete) { tombstones.add(new DDBPathMetadata(PathMetadata.tombstone(meta))); @@ -1789,7 +1790,7 @@ public Invoker getInvoker() { public RenameTracker initiateRenameOperation(final StoreContext storeContext, final Path source, final FileStatus srcStatus, final Path dest) throws IOException { - return new DelayedUpdateRenameTracker(storeContext, this, source, dest); + return new ProgressiveRenameTracker(storeContext, this, source, dest); } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java index 798443b68e715..31a229f8fdf50 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java @@ -18,6 +18,8 @@ package org.apache.hadoop.fs.s3a.s3guard; +import javax.annotation.Nullable; + import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; @@ -37,6 +39,7 @@ import java.io.IOException; import java.net.URI; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.LinkedList; import java.util.Map; @@ -191,14 +194,17 @@ public synchronized DirListingMetadata listChildren(Path p) throws } @Override - public void move(Collection pathsToDelete, - Collection pathsToCreate) throws IOException { + public void move( + @Nullable Collection pathsToDelete, + @Nullable Collection pathsToCreate) throws IOException { LOG.info("Move {} to {}", pathsToDelete, pathsToCreate); - Preconditions.checkNotNull(pathsToDelete, "pathsToDelete is null"); - Preconditions.checkNotNull(pathsToCreate, "pathsToCreate is null"); - Preconditions.checkArgument(pathsToDelete.size() == pathsToCreate.size(), - "Must supply same number of paths to delete/create."); + if (pathsToCreate == null) { + pathsToCreate = Collections.emptyList(); + } + if (pathsToDelete == null) { + pathsToDelete = Collections.emptyList(); + } // I feel dirty for using reentrant lock. :-| synchronized (this) { @@ -536,6 +542,6 @@ DirListingMetadata getDirListingMeta(Path p){ public RenameTracker initiateRenameOperation(final StoreContext storeContext, final Path source, final FileStatus srcStatus, final Path dest) throws IOException { - return new DelayedUpdateRenameTracker(storeContext, this, source, dest); + return new ProgressiveRenameTracker(storeContext, this, source, dest); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java index e6acdb75c2c2f..c880b83b177e3 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java @@ -18,6 +18,7 @@ package org.apache.hadoop.fs.s3a.s3guard; +import javax.annotation.Nullable; import java.io.Closeable; import java.io.IOException; import java.util.Collection; @@ -155,8 +156,9 @@ PathMetadata get(Path path, boolean wantEmptyDirectoryFlag) * (). * @throws IOException if there is an error */ - void move(Collection pathsToDelete, - Collection pathsToCreate) throws IOException; + void move( + @Nullable Collection pathsToDelete, + @Nullable Collection pathsToCreate) throws IOException; /** * Saves metadata for exactly one path. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java index a81bfac3aea37..6abb33323bb1b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java @@ -137,7 +137,7 @@ private NullRenameTracker( final StoreContext storeContext, final Path source, final Path dest) { - super(storeContext, source, dest); + super("rename tracker", storeContext, source, dest); } @Override diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathOrderComparators.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathOrderComparators.java index d772a779344f5..770842ea01927 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathOrderComparators.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathOrderComparators.java @@ -34,18 +34,35 @@ *
    • repeated sorts do not change the order
    • * */ +@SuppressWarnings("ComparatorNotSerializable") class PathOrderComparators { + /** + * The shallowest paths come first. + * This is to be used when adding entries. + */ static final Comparator TOPMOST_PATH_FIRST = new TopmostFirst(); + /** + * The leaves come first. + * This is to be used when deleting entries. + */ static final Comparator TOPMOST_PATH_LAST = new TopmostLast(); + /** + * The shallowest paths come first. + * This is to be used when adding entries. + */ static final Comparator TOPMOST_PM_FIRST = new PathMetadataComparator(TOPMOST_PATH_FIRST); + /** + * The leaves come first. + * This is to be used when deleting entries. + */ static final Comparator TOPMOST_PM_LAST = new PathMetadataComparator(TOPMOST_PATH_LAST); @@ -73,6 +90,11 @@ public int compare(Path pathL, Path pathR) { } } + /** + * Compare the topmost last. + * For some reason the .reverse() option wasn't giving the + * correct outcome. + */ private static final class TopmostLast extends TopmostFirst { @Override @@ -89,12 +111,15 @@ public int compare(final Path pathL, final Path pathR) { } + /** + * Compare on path status. + */ private static final class PathMetadataComparator implements Comparator { private final Comparator inner; - public PathMetadataComparator(final Comparator inner) { + private PathMetadataComparator(final Comparator inner) { this.inner = inner; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ProgressiveRenameTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ProgressiveRenameTracker.java index 8c4a4281460bd..b429c52128821 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ProgressiveRenameTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ProgressiveRenameTracker.java @@ -30,7 +30,6 @@ import org.apache.hadoop.util.DurationInfo; import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Preconditions.checkNotNull; import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.addMoveAncestors; import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.addMoveDir; @@ -42,7 +41,7 @@ *
        *
      1. * As {@link #fileCopied(Path, FileStatus, Path, long, boolean)} callbacks - * are raised, the metastore is updated. + * are raised, the metastore is updated with the new file entry. *
      2. *
      3. * Including parent entries, as appropriate. @@ -55,7 +54,10 @@ * The actual update is performed out of any synchronized block. *
      4. *
      5. - * The delete list is also (currently) built up. + * When deletes are executed, the store is also updated. + *
      6. + *
      7. + * And at the completion of a successful rename, the source directory is also removed. *
      8. *
      *
      @@ -64,13 +66,15 @@
        */
       public class ProgressiveRenameTracker extends RenameTracker {
       
      -  private final Object lock = new Object();
      -
      -  private final StoreContext storeContext;
      -
         private final MetadataStore metadataStore;
       
      -  private final Collection sourcePaths = new HashSet<>();
      +  /**
      +   * The collection of paths to delete; this is added as individual files
      +   * are renamed.
      +   * The metastore is only updated with these entries after the DELETE
      +   * call succeeds.
      +   */
      +  private final Collection pathsToDelete = new HashSet<>();
       
         private final List destMetas = new ArrayList<>();
       
      @@ -79,15 +83,24 @@ public ProgressiveRenameTracker(
             final MetadataStore metadataStore,
             final Path sourceRoot,
             final Path dest) {
      -    super(storeContext, sourceRoot, dest);
      -    this.storeContext = storeContext;
      +    super("ProgressiveRenameTracker",
      +        storeContext, sourceRoot, dest);
           this.metadataStore = metadataStore;
         }
       
      -  public StoreContext getStoreContext() {
      -    return storeContext;
      -  }
      -
      +  /**
      +   * When a file is copied, any ancestors
      +   * are calculated and then the store is updated with
      +   * the destination entries.
      +   *
      +   * The source entries are added to the {@link #pathsToDelete} list.
      +   * @param sourcePath path of source
      +   * @param sourceStatus status of source.
      +   * @param destPath destination path.
      +   * @param blockSize block size.
      +   * @param addAncestors should ancestors be added?
      +   * @throws IOException failure
      +   */
         @Override
         public void fileCopied(
             final Path sourcePath,
      @@ -97,72 +110,70 @@ public void fileCopied(
             final boolean addAncestors) throws IOException {
       
           // build the list of entries to add in a synchronized block.
      -    List entriesToAdd;
      -
      -    synchronized (lock) {
      -      checkArgument(!sourcePaths.contains(sourcePath),
      +    final List entriesToAdd = new ArrayList<>(1);
      +    LOG.debug("Updating store with copied file {}", sourcePath);
      +    synchronized (this) {
      +      checkArgument(!pathsToDelete.contains(sourcePath),
                 "File being renamed is already processed %s", destPath);
             // create the file metadata and update the local structures.
      -      PathMetadata newEntry = checkNotNull(
      -          S3Guard.addMoveFile(metadataStore,
      -              sourcePaths,
      -              destMetas,
      -              sourcePath,
      -              destPath,
      -              sourceStatus.getLen(),
      -              blockSize,
      -              getOwner()));
      +      S3Guard.addMoveFile(metadataStore,
      +          pathsToDelete,
      +          entriesToAdd,
      +          sourcePath,
      +          destPath,
      +          sourceStatus.getLen(),
      +          blockSize,
      +          getOwner());
      +      LOG.debug("New metastore entry : {}", entriesToAdd.get(0));
             if (addAncestors) {
      -        // add all new ancestors. The null check is to keep code checks
      -        // happy.
      -        entriesToAdd = checkNotNull(
      -            addMoveAncestors(
      -                metadataStore,
      -                sourcePaths,
      -                destMetas,
      -                getSourceRoot(),
      -                sourcePath,
      -                destPath,
      -                getOwner()));
      -      } else {
      -        // no ancestors, so create an empty list instead.
      -        entriesToAdd = new ArrayList<>(1);
      +        // add all new ancestors.
      +        addMoveAncestors(
      +            metadataStore,
      +            pathsToDelete,
      +            entriesToAdd,
      +            getSourceRoot(),
      +            sourcePath,
      +            destPath,
      +            getOwner());
             }
      -      // add the final entry
      -      entriesToAdd.add(newEntry);
           }
       
           // outside the lock, the entriesToAdd list has all new files to create.
           // ...so update the store.
      -    metadataStore.put(entriesToAdd);
      +    metadataStore.move(null, entriesToAdd);
         }
       
      +  /**
      +   * A directory marker has been added.
      +   * Add the new entry and record the source path as another entry to delete.
      +   * @param sourceStatus status of source.
      +   * @param destPath destination path.
      +   * @param addAncestors should ancestors be added?
      +   * @throws IOException failure.
      +   */
         @Override
         public void directoryMarkerCopied(final FileStatus sourceStatus,
             final Path destPath,
             final boolean addAncestors) throws IOException {
      -    List entriesToAdd;
      -    synchronized (lock) {
      -      PathMetadata newEntry = checkNotNull(
      -          addMoveDir(metadataStore, sourcePaths, destMetas,
      -              sourceStatus.getPath(),
      -              destPath, getOwner()));
      +    // this list is created on demand.
      +    final List entriesToAdd = new ArrayList<>(1);
      +    synchronized (this) {
      +      addMoveDir(metadataStore,
      +          pathsToDelete,
      +          entriesToAdd,
      +          sourceStatus.getPath(),
      +          destPath,
      +          getOwner());
             // Ancestor directories may not be listed, so we explicitly add them
             if (addAncestors) {
      -        entriesToAdd = checkNotNull(
      -            addMoveAncestors(metadataStore,
      -                sourcePaths,
      -                destMetas,
      -                getSourceRoot(),
      -                sourceStatus.getPath(),
      -                destPath,
      -                getOwner()));
      -      } else {
      -        // no ancestors, so create an empty list instead.
      -        entriesToAdd = new ArrayList<>(1);
      +        addMoveAncestors(metadataStore,
      +            pathsToDelete,
      +            entriesToAdd,
      +            getSourceRoot(),
      +            sourceStatus.getPath(),
      +            destPath,
      +            getOwner());
             }
      -      // add the final entry
      -      entriesToAdd.add(newEntry);
           }
           // outside the lock, the entriesToAdd list has all new files to create.
           // ...so update the store.
      @@ -173,11 +184,12 @@ public void directoryMarkerCopied(final FileStatus sourceStatus,
         }
       
         @Override
      -  public synchronized void noteSourceDirectoryMoved() throws IOException {
      -    if (!sourcePaths.contains(getSourceRoot())) {
      -      addMoveDir(metadataStore, sourcePaths, destMetas,
      +  public synchronized void moveSourceDirectory() throws IOException {
      +    if (!pathsToDelete.contains(getSourceRoot())) {
      +      addMoveDir(metadataStore, pathsToDelete, destMetas,
                 getSourceRoot(),
      -          getDest(), getOwner());
      +          getDest(),
      +          getOwner());
           }
         }
       
      @@ -192,7 +204,7 @@ public void sourceObjectsDeleted(
       
           // delete the paths from the metastore
           try (DurationInfo ignored = new DurationInfo(LOG, false,
      -        "Deleting %s metastore entries", paths.size())) {
      +        "delete %s metastore entries", paths.size())) {
             metadataStore.move(paths, null);
           }
         }
      @@ -200,13 +212,9 @@ public void sourceObjectsDeleted(
         @Override
         public void completeRename() throws IOException {
           // this should all have happened.
      -    metadataStore.move(sourcePaths, destMetas);
      +    LOG.debug("Rename completed for {}", this);
      +    metadataStore.move(pathsToDelete, destMetas);
           super.completeRename();
         }
       
      -  @Override
      -  public IOException renameFailed(final Exception ex) {
      -    LOG.debug("Rename has failed", ex);
      -    return super.renameFailed(ex);
      -  }
       }
      diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RenameTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RenameTracker.java
      index 75fe0b36ba280..ba8b8f5ffd6b0 100644
      --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RenameTracker.java
      +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RenameTracker.java
      @@ -56,21 +56,34 @@ public abstract class RenameTracker extends StoreOperation {
          */
         private final DurationInfo durationInfo;
       
      +  /**
      +   * Generated name for strings.
      +   */
      +  private final String name;
      +
         /**
          * constructor.
      +   * @param name tracker name for logs.
          * @param storeContext store context.
          * @param sourceRoot source path.
          * @param dest destination path.
          */
         protected RenameTracker(
      +      final String name,
             final StoreContext storeContext,
             final Path sourceRoot,
             final Path dest) {
           super(storeContext);
           this.sourceRoot = sourceRoot;
           this.dest = dest;
      +    this.name = String.format("%s (%s, %s)", name, sourceRoot, dest);
           durationInfo = new DurationInfo(LOG, false,
      -        "rename(%s, %s)", sourceRoot, dest);
      +        name +" (%s, %s)", sourceRoot, dest);
      +  }
      +
      +  @Override
      +  public String toString() {
      +    return name;
         }
       
         public Path getSourceRoot() {
      @@ -139,7 +152,7 @@ public IOException deleteFailed(
          * Top level directory move.
          * @throws IOException on failure
          */
      -  public void noteSourceDirectoryMoved() throws IOException {
      +  public void moveSourceDirectory() throws IOException {
       
         }
       
      @@ -183,6 +196,7 @@ protected void noteRenameFinished() {
          * @return an IOException to throw in an exception.
          */
         public IOException renameFailed(Exception ex) {
      +    LOG.debug("Rename has failed", ex);
           noteRenameFinished();
           return convertToIOException(ex);
         }
      diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java
      index 81c371b68c109..66e67f294c875 100644
      --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java
      +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java
      @@ -23,20 +23,16 @@
       import java.net.URI;
       import java.util.ArrayList;
       import java.util.Collection;
      -import java.util.LinkedList;
       import java.util.List;
       import java.util.Map;
       import java.util.Set;
       import java.util.stream.Collectors;
       
      -import com.amazonaws.services.s3.model.DeleteObjectsRequest;
      -import com.amazonaws.services.s3.model.MultiObjectDeleteException;
       import com.google.common.annotations.VisibleForTesting;
       import com.google.common.base.Preconditions;
       import org.slf4j.Logger;
       import org.slf4j.LoggerFactory;
       
      -import org.apache.commons.lang3.tuple.Pair;
       import org.apache.hadoop.classification.InterfaceAudience;
       import org.apache.hadoop.classification.InterfaceStability;
       import org.apache.hadoop.conf.Configuration;
      @@ -378,18 +374,17 @@ public static void makeDirsOrdered(MetadataStore ms, List dirs,
          * @param srcPath  source path to store
          * @param dstPath  destination path to store
          * @param owner file owner to use in created records
      -   * @return the new entry created; null if there is no metastore.
          */
      -  public static PathMetadata addMoveDir(MetadataStore ms, Collection srcPaths,
      +  public static void addMoveDir(MetadataStore ms, Collection srcPaths,
             Collection dstMetas, Path srcPath, Path dstPath,
             String owner) {
           if (isNullMetadataStore(ms)) {
      -      return null;
      +      return;
           }
           assertQualified(srcPath, dstPath);
       
           FileStatus dstStatus = createUploadFileStatus(dstPath, true, 0, 0, owner);
      -    return addMoveStatus(srcPaths, dstMetas, srcPath, dstStatus);
      +    addMoveStatus(srcPaths, dstMetas, srcPath, dstStatus);
         }
       
         /**
      @@ -404,18 +399,17 @@ public static PathMetadata addMoveDir(MetadataStore ms, Collection srcPath
          * @param size length of file moved
          * @param blockSize  blocksize to associate with destination file
          * @param owner file owner to use in created records
      -   * @return the new entry created; null if there is no metastore.
          */
      -  public static PathMetadata addMoveFile(MetadataStore ms, Collection srcPaths,
      +  public static void addMoveFile(MetadataStore ms, Collection srcPaths,
             Collection dstMetas, Path srcPath, Path dstPath,
             long size, long blockSize, String owner) {
           if (isNullMetadataStore(ms)) {
      -      return null;
      +      return;
           }
           assertQualified(srcPath, dstPath);
           FileStatus dstStatus = createUploadFileStatus(dstPath, false,
               size, blockSize, owner);
      -    return addMoveStatus(srcPaths, dstMetas, srcPath, dstStatus);
      +    addMoveStatus(srcPaths, dstMetas, srcPath, dstStatus);
         }
       
         /**
      @@ -428,29 +422,27 @@ public static PathMetadata addMoveFile(MetadataStore ms, Collection srcPat
          *
          * As {@link #addMoveFile} and {@link #addMoveDir}, this method adds resulting
          * metadata to the supplied lists. It does not store in MetadataStore.
      -   *  @param ms MetadataStore, no-op if it is NullMetadataStore
      +   *
      +   * @param ms MetadataStore, no-op if it is NullMetadataStore
          * @param srcPaths stores the source path here
          * @param dstMetas stores destination metadata here
          * @param srcRoot source root up to which (exclusive) should we add ancestors
          * @param srcPath source path of the child to add ancestors
          * @param dstPath destination path of the child to add ancestors
          * @param owner Hadoop user name
      -   * @return the list of ancestors added; null if there is no metastore.
      -   * This list is ordered such that the highest entries come in the list first.
          */
      -  public static List addMoveAncestors(MetadataStore ms,
      +  public static void addMoveAncestors(MetadataStore ms,
             Collection srcPaths, Collection dstMetas,
             Path srcRoot, Path srcPath, Path dstPath, String owner) {
           if (isNullMetadataStore(ms)) {
      -      return null;
      +      return;
           }
       
           assertQualified(srcRoot, srcPath, dstPath);
      -    LinkedList ancestors = new LinkedList<>();
       
           if (srcPath.equals(srcRoot)) {
             LOG.debug("Skip moving ancestors of source root directory {}", srcRoot);
      -      return ancestors;
      +      return;
           }
       
           Path parentSrc = srcPath.getParent();
      @@ -460,13 +452,10 @@ public static List addMoveAncestors(MetadataStore ms,
               && !parentSrc.equals(srcRoot)
               && !srcPaths.contains(parentSrc)) {
             LOG.debug("Renaming non-listed parent {} to {}", parentSrc, parentDst);
      -      PathMetadata d = S3Guard.addMoveDir(ms, srcPaths, dstMetas,
      -          parentSrc, parentDst, owner);
      -      ancestors.addFirst(d);
      +      S3Guard.addMoveDir(ms, srcPaths, dstMetas, parentSrc, parentDst, owner);
             parentSrc = parentSrc.getParent();
             parentDst = parentDst.getParent();
           }
      -    return ancestors;
         }
       
         public static void addAncestors(MetadataStore metadataStore,
      @@ -485,17 +474,17 @@ public static void addAncestors(MetadataStore metadataStore,
             }
             parent = parent.getParent();
           }
      -    metadataStore.put(newDirs);
      +    if (!newDirs.isEmpty()) {
      +      metadataStore.put(newDirs);
      +    }
         }
       
      -  private static PathMetadata addMoveStatus(Collection srcPaths,
      +  private static void addMoveStatus(Collection srcPaths,
             Collection dstMetas,
             Path srcPath,
             FileStatus dstStatus) {
           srcPaths.add(srcPath);
      -    PathMetadata d = new PathMetadata(dstStatus);
      -    dstMetas.add(d);
      -    return d;
      +    dstMetas.add(new PathMetadata(dstStatus));
         }
       
         /**
      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 433964998cd9a..4a050fbc11504 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
      @@ -18,14 +18,23 @@
       
       package org.apache.hadoop.fs.contract.s3a;
       
      +import org.junit.Test;
      +import org.slf4j.Logger;
      +import org.slf4j.LoggerFactory;
      +
       import org.apache.hadoop.conf.Configuration;
       import org.apache.hadoop.fs.contract.AbstractContractRenameTest;
       import org.apache.hadoop.fs.contract.AbstractFSContract;
       import org.apache.hadoop.fs.FileSystem;
       import org.apache.hadoop.fs.Path;
      +import org.apache.hadoop.fs.s3a.S3AFileSystem;
      +import org.apache.hadoop.fs.s3a.S3ATestUtils;
      +import org.apache.hadoop.fs.s3a.Statistic;
       
       import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
      +import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyFileContents;
       import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset;
      +import static org.apache.hadoop.fs.s3a.S3ATestConstants.S3A_TEST_TIMEOUT;
       import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeEnableS3Guard;
       
       /**
      @@ -33,6 +42,15 @@
        */
       public class ITestS3AContractRename extends AbstractContractRenameTest {
       
      +  public static final Logger LOG = LoggerFactory.getLogger(
      +      ITestS3AContractRename.class);
      +
      +
      +  @Override
      +  protected int getTestTimeoutMillis() {
      +    return S3A_TEST_TIMEOUT;
      +  }
      +
         /**
          * Create a configuration, possibly patching in S3Guard options.
          * @return a configuration
      @@ -50,6 +68,12 @@ protected AbstractFSContract createContract(Configuration conf) {
           return new S3AContract(conf);
         }
       
      +  @Override
      +  public void teardown() throws Exception {
      +    describe("\nTeardown\n");
      +    super.teardown();
      +  }
      +
         @Override
         public void testRenameDirIntoExistingDir() throws Throwable {
           describe("Verify renaming a dir into an existing dir puts the files"
      @@ -64,12 +88,55 @@ public void testRenameDirIntoExistingDir() throws Throwable {
           Path destDir = path("dest");
       
           Path destFilePath = new Path(destDir, "dest-512.txt");
      -    byte[] destDateset = dataset(512, 'A', 'Z');
      -    writeDataset(fs, destFilePath, destDateset, destDateset.length, 1024,
      +    byte[] destDataset = dataset(512, 'A', 'Z');
      +    writeDataset(fs, destFilePath, destDataset, destDataset.length, 1024,
               false);
           assertIsFile(destFilePath);
       
           boolean rename = fs.rename(srcDir, destDir);
           assertFalse("s3a doesn't support rename to non-empty directory", rename);
         }
      +
      +  /**
      +   * Test that after renaming, the nested file is moved along with all its
      +   * ancestors. It is similar to {@link #testRenamePopulatesDirectoryAncestors}.
      +   *
      +   * This is an extension testRenamePopulatesFileAncestors
      +   * of the superclass version which does better
      +   * logging of the state of the store before the assertions.
      +   */
      +  @Test
      +  public void testRenamePopulatesFileAncestors2() throws Exception {
      +    final S3AFileSystem fs = (S3AFileSystem)getFileSystem();
      +    Path base = path("testRenamePopulatesFileAncestors2");
      +    final Path src = new Path(base, "src");
      +    Path dest = new Path(base, "dest");
      +    fs.mkdirs(src);
      +    final String nestedFile = "/dir1/dir2/dir3/fileA";
      +    // size of file to create
      +    int filesize = 16 * 1024;
      +    byte[] srcDataset = dataset(filesize, 'a', 'z');
      +    Path srcFile = path(src + nestedFile);
      +    Path destFile = path(dest + nestedFile);
      +    writeDataset(fs, srcFile, srcDataset, srcDataset.length,
      +        1024, false);
      +
      +    S3ATestUtils.MetricDiff fileCopyDiff = new S3ATestUtils.MetricDiff(fs,
      +        Statistic.FILES_COPIED);
      +    S3ATestUtils.MetricDiff fileCopyBytes = new S3ATestUtils.MetricDiff(fs,
      +        Statistic.FILES_COPIED_BYTES);
      +
      +    fs.rename(src, dest);
      +
      +    describe("Rename has completed, examining data under " + base);
      +    fileCopyDiff.assertDiffEquals("Number of files copied", 1);
      +    fileCopyBytes.assertDiffEquals("Number of bytes copied", filesize);
      +    // log everything in the base directory.
      +    S3ATestUtils.lsR(fs, base, true);
      +    // look at the data.
      +    verifyFileContents(fs, destFile, srcDataset);
      +    describe("validating results");
      +    validateAncestorsMoved(src, dest, nestedFile);
      +
      +  }
       }
      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 fe4ade3b04598..2f9b2d2f6c9d8 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
      @@ -42,6 +42,7 @@
       import org.slf4j.LoggerFactory;
       
       import org.apache.hadoop.conf.Configuration;
      +import org.apache.hadoop.fs.FileStatus;
       import org.apache.hadoop.fs.FileSystem;
       import org.apache.hadoop.fs.Path;
       import org.apache.hadoop.fs.contract.ContractTestUtils;
      @@ -485,8 +486,10 @@ public void testRenameDirFailsInDelete() throws Throwable {
           // even with S3Guard on, we expect the destination to match that of our
           // the remote state.
           // the test will exist
      -    assertPathExists("Destination directory of rename", writableDir);
      -    assertIsDirectory(writableDir);
      +    describe("Verify destination directory exists");
      +    FileStatus st = roleFS.getFileStatus(writableDir);
      +    assertTrue("Not a directory: " + st,
      +        st.isDirectory());
           assertFileCount("files in the dest directory", roleFS,
               writableDir, expectedFileCount);
       
      @@ -498,13 +501,11 @@ public void testRenameFileFailsNoWrite() throws Throwable {
               + " & dest unchanged.");
           roleFS.mkdirs(writableDir);
           S3AFileSystem fs = getFileSystem();
      -//    List deletableFiles = createFiles(fs, writableDir, filecount);
      -
           Path source = new Path(writableDir, "source");
           touch(fs, source);
           fs.mkdirs(readOnlyDir);
           Path dest = new Path(readOnlyDir, "dest");
      -    LOG.info("Renaming files {} to {}", writableDir, dest);
      +    describe("Renaming files {} to {}", writableDir, dest);
           // rename fails but doesn't raise an exception. Good or bad?
           expectRenameForbidden(source, dest);
           assertIsFile(source);
      @@ -543,7 +544,6 @@ public void testCopyFileFailsOnSourceRead() throws Throwable {
         @Test
         public void testCopyDirFailsOnSourceRead() throws Throwable {
           describe("The source file isn't readable, so the COPY fails");
      -    Path source = new Path(noReadDir, "source");
           S3AFileSystem fs = getFileSystem();
           List files = createFiles(fs, noReadDir, dirDepth, fileCount,
               dirCount);
      @@ -632,6 +632,12 @@ public void testPartialDirDelete() throws Throwable {
               .containsAll(readOnlyFiles);
         }
       
      +  /**
      +   * Expect the delete() call to fail.
      +   * @param path path to delete.
      +   * @return the expected exception.
      +   * @throws Exception any other failure.
      +   */
         private AccessDeniedException expectDeleteForbidden(Path path) throws Exception {
           try(DurationInfo ignored =
                   new DurationInfo(LOG, true, "delete %s", path)) {
      @@ -733,7 +739,7 @@ private static CompletableFuture put(FileSystem fs,
             Path path, String text) {
           return submit(executor, () -> {
             try (DurationInfo ignore =
      -               new DurationInfo(LOG, "Creating %s", path)) {
      +               new DurationInfo(LOG, false, "Creating %s", path)) {
               createFile(fs, path, true, text.getBytes(Charsets.UTF_8));
               return path;
             }
      @@ -768,6 +774,15 @@ public static List createFiles(final FileSystem fs,
           }
         }
       
      +  /**
      +   * Recursive method to build up lists of files and directories.
      +   * @param filePaths list of file paths to add entries to.
      +   * @param dirPaths list of directory paths to add entries to.
      +   * @param destDir destination directory.
      +   * @param depth depth of directories
      +   * @param fileCount number of files.
      +   * @param dirCount number of directories.
      +   */
         private static void buildPaths(
             final List filePaths,
             final List dirPaths,
      diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestProgressiveRenameTracker.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestProgressiveRenameTracker.java
      new file mode 100644
      index 0000000000000..ff681a616f899
      --- /dev/null
      +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestProgressiveRenameTracker.java
      @@ -0,0 +1,25 @@
      +/*
      + * 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;
      +
      +/**
      + * Unit tests for {@link ProgressiveRenameTracker}.
      + */
      +public class TestProgressiveRenameTracker {
      +}
      
      From d497214a07e4fe14561da366a5f52ab048a12696 Mon Sep 17 00:00:00 2001
      From: Steve Loughran 
      Date: Wed, 8 May 2019 16:59:35 +0100
      Subject: [PATCH 16/22] HADOOP-15183: tracking ancestors across move
       operations.
      
      This adds the notion of an operation state which can be preserved across move() calls and so allow the metastore to update its view of the world without having to talk to any remote service.
      
      For DDB it maintains that hash table of ancestors and so a set of move operations spread across parallel threads will still share the same map of ancestors to avoid creating duplicates of.
      
      now, the bad news, as observed during test runs with debug logging: way too many metadata entries are being created. It seems to me that every putItem() call creates an ancestor list which then puts up all the ancestor markers, without checks for them existing. That is: the deeper you write, the more (expensive) write IOPs you create. If I haven't misunderstood it (and I hope I have!) then that map of ancestors isn't that useful, as entries get created anyway. All that's happening is that the write amplifcation isn't quite so big.
      
      Added instrumentation of low level record read/write requests for dynamodb.
      
      I'm still thinking about how best to deal with this, especially across a series of write operations likely to write to the same directory tree (that's rename and commit).
      
      Probably something off
      * track which parent paths we know to exist
      * and which have been created
      This would be part of the current ancestor state and, inevitably, have to be used during batch commit jobs too, so we'll know to avoid so much.
      
      For write and commit, it'd be best if finishedWrite() did the walk up the tree and stop on the first successful probe for a parent, as its isolated, and DDB GET costs less than PUT.
      That'd avoid the big write-context-spanning-operations change I'm considering but don't want to add in this patch, which is big enough anyway
      
      Change-Id: Idabb6100e90629ddb3e04dbc3e66573abaedba13
      ---
       .../apache/hadoop/fs/s3a/S3AFileSystem.java   |  55 +++++++--
       .../hadoop/fs/s3a/S3AInstrumentation.java     |  19 +++
       .../org/apache/hadoop/fs/s3a/Statistic.java   |   6 +
       .../hadoop/fs/s3a/impl/StoreContext.java      |   2 +-
       .../s3guard/DelayedUpdateRenameTracker.java   |  26 ++---
       .../fs/s3a/s3guard/DynamoDBMetadataStore.java | 108 +++++++++++++++---
       .../fs/s3a/s3guard/LocalMetadataStore.java    |   9 +-
       .../hadoop/fs/s3a/s3guard/MetadataStore.java  |  14 ++-
       .../fs/s3a/s3guard/NullMetadataStore.java     |  15 ++-
       .../s3a/s3guard/ProgressiveRenameTracker.java |  30 ++---
       .../hadoop/fs/s3a/s3guard/RenameTracker.java  |  58 +++++++++-
       .../apache/hadoop/fs/s3a/s3guard/S3Guard.java |  12 +-
       .../s3a/impl/ITestPartialRenamesDeletes.java  |   9 --
       .../s3a/impl/TestPartialDeleteFailures.java   |  10 +-
       .../s3guard/ITestDynamoDBMetadataStore.java   |   4 +-
       .../fs/s3a/s3guard/MetadataStoreTestBase.java |   3 +-
       .../AbstractITestS3AMetadataStoreScale.java   |   2 +-
       17 files changed, 282 insertions(+), 100 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 c5ee9808242ae..c10409fcecc0a 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
      @@ -243,8 +243,33 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
         /** Principal who created the FS; recorded during initialization. */
         private UserGroupInformation owner;
       
      -  // The maximum number of entries that can be deleted in any call to s3
      +  /**
      +   * The maximum number of entries that can be deleted in any bulk delete
      +   * call to S3 {@value}.
      +   */
         private static final int MAX_ENTRIES_TO_DELETE = 1000;
      +
      +  /**
      +   * This is an arbitrary value: {@value}.
      +   * It declares how many parallel copy operations
      +   * in a single rename can be queued before the operation pauses
      +   * and awaits completion.
      +   * A very large value wouldn't just starve other threads from
      +   * performing work, there's a risk that the S3 store itself would
      +   * throttle operations (which all go to the same shard).
      +   * It is not currently configurable just to avoid people choosing values
      +   * which work on a microbenchmark (single rename, no other work, ...)
      +   * but don't scale well to execution in a large process against a common
      +   * store, all while separate processes are working with the same shard
      +   * of storage.
      +   *
      +   * It should be a factor of {@link #MAX_ENTRIES_TO_DELETE} so that
      +   * all copies will have finished before deletion is contemplated.
      +   * (There's always a block for that, it just makes more sense to
      +   * perform the bulk delete after another block of copies have completed).
      +   */
      +  public static final int RENAME_PARALLEL_LIMIT = 10;
      +
         private String blockOutputBuffer;
         private S3ADataBlocks.BlockFactory blockFactory;
         private int blockOutputActiveBlocks;
      @@ -257,6 +282,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       
         private S3Guard.ITtlTimeProvider ttlTimeProvider;
       
      +  /** Where is the bucket? Null if the caller could not determine this. */
         private String bucketLocation;
       
         /** Add any deprecated keys. */
      @@ -1238,7 +1264,7 @@ private long innerRename(Path source, Path dest)
                   createStoreContext(),
                   src, srcStatus, dest);
           final AtomicLong bytesCopied = new AtomicLong();
      -    int renameParallelLimit = 10;
      +    int renameParallelLimit = RENAME_PARALLEL_LIMIT;
           final List> activeCopies =
               new ArrayList<>(renameParallelLimit);
           // aggregate operation to wait for the copies to complete then reset
      @@ -1283,7 +1309,7 @@ private long innerRename(Path source, Path dest)
             } else {
               LOG.debug("rename: renaming directory {} to {}", src, dst);
       
      -        // This is a directory to directory copy
      +        // This is a directory-to-directory copy
               dstKey = maybeAddTrailingSlash(dstKey);
               srcKey = maybeAddTrailingSlash(srcKey);
       
      @@ -1297,14 +1323,14 @@ private long innerRename(Path source, Path dest)
               // latter being used to update the rename tracker.
               final List keysToDelete = new ArrayList<>();
               final List pathsToDelete = new ArrayList<>();
      -        // the operation to update the lists of keys and paths.
      +        // to update the lists of keys and paths.
               final BiFunction queueToDelete =
                   (Path path, String key) -> {
                     pathsToDelete.add(path);
                     keysToDelete.add(new DeleteObjectsRequest.KeyVersion(key));
                     return null;
                   };
      -        // the operation to block waiting for ay active copies to finish
      +        // a lambda-expression to block waiting for ay active copies to finish
               // then delete all queued keys + paths to delete.
               final FunctionsRaisingIOE.FunctionRaisingIOE
                   completeActiveCopiesAndDeleteSources =
      @@ -1320,7 +1346,8 @@ private long innerRename(Path source, Path dest)
               if (dstStatus != null && dstStatus.isEmptyDirectory() == Tristate.TRUE) {
                 // delete unnecessary fake directory at the destination.
                 // this MUST be done before anything else so that
      -          // rollback code doesn't get confused.
      +          // rollback code doesn't get confused and insert a tombstone
      +          // marker.
                 deleteObjectAtPath(dstStatus.getPath(), dstKey, false);
               }
       
      @@ -1341,7 +1368,7 @@ private long innerRename(Path source, Path dest)
       
                 Path childDestPath = keyToQualifiedPath(newDstKey);
       
      -          // queue the copy for execution.
      +          // queue this copy for execution.
                 CompletableFuture copy = submit(boundedThreadPool, () ->
                     copySourceAndUpdateTracker(renameTracker,
                         childSourcePath,
      @@ -1353,17 +1380,22 @@ private long innerRename(Path source, Path dest)
                 bytesCopied.addAndGet(srcStatus.getLen());
                 activeCopies.add(copy);
                 if (activeCopies.size() == renameParallelLimit) {
      +            // the limit of active copies has been reached;
      +            // wait for completion or errors to surface.
                   LOG.debug("Waiting for active copies to complete");
                   completeActiveCopies.apply("batch threshold reached");
                 }
                 if (keysToDelete.size() == MAX_ENTRIES_TO_DELETE) {
                   // 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
      +            // simply jump straight to the delete.
                   completeActiveCopiesAndDeleteSources.apply("paged delete");
                 }
               }
               // end of iteration
       
      -        // await final set of copies and then delete
      +        // await the final set of copies and then delete
               // This will notify the renameTracker that these objects
               // have been deleted.
               completeActiveCopiesAndDeleteSources.apply("final copy and delete");
      @@ -1374,7 +1406,7 @@ private long innerRename(Path source, Path dest)
             }
           } catch (AmazonClientException | IOException ex) {
             // rename failed.
      -      // block for all ongoing copies to complete
      +      // block for all ongoing copies to complete, successfully or not
             try {
               completeActiveCopies.apply("failure handling");
             } catch (IOException e) {
      @@ -1403,8 +1435,9 @@ private long innerRename(Path source, Path dest)
         }
       
         /**
      -   * Remove source objects, and update the metastore
      -   * @param renameTracker rename state to update.
      +   * Remove source objects and update the metastore by way of
      +   * the rename tracker.
      +   * @param renameTracker rename tracker to update.
          * @param keysToDelete list of keys to delete
          * @param pathsToDelete list of paths matching the keys to delete 1:1.
          * @throws IOException failure
      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 42f0335550820..14c602f7c5033 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
      @@ -183,6 +183,8 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
             COMMITTER_MAGIC_FILES_CREATED,
             S3GUARD_METADATASTORE_PUT_PATH_REQUEST,
             S3GUARD_METADATASTORE_INITIALIZATION,
      +      S3GUARD_METADATASTORE_RECORD_READS,
      +      S3GUARD_METADATASTORE_RECORD_WRITES,
             S3GUARD_METADATASTORE_RETRY,
             S3GUARD_METADATASTORE_THROTTLED,
             STORE_IO_THROTTLED,
      @@ -1062,6 +1064,23 @@ public void throttled() {
           public void retrying() {
             // counters are incremented by owner.
           }
      +
      +    /**
      +     * Records have been read.
      +     * @param count the number of records read
      +     */
      +    public void recordsRead(int count) {
      +      incrementCounter(S3GUARD_METADATASTORE_RECORD_READS, count);
      +    }
      +
      +    /**
      +     * records have been written (including deleted).
      +     * @param count number of records written.
      +     */
      +    public void recordsWritten(int count) {
      +      incrementCounter(S3GUARD_METADATASTORE_RECORD_WRITES, count);
      +    }
      +
         }
       
         /**
      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 7a5672830ede2..71f9ddb98114a 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
      @@ -209,6 +209,12 @@ public enum Statistic {
             "S3Guard metadata store put one metadata path latency"),
         S3GUARD_METADATASTORE_INITIALIZATION("s3guard_metadatastore_initialization",
             "S3Guard metadata store initialization times"),
      +  S3GUARD_METADATASTORE_RECORD_READS(
      +      "s3guard_metadatastore_record_reads",
      +      "S3Guard metadata store records read"),
      +  S3GUARD_METADATASTORE_RECORD_WRITES(
      +      "s3guard_metadatastore_record_writes",
      +      "S3Guard metadata store records written"),
         S3GUARD_METADATASTORE_RETRY("s3guard_metadatastore_retry",
             "S3Guard metadata store retry events"),
         S3GUARD_METADATASTORE_THROTTLED("s3guard_metadatastore_throttled",
      diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java
      index 3d823828c7729..9bfa198506cbe 100644
      --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java
      +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java
      @@ -90,7 +90,7 @@ public class StoreContext {
         /** Invoker of operations. */
         private final Invoker invoker;
       
      -  /* Instrumentation and statistics. */
      +  /** Instrumentation and statistics. */
         private final S3AInstrumentation instrumentation;
         private final S3AStorageStatistics storageStatistics;
       
      diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameTracker.java
      index b4e5e2f5f9cf8..d0a832e421286 100644
      --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameTracker.java
      +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameTracker.java
      @@ -18,10 +18,10 @@
       
       package org.apache.hadoop.fs.s3a.s3guard;
       
      +import java.io.Closeable;
       import java.io.IOException;
       import java.util.ArrayList;
       import java.util.Collection;
      -import java.util.Comparator;
       import java.util.HashSet;
       import java.util.List;
       import java.util.Set;
      @@ -44,8 +44,6 @@
        */
       public class DelayedUpdateRenameTracker extends RenameTracker {
       
      -  private final StoreContext storeContext;
      -
         private final MetadataStore metadataStore;
       
         private final Collection sourcePaths = new HashSet<>();
      @@ -58,14 +56,11 @@ public DelayedUpdateRenameTracker(
             final StoreContext storeContext,
             final MetadataStore metadataStore,
             final Path sourceRoot,
      -      final Path dest) {
      -    super("DelayedUpdateRenameTracker", storeContext, sourceRoot, dest);
      -    this.storeContext = storeContext;
      -    this.metadataStore = metadataStore;
      -  }
      -
      -  public StoreContext getStoreContext() {
      -    return storeContext;
      +      final Path dest,
      +      final Closeable moveState) {
      +    super("DelayedUpdateRenameTracker", storeContext, metadataStore, sourceRoot, dest,
      +        moveState);
      +    this.metadataStore = storeContext.getMetadataStore();
         }
       
         @Override
      @@ -132,7 +127,7 @@ public synchronized void sourceObjectsDeleted(
       
         @Override
         public void completeRename() throws IOException {
      -    metadataStore.move(sourcePaths, destMetas);
      +    metadataStore.move(sourcePaths, destMetas, getMoveState());
           super.completeRename();
         }
       
      @@ -142,7 +137,7 @@ public IOException renameFailed(final Exception ex) {
           try {
             // the destination paths are updated; the source is left alone.
             // either the delete operation didn't begin, or the
      -      metadataStore.move(new ArrayList<>(0), destMetas);
      +      metadataStore.move(new ArrayList<>(0), destMetas, getMoveState());
             for (Path deletedPath : deletedPaths) {
               // this is not ideal in that it may leave parent stuff around.
               metadataStore.delete(deletedPath);
      @@ -173,10 +168,7 @@ private void deleteParentPaths() throws IOException {
           // that way: when we check for a parent path existing we can
           // see if it really is empty.
           List parents = new ArrayList<>(parentPaths);
      -    parents.sort(
      -        Comparator.comparing(
      -            (Path p) -> p.depth())
      -            .thenComparing((Path p) -> p.toUri().getPath()));
      +    parents.sort(PathOrderComparators.TOPMOST_PATH_LAST);
           for (Path parent : parents) {
             PathMetadata md = metadataStore.get(parent, true);
             if (md != null && md.isEmptyDirectory().equals(Tristate.TRUE)) {
      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 a36347d66b26f..637d75ce109a8 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
      @@ -19,6 +19,7 @@
       package org.apache.hadoop.fs.s3a.s3guard;
       
       import javax.annotation.Nullable;
      +import java.io.Closeable;
       import java.io.FileNotFoundException;
       import java.io.IOException;
       import java.io.InterruptedIOException;
      @@ -187,7 +188,7 @@
        * sub-tree.
        *
        * Some mutating operations, notably {@link #deleteSubtree(Path)} and
      - * {@link #move(Collection, Collection)}, are less efficient with this schema.
      + * {@link MetadataStore#move(Collection, Collection, AutoCloseable)}, are less efficient with this schema.
        * They require mutating multiple items in the DynamoDB table.
        *
        * By default, DynamoDB access is performed within the same AWS region as
      @@ -585,10 +586,14 @@ private Item getConsistentItem(final Path path) throws IOException {
           final GetItemSpec spec = new GetItemSpec()
               .withPrimaryKey(key)
               .withConsistentRead(true); // strictly consistent read
      -    return readOp.retry("get",
      +    Item item = readOp.retry("get",
               path.toString(),
               true,
               () -> table.getItem(spec));
      +    if (instrumentation != null) {
      +      instrumentation.recordsRead(1);
      +    }
      +    return item;
         }
       
         @Override
      @@ -731,14 +736,18 @@ DirListingMetadata getDirListingMetadataFromDirMetaAndList(Path path,
         }
       
         /**
      -   * build the list of all parent entries.
      +   * Build the list of all parent entries.
      +   * Thread safety: none. if access the move state is to be synchronized,
      +   * callers must do this.
          * @param pathsToCreate paths to create
      +   * @param ancestorState ongoing ancestor state.
          * @return the full ancestry paths
          */
      -  Collection completeAncestry(
      -      Collection pathsToCreate) {
      +  private Collection completeAncestry(
      +      final Collection pathsToCreate,
      +      final AncestorState ancestorState) {
           // Key on path to allow fast lookup
      -    Map ancestry = new HashMap<>();
      +    Map ancestry = ancestorState.getAncestry();
       
           for (DDBPathMetadata meta : pathsToCreate) {
             Preconditions.checkArgument(meta != null);
      @@ -774,13 +783,15 @@ Collection completeAncestry(
          * @param pathsToCreate Collection of all PathMetadata for the new paths
          *                      that were created at the destination of the rename
          *                      ().
      +   * @param closeable
          * @throws IOException
          */
         @Override
         @Retries.RetryTranslated
         public void move(
             @Nullable Collection pathsToDelete,
      -      @Nullable Collection pathsToCreate) throws IOException {
      +      @Nullable Collection pathsToCreate,
      +      @Nullable final Closeable closeable) throws IOException {
           if (pathsToDelete == null && pathsToCreate == null) {
             return;
           }
      @@ -797,9 +808,16 @@ public void move(
           // Following code is to maintain this invariant by putting all ancestor
           // directories of the paths to create.
           // ancestor paths that are not explicitly added to paths to create
      +    AncestorState ancestorState = extractOrCreate(closeable);
           List newItems = new ArrayList<>();
           if (pathsToCreate != null) {
      -      newItems.addAll(completeAncestry(pathMetaToDDBPathMeta(pathsToCreate)));
      +      // create all parent entries.
      +      // this is synchronized on the move state so that across both serialized
      +      // and parallelized renames, duplicate ancestor entries are not created.
      +      synchronized (ancestorState) {
      +        newItems.addAll(completeAncestry(pathMetaToDDBPathMeta(pathsToCreate),
      +            ancestorState));
      +      }
           }
           // sort all the new items topmost first.
           newItems.sort(PathOrderComparators.TOPMOST_PM_FIRST);
      @@ -835,6 +853,7 @@ private int processBatchWriteRequest(PrimaryKey[] keysToDelete,
           while (count < totalToDelete + totalToPut) {
             final TableWriteItems writeItems = new TableWriteItems(tableName);
             int numToDelete = 0;
      +      int batchSize = 0;
             if (keysToDelete != null
                 && count < totalToDelete) {
               numToDelete = Math.min(S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT,
      @@ -842,6 +861,7 @@ private int processBatchWriteRequest(PrimaryKey[] keysToDelete,
               writeItems.withPrimaryKeysToDelete(
                   Arrays.copyOfRange(keysToDelete, count, count + numToDelete));
               count += numToDelete;
      +        batchSize = numToDelete;
             }
       
             if (numToDelete < S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT
      @@ -854,6 +874,7 @@ private int processBatchWriteRequest(PrimaryKey[] keysToDelete,
               writeItems.withItemsToPut(
                   Arrays.copyOfRange(itemsToPut, index, index + numToPut));
               count += numToPut;
      +        batchSize += numToPut;
             }
       
             // if there's a retry and another process updates things then it's not
      @@ -866,10 +887,14 @@ private int processBatchWriteRequest(PrimaryKey[] keysToDelete,
                 () -> dynamoDB.batchWriteItem(writeItems));
             // Check for unprocessed keys in case of exceeding provisioned throughput
             Map> unprocessed = res.getUnprocessedItems();
      +      if (instrumentation != null) {
      +        instrumentation.recordsWritten(batchSize - unprocessed.size());
      +      }
             int retryCount = 0;
             while (!unprocessed.isEmpty()) {
               batchWriteCapacityExceededEvents.incrementAndGet();
               batches++;
      +        batchSize = unprocessed.size();
               retryBackoffOnBatchWrite(retryCount++);
               // use a different reference to keep the compiler quiet
               final Map> upx = unprocessed;
      @@ -879,6 +904,9 @@ private int processBatchWriteRequest(PrimaryKey[] keysToDelete,
                   true,
                   () -> dynamoDB.batchWriteItemUnprocessed(upx));
               unprocessed = res.getUnprocessedItems();
      +        if (instrumentation != null) {
      +          instrumentation.recordsWritten(batchSize - unprocessed.size());
      +        }
             }
           }
           return batches;
      @@ -960,21 +988,23 @@ private void innerPut(Collection metas) throws IOException {
                 new Exception("source"));
             return;
           }
      -    Item[] items = pathMetadataToItem(completeAncestry(metas));
      +    Item[] items = pathMetadataToItem(completeAncestry(metas, new AncestorState()));
           LOG.debug("Saving batch of {} items to table {}, region {}", items.length,
               tableName, region);
           processBatchWriteRequest(null, items);
         }
       
         /**
      -   * Helper method to get full path of ancestors that are nonexistent in table.
      +   * Get full path of ancestors that are nonexistent in table.
      +   *
      +   * This queries DDB.
          */
         @VisibleForTesting
         @Retries.RetryTranslated
      -  Collection fullPathsToPut(DDBPathMetadata meta)
      +  List fullPathsToPut(DDBPathMetadata meta)
             throws IOException {
           checkPathMetadata(meta);
      -    final Collection metasToPut = new ArrayList<>();
      +    final List metasToPut = new ArrayList<>();
           // root path is not persisted
           if (!meta.getFileStatus().getPath().isRoot()) {
             metasToPut.add(meta);
      @@ -1034,11 +1064,14 @@ public void put(DirListingMetadata meta) throws IOException {
                   false, meta.isAuthoritative(), meta.getLastUpdated());
       
           // First add any missing ancestors...
      -    final Collection metasToPut = fullPathsToPut(ddbPathMeta);
      +    final List metasToPut = fullPathsToPut(ddbPathMeta);
       
           // next add all children of the directory
           metasToPut.addAll(pathMetaToDDBPathMeta(meta.getListing()));
       
      +    // and sort
      +    metasToPut.sort(PathOrderComparators.TOPMOST_PM_FIRST);
      +
           processBatchWriteRequest(null, pathMetadataToItem(metasToPut));
         }
       
      @@ -1786,11 +1819,21 @@ public Invoker getInvoker() {
           return invoker;
         }
       
      +  /**
      +   * Initiate the rename operation by creating the tracker and the ongoing
      +   * move state.
      +   * @param storeContext store context.
      +   * @param source source path
      +   * @param sourceStatus status of the source file/dir
      +   * @param dest destination path.
      +   * @return the rename tracker
      +   */
         @Override
         public RenameTracker initiateRenameOperation(final StoreContext storeContext,
             final Path source,
      -      final FileStatus srcStatus, final Path dest) throws IOException {
      -    return new ProgressiveRenameTracker(storeContext, this, source, dest);
      +      final FileStatus sourceStatus, final Path dest) {
      +    return new ProgressiveRenameTracker(storeContext, this, source, dest,
      +        new AncestorState());
         }
       
         /**
      @@ -1834,4 +1877,39 @@ static IOException translateTableWaitFailure(
             return new IOException(e);
           }
         }
      +
      +  /**
      +   * Get the move state passed in; create a new one if needed.
      +   * @param closeable state.
      +   * @return the cast or created state.
      +   */
      +  @VisibleForTesting
      +  static AncestorState extractOrCreate(@Nullable Closeable closeable) {
      +    if (closeable != null) {
      +      return (AncestorState) closeable;
      +    } else {
      +      return new AncestorState();
      +    }
      +  }
      +
      +  /**
      +   * This tracks all the ancestors created,
      +   * across multiple move/write operations.
      +   * This is to avoid duplicate creation of ancestors even during
      +   * rename operations managed by a rename tracker.
      +   */
      +  @VisibleForTesting
      +  static final class AncestorState implements Closeable {
      +
      +    private final Map ancestry = new HashMap<>();
      +
      +    public Map getAncestry() {
      +      return ancestry;
      +    }
      +
      +    @Override
      +    public void close() {
      +
      +    }
      +  }
       }
      diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java
      index 31a229f8fdf50..0185fc46e0f8e 100644
      --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java
      +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java
      @@ -36,6 +36,7 @@
       import org.slf4j.Logger;
       import org.slf4j.LoggerFactory;
       
      +import java.io.Closeable;
       import java.io.IOException;
       import java.net.URI;
       import java.util.Collection;
      @@ -196,7 +197,8 @@ public synchronized DirListingMetadata listChildren(Path p) throws
         @Override
         public void move(
             @Nullable Collection pathsToDelete,
      -      @Nullable Collection pathsToCreate) throws IOException {
      +      @Nullable Collection pathsToCreate,
      +      @Nullable final Closeable moveState) throws IOException {
           LOG.info("Move {} to {}", pathsToDelete, pathsToCreate);
       
           if (pathsToCreate == null) {
      @@ -541,7 +543,8 @@ DirListingMetadata getDirListingMeta(Path p){
         @Override
         public RenameTracker initiateRenameOperation(final StoreContext storeContext,
             final Path source,
      -      final FileStatus srcStatus, final Path dest) throws IOException {
      -    return new ProgressiveRenameTracker(storeContext, this, source, dest);
      +      final FileStatus sourceStatus, final Path dest) throws IOException {
      +    return new ProgressiveRenameTracker(storeContext, this, source, dest,
      +        null);
         }
       }
      diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java
      index c880b83b177e3..a68da964c227f 100644
      --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java
      +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java
      @@ -152,13 +152,15 @@ PathMetadata get(Path path, boolean wantEmptyDirectoryFlag)
          * @param pathsToDelete Collection of all paths that were removed from the
          *                      source directory tree of the move.
          * @param pathsToCreate Collection of all PathMetadata for the new paths
      -   *                      that were created at the destination of the rename
      -   *                      ().
      +   *                      that were created at the destination of the rename().
      +   * @param moveState     Any ongoing state supplied to the rename tracker
      +   *                      which is to be passed in with each move operation.
          * @throws IOException if there is an error
          */
         void move(
             @Nullable Collection pathsToDelete,
      -      @Nullable Collection pathsToCreate) throws IOException;
      +      @Nullable Collection pathsToCreate,
      +      @Nullable Closeable moveState) throws IOException;
       
         /**
          * Saves metadata for exactly one path.
      @@ -262,15 +264,15 @@ void prune(long modTime, String keyPrefix)
          *
          * @param storeContext store context.
          * @param source source path
      -   * @param srcStatus
      +   * @param sourceStatus status of the source file/dir
          * @param dest destination path.
      -   * @return the rename operation to update
      +   * @return the rename tracker
          * @throws IOException Failure.
          */
         RenameTracker initiateRenameOperation(
             StoreContext storeContext,
             Path source,
      -      FileStatus srcStatus,
      +      FileStatus sourceStatus,
             Path dest)
             throws IOException;
       }
      diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java
      index 6abb33323bb1b..82821cad494a3 100644
      --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java
      +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java
      @@ -24,6 +24,7 @@
       import org.apache.hadoop.fs.Path;
       import org.apache.hadoop.fs.s3a.impl.StoreContext;
       
      +import java.io.Closeable;
       import java.io.IOException;
       import java.util.Collection;
       import java.util.HashMap;
      @@ -78,7 +79,8 @@ public DirListingMetadata listChildren(Path path) throws IOException {
       
         @Override
         public void move(Collection pathsToDelete,
      -      Collection pathsToCreate) throws IOException {
      +      Collection pathsToCreate,
      +      final Closeable moveState) throws IOException {
         }
       
         @Override
      @@ -126,18 +128,19 @@ public void updateParameters(Map parameters)
         @Override
         public RenameTracker initiateRenameOperation(final StoreContext storeContext,
             final Path source,
      -      final FileStatus srcStatus, final Path dest)
      +      final FileStatus sourceStatus,
      +      final Path dest)
             throws IOException {
      -    return new NullRenameTracker(storeContext, source, dest);
      +    return new NullRenameTracker(storeContext, source, dest, this);
         }
       
      -  private static class NullRenameTracker extends RenameTracker {
      +  private static final class NullRenameTracker extends RenameTracker {
       
           private NullRenameTracker(
               final StoreContext storeContext,
               final Path source,
      -        final Path dest) {
      -      super("rename tracker", storeContext, source, dest);
      +        final Path dest, MetadataStore metadataStore) {
      +      super("null tracker", storeContext, metadataStore, source, dest, null);
           }
       
           @Override
      diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ProgressiveRenameTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ProgressiveRenameTracker.java
      index b429c52128821..69052986c71ad 100644
      --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ProgressiveRenameTracker.java
      +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ProgressiveRenameTracker.java
      @@ -18,6 +18,7 @@
       
       package org.apache.hadoop.fs.s3a.s3guard;
       
      +import java.io.Closeable;
       import java.io.IOException;
       import java.util.ArrayList;
       import java.util.Collection;
      @@ -66,8 +67,6 @@
        */
       public class ProgressiveRenameTracker extends RenameTracker {
       
      -  private final MetadataStore metadataStore;
      -
         /**
          * The collection of paths to delete; this is added as individual files
          * are renamed.
      @@ -82,10 +81,10 @@ public ProgressiveRenameTracker(
             final StoreContext storeContext,
             final MetadataStore metadataStore,
             final Path sourceRoot,
      -      final Path dest) {
      +      final Path dest,
      +      final Closeable moveState) {
           super("ProgressiveRenameTracker",
      -        storeContext, sourceRoot, dest);
      -    this.metadataStore = metadataStore;
      +        storeContext, metadataStore, sourceRoot, dest, moveState);
         }
       
         /**
      @@ -112,11 +111,13 @@ public void fileCopied(
           // build the list of entries to add in a synchronized block.
           final List entriesToAdd = new ArrayList<>(1);
           LOG.debug("Updating store with copied file {}", sourcePath);
      +    MetadataStore store = getMetadataStore();
           synchronized (this) {
             checkArgument(!pathsToDelete.contains(sourcePath),
                 "File being renamed is already processed %s", destPath);
             // create the file metadata and update the local structures.
      -      S3Guard.addMoveFile(metadataStore,
      +      S3Guard.addMoveFile(
      +          store,
                 pathsToDelete,
                 entriesToAdd,
                 sourcePath,
      @@ -128,7 +129,7 @@ public void fileCopied(
             if (addAncestors) {
               // add all new ancestors.
               addMoveAncestors(
      -            metadataStore,
      +            store,
                   pathsToDelete,
                   entriesToAdd,
                   getSourceRoot(),
      @@ -140,7 +141,7 @@ public void fileCopied(
       
           // outside the lock, the entriesToAdd list has all new files to create.
           // ...so update the store.
      -    metadataStore.move(null, entriesToAdd);
      +    store.move(null, entriesToAdd, getMoveState());
         }
       
         /**
      @@ -157,8 +158,9 @@ public void directoryMarkerCopied(final FileStatus sourceStatus,
             final boolean addAncestors) throws IOException {
           // this list is created on demand.
           final List entriesToAdd = new ArrayList<>(1);
      +    MetadataStore store = getMetadataStore();
           synchronized (this) {
      -      addMoveDir(metadataStore,
      +      addMoveDir(store,
                 pathsToDelete,
                 entriesToAdd,
                 sourceStatus.getPath(),
      @@ -166,7 +168,7 @@ public void directoryMarkerCopied(final FileStatus sourceStatus,
                 getOwner());
             // Ancestor directories may not be listed, so we explicitly add them
             if (addAncestors) {
      -        addMoveAncestors(metadataStore,
      +        addMoveAncestors(store,
                   pathsToDelete,
                   entriesToAdd,
                   getSourceRoot(),
      @@ -179,14 +181,14 @@ public void directoryMarkerCopied(final FileStatus sourceStatus,
           // ...so update the store.
           try (DurationInfo ignored = new DurationInfo(LOG, false,
               "adding %s metastore entries", entriesToAdd.size())) {
      -      metadataStore.move(null, entriesToAdd);
      +      store.move(null, entriesToAdd, getMoveState());
           }
         }
       
         @Override
         public synchronized void moveSourceDirectory() throws IOException {
           if (!pathsToDelete.contains(getSourceRoot())) {
      -      addMoveDir(metadataStore, pathsToDelete, destMetas,
      +      addMoveDir(getMetadataStore(), pathsToDelete, destMetas,
                 getSourceRoot(),
                 getDest(),
                 getOwner());
      @@ -205,7 +207,7 @@ public void sourceObjectsDeleted(
           // delete the paths from the metastore
           try (DurationInfo ignored = new DurationInfo(LOG, false,
               "delete %s metastore entries", paths.size())) {
      -      metadataStore.move(paths, null);
      +      getMetadataStore().move(paths, null, getMoveState());
           }
         }
       
      @@ -213,7 +215,7 @@ public void sourceObjectsDeleted(
         public void completeRename() throws IOException {
           // this should all have happened.
           LOG.debug("Rename completed for {}", this);
      -    metadataStore.move(pathsToDelete, destMetas);
      +    getMetadataStore().move(pathsToDelete, destMetas, getMoveState());
           super.completeRename();
         }
       
      diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RenameTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RenameTracker.java
      index ba8b8f5ffd6b0..03b6d43c6716a 100644
      --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RenameTracker.java
      +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RenameTracker.java
      @@ -18,6 +18,7 @@
       
       package org.apache.hadoop.fs.s3a.s3guard;
       
      +import java.io.Closeable;
       import java.io.IOException;
       import java.util.List;
       
      @@ -29,8 +30,10 @@
       import org.apache.hadoop.fs.Path;
       import org.apache.hadoop.fs.s3a.impl.StoreContext;
       import org.apache.hadoop.fs.s3a.impl.StoreOperation;
      +import org.apache.hadoop.io.IOUtils;
       import org.apache.hadoop.util.DurationInfo;
       
      +import static com.google.common.base.Preconditions.checkNotNull;
       import static org.apache.hadoop.fs.s3a.S3AUtils.translateException;
       
       /**
      @@ -38,6 +41,13 @@
        * as initiated in the S3AFilesystem rename.
        * Subclasses must provide an implementation and return it in
        * {@link MetadataStore#initiateRenameOperation(StoreContext, Path, FileStatus, Path)}.
      + * The {@code moveState} field/constructor argument is an opaque state to
      + * be passed down to the metastore in its move operations; this allows the
      + * stores to manage ongoing state -while still being able to share
      + * rename tracker implementations.
      + * This is to avoid performance problems wherein the progressive rename
      + * tracker causes the store to repeatedly create and write duplicate
      + * ancestor entries for every file added.
        */
       public abstract class RenameTracker extends StoreOperation {
       
      @@ -50,7 +60,6 @@ public abstract class RenameTracker extends StoreOperation {
         /** destination path. */
         private final Path dest;
       
      -
         /**
          * Track the duration of this operation.
          */
      @@ -62,20 +71,43 @@ public abstract class RenameTracker extends StoreOperation {
         private final String name;
       
         /**
      -   * constructor.
      +   * Any ongoing state supplied to the rename tracker
      +   * which is to be passed in with each move operation.
      +   * This must be closed at the end of the tracker's life.
      +   */
      +  private final Closeable moveState;
      +
      +  /**
      +   * The metadata store for this tracker.
      +   * Always non-null.
      +   * This is passed in separate from the store context to guarantee
      +   * that whichever store creates a tracker is explicitly bound to that
      +   * instance.
      +   */
      +  private final MetadataStore metadataStore;
      +
      +  /**
      +   * Constructor.
          * @param name tracker name for logs.
          * @param storeContext store context.
      +   * @param metadataStore the stopre
          * @param sourceRoot source path.
          * @param dest destination path.
      +   * @param moveState ongoing move state.
          */
         protected RenameTracker(
             final String name,
             final StoreContext storeContext,
      +      final MetadataStore metadataStore,
             final Path sourceRoot,
      -      final Path dest) {
      -    super(storeContext);
      -    this.sourceRoot = sourceRoot;
      -    this.dest = dest;
      +      final Path dest,
      +      Closeable moveState) {
      +    super(checkNotNull(storeContext));
      +    checkNotNull(storeContext.getUsername(), "No username");
      +    this.metadataStore = checkNotNull(metadataStore);
      +    this.sourceRoot = checkNotNull(sourceRoot);
      +    this.dest = checkNotNull(dest);
      +    this.moveState = moveState;
           this.name = String.format("%s (%s, %s)", name, sourceRoot, dest);
           durationInfo = new DurationInfo(LOG, false,
               name +" (%s, %s)", sourceRoot, dest);
      @@ -98,6 +130,18 @@ public String getOwner() {
           return getStoreContext().getUsername();
         }
       
      +  public Closeable getMoveState() {
      +    return moveState;
      +  }
      +
      +  /**
      +   * Get the metadata store.
      +   * @return a non-null store.
      +   */
      +  protected MetadataStore getMetadataStore() {
      +    return metadataStore;
      +  }
      +
         /**
          * A file has been copied.
          *
      @@ -170,6 +214,7 @@ public void sourceObjectsDeleted(
          * @throws IOException failure.
          */
         public void completeRename() throws IOException {
      +    IOUtils.cleanupWithLogger(LOG, moveState);
           noteRenameFinished();
         }
       
      @@ -197,6 +242,7 @@ protected void noteRenameFinished() {
          */
         public IOException renameFailed(Exception ex) {
           LOG.debug("Rename has failed", ex);
      +    IOUtils.cleanupWithLogger(LOG, moveState);
           noteRenameFinished();
           return convertToIOException(ex);
         }
      diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java
      index 66e67f294c875..5b614a6a82992 100644
      --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java
      +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java
      @@ -150,10 +150,14 @@ public static S3AFileStatus putAndReturn(MetadataStore ms,
             S3AFileStatus status,
             S3AInstrumentation instrumentation) throws IOException {
           long startTimeNano = System.nanoTime();
      -    ms.put(new PathMetadata(status));
      -    instrumentation.addValueToQuantiles(S3GUARD_METADATASTORE_PUT_PATH_LATENCY,
      -        (System.nanoTime() - startTimeNano));
      -    instrumentation.incrementCounter(S3GUARD_METADATASTORE_PUT_PATH_REQUEST, 1);
      +    try {
      +      ms.put(new PathMetadata(status));
      +    } finally {
      +      instrumentation.addValueToQuantiles(S3GUARD_METADATASTORE_PUT_PATH_LATENCY,
      +          (System.nanoTime() - startTimeNano));
      +      instrumentation.incrementCounter(S3GUARD_METADATASTORE_PUT_PATH_REQUEST, 1);
      +    }
      +
           return status;
         }
       
      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 2f9b2d2f6c9d8..7e3524d8aa702 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
      @@ -492,7 +492,6 @@ public void testRenameDirFailsInDelete() throws Throwable {
               st.isDirectory());
           assertFileCount("files in the dest directory", roleFS,
               writableDir, expectedFileCount);
      -
         }
       
         @Test
      @@ -686,14 +685,6 @@ private void pathMustExist(Path p) {
           eval(() -> assertPathExists("Missing path", p));
         }
       
      -  /**
      -   * Assert that a path must exist, map IOEs to RTEs for loops.
      -   * @param p path.
      -   */
      -  private void pathMustNotExist(Path p) {
      -    eval(() -> assertPathDoesNotExist("Path should not exist", p));
      -  }
      -
         /**
          * Prune the store for everything under the test path.
          * @param path path.
      diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java
      index 7c649db8a08a5..73d954d043941 100644
      --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java
      +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java
      @@ -18,6 +18,8 @@
       
       package org.apache.hadoop.fs.s3a.impl;
       
      +import javax.annotation.Nullable;
      +import java.io.Closeable;
       import java.io.IOException;
       import java.net.URI;
       import java.net.URISyntaxException;
      @@ -269,8 +271,10 @@ public DirListingMetadata listChildren(final Path path) {
           }
       
           @Override
      -    public void move(final Collection pathsToDelete,
      -        final Collection pathsToCreate) {
      +    public void move(
      +        @Nullable final Collection pathsToDelete,
      +        @Nullable final Collection pathsToCreate,
      +        @Nullable final Closeable moveState) {
       
           }
       
      @@ -329,7 +333,7 @@ public List getCreated() {
           @Override
           public RenameTracker initiateRenameOperation(final StoreContext storeContext,
               final Path source,
      -        final FileStatus srcStatus, final Path dest) throws IOException {
      +        final FileStatus sourceStatus, final Path dest) throws IOException {
             throw new UnsupportedOperationException("unsupported");
           }
         }
      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 158f13d3fc763..f42d7dcfa5f1a 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
      @@ -388,7 +388,7 @@ private void doTestBatchWrite(int numDelete, int numPut,
           }
       
           // move the old paths to new paths and verify
      -    ms.move(pathsToDelete, newMetas);
      +    ms.move(pathsToDelete, newMetas, null);
           assertEquals(0, ms.listChildren(oldDir).withoutTombstones().numEntries());
           if (newMetas != null) {
             assertTrue(CollectionUtils
      @@ -587,7 +587,7 @@ public void testMovePopulatesAncestors() throws IOException {
                   1024, false))
           );
       
      -    ddbms.move(fullSourcePaths, pathsToCreate);
      +    ddbms.move(fullSourcePaths, pathsToCreate, null);
       
           // assert that all the ancestors should have been populated automatically
           assertCached(testRoot + "/c");
      diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreTestBase.java
      index 799c5a046bc22..1acce2ddc8700 100644
      --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreTestBase.java
      +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreTestBase.java
      @@ -43,7 +43,6 @@
       import org.apache.hadoop.io.IOUtils;
       import org.apache.hadoop.test.HadoopTestBase;
       
      -import static org.apache.hadoop.fs.s3a.S3ATestUtils.isMetadataStoreAuthoritative;
       import static org.apache.hadoop.fs.s3a.S3ATestUtils.metadataStorePersistsAuthoritativeBit;
       
       /**
      @@ -584,7 +583,7 @@ public void testMove() throws Exception {
           destMetas.add(new PathMetadata(makeDirStatus("/b1")));
           destMetas.add(new PathMetadata(makeFileStatus("/b1/file1", 100)));
           destMetas.add(new PathMetadata(makeFileStatus("/b1/file2", 100)));
      -    ms.move(srcPaths, destMetas);
      +    ms.move(srcPaths, destMetas, null);
       
           // Assert src is no longer there
           dirMeta = ms.listChildren(strToPath("/a1"));
      diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractITestS3AMetadataStoreScale.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractITestS3AMetadataStoreScale.java
      index 0e6a1d8d09245..d015161275391 100644
      --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractITestS3AMetadataStoreScale.java
      +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractITestS3AMetadataStoreScale.java
      @@ -129,7 +129,7 @@ public void test_020_Moves() throws Throwable {
                   toDelete = movedPaths;
                   toCreate = origMetas;
                 }
      -          ms.move(toDelete, toCreate);
      +          ms.move(toDelete, toCreate, null);
               }
               moveTimer.end();
               printTiming(LOG, "move", moveTimer, operations);
      
      From 9247bca1a34891893d435d15f9288c3cb99ec5e1 Mon Sep 17 00:00:00 2001
      From: Steve Loughran 
      Date: Wed, 15 May 2019 18:15:36 +0200
      Subject: [PATCH 17/22] HADOOP-15183: bulk metastore operation
       performance/scale
      
      convert the bulk operation for move into into an optional BulkOperationState for put operations in the metastore.
      
      This is then used all the way through to commit operations so as to address HADOOP-15604.
      
      I've not yet completed up all the wiring in the metastore: This is the first step, having an optional bulk update
      
      Change-Id: I0275a344715eab002024f1644db8858059c71bb8
      Also: revert changes in ContractTestUtils as it created link conflict, and because that library gets used in contract tests by external stores, I don't want to needlessly break things.
      ---
       .../hadoop/fs/contract/ContractTestUtils.java |   8 +-
       .../apache/hadoop/fs/s3a/S3AFileSystem.java   |  17 ++-
       .../hadoop/fs/s3a/WriteOperationHelper.java   |  62 ++++++++-
       .../fs/s3a/commit/AbstractS3ACommitter.java   |  42 ++++--
       .../fs/s3a/commit/CommitOperations.java       | 129 ++++++++++++++++--
       .../s3a/commit/staging/StagingCommitter.java  |  12 +-
       .../fs/s3a/s3guard/BulkOperationState.java    |  42 ++++++
       .../s3guard/DelayedUpdateRenameTracker.java   |   9 +-
       .../fs/s3a/s3guard/DynamoDBMetadataStore.java |  66 +++++----
       .../fs/s3a/s3guard/LocalMetadataStore.java    |  20 ++-
       .../hadoop/fs/s3a/s3guard/MetadataStore.java  |  36 ++++-
       .../fs/s3a/s3guard/NullMetadataStore.java     |  13 +-
       .../s3a/s3guard/ProgressiveRenameTracker.java |  13 +-
       .../hadoop/fs/s3a/s3guard/RenameTracker.java  |  19 ++-
       .../apache/hadoop/fs/s3a/s3guard/S3Guard.java |  62 ++++++++-
       .../hadoop/fs/s3a/s3guard/S3GuardTool.java    |  16 ++-
       .../ITestS3AMetadataPersistenceException.java |  10 +-
       .../hadoop/fs/s3a/MockS3AFileSystem.java      |   6 +-
       .../hadoop/fs/s3a/auth/ITestAssumeRole.java   |  16 ++-
       .../fs/s3a/commit/ITestCommitOperations.java  |  17 ++-
       .../magic/ITestS3AHugeMagicCommits.java       |   9 +-
       .../s3a/impl/ITestPartialRenamesDeletes.java  |   5 +-
       .../s3a/impl/TestPartialDeleteFailures.java   |  18 ++-
       .../s3guard/AbstractS3GuardToolTestBase.java  |   7 +-
       .../s3guard/ITestDynamoDBMetadataStore.java   |  10 +-
       .../ITestDynamoDBMetadataStoreScale.java      |  16 ++-
       .../fs/s3a/s3guard/MetadataStoreTestBase.java |  53 +++----
       .../AbstractITestS3AMetadataStoreScale.java   |   4 +-
       28 files changed, 557 insertions(+), 180 deletions(-)
       create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/BulkOperationState.java
      
      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 fd1e2f10e1b4c..b4db3a5803ad8 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
      @@ -624,10 +624,9 @@ public static byte[] writeTextFile(FileSystem fs,
          * @param path       path to write
          * @param overwrite overwrite flag
          * @param data source dataset. Can be null
      -   * @return the path written to.
          * @throws IOException on any problem
          */
      -  public static Path createFile(FileSystem fs,
      +  public static void createFile(FileSystem fs,
                                        Path path,
                                        boolean overwrite,
                                        byte[] data) throws IOException {
      @@ -637,7 +636,6 @@ public static Path createFile(FileSystem fs,
               stream.write(data);
             }
             stream.close();
      -      return path;
           } finally {
             IOUtils.closeStream(stream);
           }
      @@ -664,13 +662,11 @@ public static void appendFile(FileSystem fs,
          * Touch a file.
          * @param fs filesystem
          * @param path path
      -   * @return the patch created.
          * @throws IOException IO problems
          */
      -  public static Path touch(FileSystem fs,
      +  public static void touch(FileSystem fs,
                                  Path path) throws IOException {
           createFile(fs, path, true, null);
      -    return path;
         }
       
         /**
      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 c10409fcecc0a..42e644bff5480 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.impl.FunctionsRaisingIOE;
       import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport;
       import org.apache.hadoop.fs.s3a.impl.StoreContext;
      +import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
       import org.apache.hadoop.fs.s3a.s3guard.RenameTracker;
       import org.apache.hadoop.fs.s3a.select.InternalSelectConstants;
       import org.apache.hadoop.util.DurationInfo;
      @@ -2007,7 +2008,7 @@ PutObjectResult putObjectDirect(PutObjectRequest putObjectRequest)
             PutObjectResult result = s3.putObject(putObjectRequest);
             incrementPutCompletedStatistics(true, len);
             // update metadata
      -      finishedWrite(putObjectRequest.getKey(), len);
      +      finishedWrite(putObjectRequest.getKey(), len, null);
             return result;
           } catch (AmazonClientException e) {
             incrementPutCompletedStatistics(false, len);
      @@ -3066,7 +3067,7 @@ UploadResult executePut(PutObjectRequest putObjectRequest,
           UploadResult result = waitForUploadCompletion(key, info);
           listener.uploadCompleted();
           // post-write actions
      -    finishedWrite(key, info.getLength());
      +    finishedWrite(key, info.getLength(), null);
           return result;
         }
       
      @@ -3376,7 +3377,7 @@ private Optional generateSSECustomerKey() {
         /**
          * Perform post-write actions.
          * Calls {@link #deleteUnnecessaryFakeDirectories(Path)} and then
      -   * {@link S3Guard#addAncestors(MetadataStore, Path, String)}}.
      +   * {@link S3Guard#addAncestors(MetadataStore, Path, String, BulkOperationState)}}.
          * This operation MUST be called after any PUT/multipart PUT completes
          * successfully.
          *
      @@ -3388,6 +3389,7 @@ private Optional generateSSECustomerKey() {
          * 
          * @param key key written to
          * @param length  total length of file written
      +   * @param operationState state of any ongoing bulk operation.
          * @throws MetadataPersistenceException if metadata about the write could
          * not be saved to the metadata store and
          * fs.s3a.metadatastore.fail.on.write.error=true
      @@ -3395,7 +3397,9 @@ private Optional generateSSECustomerKey() {
         @InterfaceAudience.Private
         @Retries.RetryTranslated("Except if failOnMetadataWriteError=false, in which"
             + " case RetryExceptionsSwallowed")
      -  void finishedWrite(String key, long length)
      +  void finishedWrite(String key,
      +      long length,
      +      @Nullable final BulkOperationState operationState)
             throws MetadataPersistenceException {
           LOG.debug("Finished write to {}, len {}", key, length);
           Path p = keyToQualifiedPath(key);
      @@ -3405,11 +3409,12 @@ void finishedWrite(String key, long length)
           // See note about failure semantics in S3Guard documentation
           try {
             if (hasMetadataStore()) {
      -        S3Guard.addAncestors(metadataStore, p, username);
      +        S3Guard.addAncestors(metadataStore, p, username, operationState);
               S3AFileStatus status = createUploadFileStatus(p,
                   S3AUtils.objectRepresentsDirectory(key, length), length,
                   getDefaultBlockSize(p), username);
      -        S3Guard.putAndReturn(metadataStore, status, instrumentation);
      +        S3Guard.putAndReturn(metadataStore, status, instrumentation,
      +            operationState);
             }
           } catch (IOException e) {
             if (failOnMetadataWriteError) {
      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 ea091720c2705..080b5713f671d 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
      @@ -18,6 +18,7 @@
       
       package org.apache.hadoop.fs.s3a;
       
      +import javax.annotation.Nullable;
       import java.io.File;
       import java.io.FileNotFoundException;
       import java.io.IOException;
      @@ -48,6 +49,9 @@
       import org.apache.hadoop.classification.InterfaceAudience;
       import org.apache.hadoop.classification.InterfaceStability;
       import org.apache.hadoop.fs.Path;
      +import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
      +import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
      +import org.apache.hadoop.fs.s3a.s3guard.S3Guard;
       import org.apache.hadoop.fs.s3a.select.SelectBinding;
       import org.apache.hadoop.util.DurationInfo;
       
      @@ -226,13 +230,14 @@ public String initiateMultiPartUpload(String destKey) throws IOException {
         /**
          * Finalize a multipart PUT operation.
          * This completes the upload, and, if that works, calls
      -   * {@link S3AFileSystem#finishedWrite(String, long)} to update the filesystem.
      +   * {@link S3AFileSystem#finishedWrite(String, long, BulkOperationState)} to update the filesystem.
          * Retry policy: retrying, translated.
          * @param destKey destination of the commit
          * @param uploadId multipart operation Id
          * @param partETags list of partial uploads
          * @param length length of the upload
          * @param retrying retrying callback
      +   * @param operationState (nullable) operational state for a bulk update
          * @return the result of the operation.
          * @throws IOException on problems.
          */
      @@ -242,7 +247,8 @@ private CompleteMultipartUploadResult finalizeMultipartUpload(
             String uploadId,
             List partETags,
             long length,
      -      Retried retrying) throws IOException {
      +      Retried retrying,
      +      @Nullable BulkOperationState operationState) throws IOException {
           if (partETags.isEmpty()) {
             throw new IOException(
                 "No upload parts in multipart upload to " + destKey);
      @@ -260,7 +266,7 @@ private CompleteMultipartUploadResult finalizeMultipartUpload(
                         new ArrayList<>(partETags)));
               }
           );
      -    owner.finishedWrite(destKey, length);
      +    owner.finishedWrite(destKey, length, operationState);
           return uploadResult;
         }
       
      @@ -295,7 +301,8 @@ public CompleteMultipartUploadResult completeMPUwithRetries(
               uploadId,
               partETags,
               length,
      -        (text, e, r, i) -> errorCount.incrementAndGet());
      +        (text, e, r, i) -> errorCount.incrementAndGet(),
      +        null);
         }
       
         /**
      @@ -486,6 +493,53 @@ public void revertCommit(String destKey) throws IOException {
           );
         }
       
      +  /**
      +   * This completes a multipart upload to the destination key via
      +   * {@code finalizeMultipartUpload()}.
      +   * Retry policy: retrying, translated.
      +   * Retries increment the {@code errorCount} counter.
      +   * @param destKey destination
      +   * @param uploadId multipart operation Id
      +   * @param partETags list of partial uploads
      +   * @param length length of the upload
      +   * @param errorCount a counter incremented by 1 on every error; for
      +   * use in statistics
      +   * @param operationState operational state for a bulk update
      +   * @return the result of the operation.
      +   * @throws IOException if problems arose which could not be retried, or
      +   * the retry count was exceeded
      +   */
      +  @Retries.RetryTranslated
      +  public CompleteMultipartUploadResult commitUpload(
      +      String destKey,
      +      String uploadId,
      +      List partETags,
      +      long length,
      +      @Nullable BulkOperationState operationState)
      +      throws IOException {
      +    checkNotNull(uploadId);
      +    checkNotNull(partETags);
      +    LOG.debug("Completing multipart upload {} with {} parts",
      +        uploadId, partETags.size());
      +    return finalizeMultipartUpload(destKey,
      +        uploadId,
      +        partETags,
      +        length,
      +        Invoker.NO_OP,
      +        operationState);
      +  }
      +
      +  /**
      +   * Initiate a commit operation through any metastore.
      +   * @param path path under which the writes will all take place.
      +   * @return an possibly null operation state from the metastore.
      +   * @throws IOException failure to instantiate.
      +   */
      +  public BulkOperationState initiateCommitOperation(
      +      Path path) throws IOException {
      +    return S3Guard.initiateBulkWrite(owner.getMetadataStore(), path);
      +  }
      +
         /**
          * Upload part of a multi-partition file.
          * @param request request
      diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java
      index ed608cb983186..a49ab52b1ffd2 100644
      --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java
      +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java
      @@ -442,14 +442,27 @@ protected void commitPendingUploads(JobContext context,
           }
           LOG.debug("{}: committing the output of {} task(s)",
               getRole(), pending.size());
      -    Tasks.foreach(pending)
      -        .stopOnFailure()
      -        .executeWith(buildThreadPool(context))
      -        .onFailure((commit, exception) ->
      -                getCommitOperations().abortSingleCommit(commit))
      -        .abortWith(commit -> getCommitOperations().abortSingleCommit(commit))
      -        .revertWith(commit -> getCommitOperations().revertCommit(commit))
      -        .run(commit -> getCommitOperations().commitOrFail(commit));
      +    try(CommitOperations.CommitContext commitContext
      +            = initiateCommitOperation()) {
      +      Tasks.foreach(pending)
      +          .stopOnFailure()
      +          .executeWith(buildThreadPool(context))
      +          .onFailure((commit, exception) ->
      +              commitContext.abortSingleCommit(commit))
      +          .abortWith(commitContext::abortSingleCommit)
      +          .revertWith(commitContext::revertCommit)
      +          .run(commitContext::commitOrFail);
      +    }
      +  }
      +
      +  /**
      +   * Start the final commit/abort commit operations.
      +   * @return a commit context through which the operations can be invoked.
      +   * @throws IOException failure.
      +   */
      +  protected CommitOperations.CommitContext initiateCommitOperation()
      +      throws IOException {
      +    return getCommitOperations().initiateCommitOperation(getOutputPath());
         }
       
         /**
      @@ -531,7 +544,9 @@ protected void abortPendingUploadsInCleanup(
           Path dest = getOutputPath();
           try (DurationInfo d =
                    new DurationInfo(LOG, "Aborting all pending commits under %s",
      -                 dest)) {
      +                 dest);
      +         CommitOperations.CommitContext commitContext
      +             = initiateCommitOperation()) {
             CommitOperations ops = getCommitOperations();
             List pending;
             try {
      @@ -544,7 +559,8 @@ protected void abortPendingUploadsInCleanup(
             Tasks.foreach(pending)
                 .executeWith(buildThreadPool(getJobContext()))
                 .suppressExceptions(suppressExceptions)
      -          .run(u -> ops.abortMultipartCommit(u.getKey(), u.getUploadId()));
      +          .run(u -> commitContext.abortMultipartCommit(
      +              u.getKey(), u.getUploadId()));
           }
         }
       
      @@ -752,11 +768,13 @@ protected void abortPendingUploads(JobContext context,
             LOG.info("{}: no pending commits to abort", getRole());
           } else {
             try (DurationInfo d = new DurationInfo(LOG,
      -          "Aborting %s uploads", pending.size())) {
      +          "Aborting %s uploads", pending.size());
      +           CommitOperations.CommitContext commitContext
      +               = initiateCommitOperation()) {
               Tasks.foreach(pending)
                   .executeWith(buildThreadPool(context))
                   .suppressExceptions(suppressExceptions)
      -            .run(commit -> getCommitOperations().abortSingleCommit(commit));
      +            .run(commitContext::abortSingleCommit);
             }
           }
         }
      diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java
      index 55ace17b8a21e..7bee5c9f48927 100644
      --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java
      +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java
      @@ -18,13 +18,14 @@
       
       package org.apache.hadoop.fs.s3a.commit;
       
      +import javax.annotation.Nullable;
      +import java.io.Closeable;
       import java.io.File;
       import java.io.FileNotFoundException;
       import java.io.IOException;
       import java.util.ArrayList;
       import java.util.List;
       import java.util.Map;
      -import java.util.concurrent.atomic.AtomicInteger;
       import java.util.stream.Collectors;
       import java.util.stream.IntStream;
       
      @@ -49,6 +50,8 @@
       import org.apache.hadoop.fs.s3a.commit.files.PendingSet;
       import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit;
       import org.apache.hadoop.fs.s3a.commit.files.SuccessData;
      +import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
      +import org.apache.hadoop.io.IOUtils;
       
       import static org.apache.hadoop.fs.s3a.S3AUtils.*;
       import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
      @@ -128,10 +131,12 @@ protected S3AInstrumentation.CommitterStatistics getStatistics() {
         /**
          * Commit the operation, throwing an exception on any failure.
          * @param commit commit to execute
      +   * @param operationState S3Guard state of ongoing operation.
          * @throws IOException on a failure
          */
      -  public void commitOrFail(SinglePendingCommit commit) throws IOException {
      -    commit(commit, commit.getFilename()).maybeRethrow();
      +  private void commitOrFail(SinglePendingCommit commit,
      +      final BulkOperationState operationState) throws IOException {
      +    commit(commit, commit.getFilename(), operationState).maybeRethrow();
         }
       
         /**
      @@ -139,16 +144,20 @@ public void commitOrFail(SinglePendingCommit commit) throws IOException {
          * and converted to an outcome.
          * @param commit entry to commit
          * @param origin origin path/string for outcome text
      +   * @param operationState S3Guard state of ongoing operation.
          * @return the outcome
          */
      -  public MaybeIOE commit(SinglePendingCommit commit, String origin) {
      +  private MaybeIOE commit(
      +      final SinglePendingCommit commit,
      +      final String origin,
      +      final BulkOperationState operationState) {
           LOG.debug("Committing single commit {}", commit);
           MaybeIOE outcome;
           String destKey = "unknown destination";
           try {
             commit.validate();
             destKey = commit.getDestinationKey();
      -      long l = innerCommit(commit);
      +      long l = innerCommit(commit, operationState);
             LOG.debug("Successful commit of file length {}", l);
             outcome = MaybeIOE.NONE;
             statistics.commitCompleted(commit.getLength());
      @@ -171,17 +180,20 @@ public MaybeIOE commit(SinglePendingCommit commit, String origin) {
         /**
          * Inner commit operation.
          * @param commit entry to commit
      +   * @param operationState S3Guard state of ongoing operation.
          * @return bytes committed.
          * @throws IOException failure
          */
      -  private long innerCommit(SinglePendingCommit commit) throws IOException {
      +  private long innerCommit(
      +      final SinglePendingCommit commit,
      +      final BulkOperationState operationState) throws IOException {
           // finalize the commit
      -    writeOperations.completeMPUwithRetries(
      +    writeOperations.commitUpload(
               commit.getDestinationKey(),
                     commit.getUploadId(),
                     toPartEtags(commit.getEtags()),
                     commit.getLength(),
      -              new AtomicInteger(0));
      +              operationState);
           return commit.getLength();
         }
       
      @@ -249,7 +261,7 @@ public IOException makeIOE(String key, Exception ex) {
          * @throws FileNotFoundException if the abort ID is unknown
          * @throws IOException on any failure
          */
      -  public void abortSingleCommit(SinglePendingCommit commit)
      +  private void abortSingleCommit(SinglePendingCommit commit)
             throws IOException {
           String destKey = commit.getDestinationKey();
           String origin = commit.getFilename() != null
      @@ -268,7 +280,7 @@ public void abortSingleCommit(SinglePendingCommit commit)
          * @throws FileNotFoundException if the abort ID is unknown
          * @throws IOException on any failure
          */
      -  public void abortMultipartCommit(String destKey, String uploadId)
      +  private void abortMultipartCommit(String destKey, String uploadId)
             throws IOException {
           try {
             writeOperations.abortMultipartCommit(destKey, uploadId);
      @@ -520,6 +532,103 @@ public void jobCompleted(boolean success) {
           statistics.jobCompleted(success);
         }
       
      +  /**
      +   * Begin the final commit.
      +   * @param path path for all work.
      +   * @return the commit context to pass in.
      +   * @throws IOException failure.
      +   */
      +  public CommitContext initiateCommitOperation(Path path) throws IOException {
      +    return new CommitContext(writeOperations.initiateCommitOperation(path));
      +  }
      +
      +  /**
      +   * Commit context.
      +   *
      +   * It is used to manage the final commit sequence where files become
      +   * visible. It contains a {@link BulkOperationState} field, which, if
      +   * there is a metastore, will be requested from the store so that it
      +   * can track multiple creation operations within the same overall operation.
      +   * This will be null if there is no metastore, or the store chooses not
      +   * to provide one.
      +   *
      +   * This can only be created through {@link #initiateCommitOperation(Path)}.
      +   *
      +   * Once the commit operation has completed, it must be closed.
      +   * It must not be reused.
      +   */
      +  public final class CommitContext implements Closeable {
      +
      +    private final BulkOperationState operationState;
      +
      +    private CommitContext(@Nullable final BulkOperationState operationState) {
      +      this.operationState = operationState;
      +    }
      +
      +    /**
      +     * Commit the operation, throwing an exception on any failure.
      +     * See {@link CommitOperations#commitOrFail(SinglePendingCommit, BulkOperationState)}.
      +     * @param commit commit to execute
      +     * @throws IOException on a failure
      +     */
      +    public void commitOrFail(SinglePendingCommit commit) throws IOException {
      +      CommitOperations.this.commitOrFail(commit, operationState);
      +    }
      +
      +    /**
      +     * Commit a single pending commit; exceptions are caught
      +     * and converted to an outcome.
      +     * See {@link CommitOperations#commit(SinglePendingCommit, String, BulkOperationState)}.
      +     * @param commit entry to commit
      +     * @param origin origin path/string for outcome text
      +     * @return the outcome
      +     */
      +    public MaybeIOE commit(SinglePendingCommit commit,
      +        String origin) {
      +      return CommitOperations.this.commit(commit, origin, operationState);
      +    }
      +
      +    /**
      +     * See {@link CommitOperations#abortSingleCommit(SinglePendingCommit)}.
      +     * @param commit pending commit to abort
      +     * @throws FileNotFoundException if the abort ID is unknown
      +     * @throws IOException on any failure
      +     */
      +    public void abortSingleCommit(final SinglePendingCommit commit)
      +        throws IOException {
      +      CommitOperations.this.abortSingleCommit(commit);
      +    }
      +
      +    /**
      +     * See {@link CommitOperations#revertCommit(SinglePendingCommit)}.
      +     * @param commit pending commit
      +     * @throws IOException failure
      +     */
      +    public void revertCommit(final SinglePendingCommit commit)
      +        throws IOException {
      +      CommitOperations.this.revertCommit(commit);
      +    }
      +
      +    /**
      +     * See {@link CommitOperations#abortMultipartCommit(String, String)}..
      +     * @param destKey destination key
      +     * @param uploadId upload to cancel
      +     * @throws FileNotFoundException if the abort ID is unknown
      +     * @throws IOException on any failure
      +     */
      +    public void abortMultipartCommit(
      +        final String destKey,
      +        final String uploadId)
      +        throws IOException {
      +      CommitOperations.this.abortMultipartCommit(destKey, uploadId);
      +    }
      +
      +    @Override
      +    public void close() throws IOException {
      +      IOUtils.cleanupWithLogger(LOG, operationState);
      +    }
      +  }
      +
         /**
          * A holder for a possible IOException; the call {@link #maybeRethrow()}
          * will throw any exception passed into the constructor, and be a no-op
      diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitter.java
      index f26384de49e83..518d789718a80 100644
      --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitter.java
      +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitter.java
      @@ -41,6 +41,7 @@
       import org.apache.hadoop.fs.s3a.S3AFileSystem;
       import org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitter;
       import org.apache.hadoop.fs.s3a.commit.CommitConstants;
      +import org.apache.hadoop.fs.s3a.commit.CommitOperations;
       import org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants;
       import org.apache.hadoop.fs.s3a.commit.Tasks;
       import org.apache.hadoop.fs.s3a.commit.files.PendingSet;
      @@ -729,9 +730,14 @@ protected int commitTaskInternal(final TaskAttemptContext context,
                 LOG.error(
                     "{}: Exception during commit process, aborting {} commit(s)",
                     getRole(), commits.size());
      -          Tasks.foreach(commits)
      -              .suppressExceptions()
      -              .run(commit -> getCommitOperations().abortSingleCommit(commit));
      +          try(CommitOperations.CommitContext commitContext
      +                  = initiateCommitOperation();
      +              DurationInfo ignored = new DurationInfo(LOG,
      +                  "Aborting %s uploads", commits.size())) {
      +            Tasks.foreach(commits)
      +                .suppressExceptions()
      +                .run(commitContext::abortSingleCommit);
      +          }
                 deleteTaskAttemptPathQuietly(context);
               }
             }
      diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/BulkOperationState.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/BulkOperationState.java
      new file mode 100644
      index 0000000000000..9f2b90c2b5256
      --- /dev/null
      +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/BulkOperationState.java
      @@ -0,0 +1,42 @@
      +/*
      + * Licensed to the Apache Software Foundation (ASF) under one
      + * or more contributor license agreements.  See the NOTICE file
      + * distributed with this work for additional information
      + * regarding copyright ownership.  The ASF licenses this file
      + * to you under the Apache License, Version 2.0 (the
      + * "License"); you may not use this file except in compliance
      + * with the License.  You may obtain a copy of the License at
      + *
      + *     http://www.apache.org/licenses/LICENSE-2.0
      + *
      + * Unless required by applicable law or agreed to in writing, software
      + * distributed under the License is distributed on an "AS IS" BASIS,
      + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
      + * See the License for the specific language governing permissions and
      + * limitations under the License.
      + */
      +
      +package org.apache.hadoop.fs.s3a.s3guard;
      +
      +import java.io.Closeable;
      +import java.io.IOException;
      +
      +/**
      + * This represents state which may be passed to bulk IO operations
      + * to enable them to store information about the state of the ongoing
      + * operation across invocations.
      + *
      + * A bulk operation state MUST only be be used for the single store
      + * from which it was created, and MUSTonly for the duration of a single
      + * bulk update operation.
      + *
      + * After the operation has completed, it MUST be closed so
      + * as to guarantee that all state is released.
      + */
      +public class BulkOperationState implements Closeable {
      +
      +  @Override
      +  public void close() throws IOException {
      +
      +  }
      +}
      diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameTracker.java
      index d0a832e421286..92381fc467d7b 100644
      --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameTracker.java
      +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameTracker.java
      @@ -18,7 +18,6 @@
       
       package org.apache.hadoop.fs.s3a.s3guard;
       
      -import java.io.Closeable;
       import java.io.IOException;
       import java.util.ArrayList;
       import java.util.Collection;
      @@ -57,9 +56,9 @@ public DelayedUpdateRenameTracker(
             final MetadataStore metadataStore,
             final Path sourceRoot,
             final Path dest,
      -      final Closeable moveState) {
      +      final BulkOperationState operationState) {
           super("DelayedUpdateRenameTracker", storeContext, metadataStore, sourceRoot, dest,
      -        moveState);
      +        operationState);
           this.metadataStore = storeContext.getMetadataStore();
         }
       
      @@ -127,7 +126,7 @@ public synchronized void sourceObjectsDeleted(
       
         @Override
         public void completeRename() throws IOException {
      -    metadataStore.move(sourcePaths, destMetas, getMoveState());
      +    metadataStore.move(sourcePaths, destMetas, getOperationState());
           super.completeRename();
         }
       
      @@ -137,7 +136,7 @@ public IOException renameFailed(final Exception ex) {
           try {
             // the destination paths are updated; the source is left alone.
             // either the delete operation didn't begin, or the
      -      metadataStore.move(new ArrayList<>(0), destMetas, getMoveState());
      +      metadataStore.move(new ArrayList<>(0), destMetas, getOperationState());
             for (Path deletedPath : deletedPaths) {
               // this is not ideal in that it may leave parent stuff around.
               metadataStore.delete(deletedPath);
      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 637d75ce109a8..0c84aaae900d5 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
      @@ -19,7 +19,6 @@
       package org.apache.hadoop.fs.s3a.s3guard;
       
       import javax.annotation.Nullable;
      -import java.io.Closeable;
       import java.io.FileNotFoundException;
       import java.io.IOException;
       import java.io.InterruptedIOException;
      @@ -783,15 +782,16 @@ private Collection completeAncestry(
          * @param pathsToCreate Collection of all PathMetadata for the new paths
          *                      that were created at the destination of the rename
          *                      ().
      -   * @param closeable
      -   * @throws IOException
      +   * @param operationState Any ongoing state supplied to the rename tracker
      +   *                      which is to be passed in with each move operation.
      +   * @throws IOException if there is an error
          */
         @Override
         @Retries.RetryTranslated
         public void move(
             @Nullable Collection pathsToDelete,
             @Nullable Collection pathsToCreate,
      -      @Nullable final Closeable closeable) throws IOException {
      +      @Nullable final BulkOperationState operationState) throws IOException {
           if (pathsToDelete == null && pathsToCreate == null) {
             return;
           }
      @@ -808,15 +808,17 @@ public void move(
           // Following code is to maintain this invariant by putting all ancestor
           // directories of the paths to create.
           // ancestor paths that are not explicitly added to paths to create
      -    AncestorState ancestorState = extractOrCreate(closeable);
      +    AncestorState ancestorState = extractOrCreate(operationState);
           List newItems = new ArrayList<>();
           if (pathsToCreate != null) {
             // create all parent entries.
             // this is synchronized on the move state so that across both serialized
             // and parallelized renames, duplicate ancestor entries are not created.
             synchronized (ancestorState) {
      -        newItems.addAll(completeAncestry(pathMetaToDDBPathMeta(pathsToCreate),
      -            ancestorState));
      +        newItems.addAll(
      +            completeAncestry(
      +                pathMetaToDDBPathMeta(pathsToCreate),
      +                ancestorState));
             }
           }
           // sort all the new items topmost first.
      @@ -959,7 +961,15 @@ private void retryBackoffOnBatchWrite(int retryCount) throws IOException {
       
         @Override
         @Retries.RetryTranslated
      -  public void put(PathMetadata meta) throws IOException {
      +  public void put(final PathMetadata meta) throws IOException {
      +    put(meta, null);
      +  }
      +
      +  @Override
      +  @Retries.RetryTranslated
      +  public void put(
      +      final PathMetadata meta,
      +      @Nullable final BulkOperationState operationState) throws IOException {
           // For a deeply nested path, this method will automatically create the full
           // ancestry and save respective item in DynamoDB table.
           // So after put operation, we maintain the invariant that if a path exists,
      @@ -970,25 +980,35 @@ public void put(PathMetadata meta) throws IOException {
       
           Collection wrapper = new ArrayList<>(1);
           wrapper.add(meta);
      -    put(wrapper);
      +    put(wrapper, operationState);
         }
       
         @Override
         @Retries.RetryTranslated
      -  public void put(Collection metas) throws IOException {
      -    innerPut(pathMetaToDDBPathMeta(metas));
      +  public void put(
      +      final Collection metas,
      +      @Nullable final BulkOperationState operationState) throws IOException {
      +    innerPut(pathMetaToDDBPathMeta(metas), operationState);
         }
       
      -  @Retries.OnceRaw
      -  private void innerPut(Collection metas) throws IOException {
      +  @Retries.RetryTranslated
      +  private void innerPut(
      +      final Collection metas,
      +      @Nullable final BulkOperationState operationState) throws IOException {
           if (metas.isEmpty()) {
      -      // this seems to appear in the logs, so log the full stack to
      +      // this sometimes to appear in the logs, so log the full stack to
             // identify it.
             LOG.debug("Ignoring empty list of entries to put",
                 new Exception("source"));
             return;
           }
      -    Item[] items = pathMetadataToItem(completeAncestry(metas, new AncestorState()));
      +    final AncestorState ancestorState = extractOrCreate(operationState);
      +
      +    Item[] items;
      +    synchronized (ancestorState) {
      +      items = pathMetadataToItem(
      +          completeAncestry(metas, ancestorState));
      +    }
           LOG.debug("Saving batch of {} items to table {}, region {}", items.length,
               tableName, region);
           processBatchWriteRequest(null, items);
      @@ -1232,7 +1252,7 @@ private void removeAuthoritativeDirFlag(Set pathSet)
           try {
             LOG.debug("innerPut on metas: {}", metas);
             if (!metas.isEmpty()) {
      -        innerPut(metas);
      +        innerPut(metas, null);
             }
           } catch (IOException e) {
             String msg = String.format("IOException while setting false "
      @@ -1880,13 +1900,13 @@ static IOException translateTableWaitFailure(
       
         /**
          * Get the move state passed in; create a new one if needed.
      -   * @param closeable state.
      +   * @param state state.
          * @return the cast or created state.
          */
         @VisibleForTesting
      -  static AncestorState extractOrCreate(@Nullable Closeable closeable) {
      -    if (closeable != null) {
      -      return (AncestorState) closeable;
      +  static AncestorState extractOrCreate(@Nullable BulkOperationState state) {
      +    if (state != null) {
      +      return (AncestorState) state;
           } else {
             return new AncestorState();
           }
      @@ -1899,7 +1919,7 @@ static AncestorState extractOrCreate(@Nullable Closeable closeable) {
          * rename operations managed by a rename tracker.
          */
         @VisibleForTesting
      -  static final class AncestorState implements Closeable {
      +  static final class AncestorState extends BulkOperationState {
       
           private final Map ancestry = new HashMap<>();
       
      @@ -1907,9 +1927,5 @@ public Map getAncestry() {
             return ancestry;
           }
       
      -    @Override
      -    public void close() {
      -
      -    }
         }
       }
      diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java
      index 0185fc46e0f8e..bc728b6258944 100644
      --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java
      +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java
      @@ -36,7 +36,6 @@
       import org.slf4j.Logger;
       import org.slf4j.LoggerFactory;
       
      -import java.io.Closeable;
       import java.io.IOException;
       import java.net.URI;
       import java.util.Collection;
      @@ -198,7 +197,7 @@ public synchronized DirListingMetadata listChildren(Path p) throws
         public void move(
             @Nullable Collection pathsToDelete,
             @Nullable Collection pathsToCreate,
      -      @Nullable final Closeable moveState) throws IOException {
      +      @Nullable final BulkOperationState operationState) throws IOException {
           LOG.info("Move {} to {}", pathsToDelete, pathsToCreate);
       
           if (pathsToCreate == null) {
      @@ -220,7 +219,7 @@ public void move(
             // 2. Create new destination path metadata
             for (PathMetadata meta : pathsToCreate) {
               LOG.debug("move: adding metadata {}", meta);
      -        put(meta);
      +        put(meta, null);
             }
       
             // 3. We now know full contents of all dirs in destination subtree
      @@ -238,7 +237,13 @@ public void move(
         }
       
         @Override
      -  public void put(PathMetadata meta) throws IOException {
      +  public void put(final PathMetadata meta) throws IOException {
      +    put(meta, null);
      +  }
      +
      +  @Override
      +  public void put(PathMetadata meta,
      +      final BulkOperationState operationState) throws IOException {
       
           Preconditions.checkNotNull(meta);
           FileStatus status = meta.getFileStatus();
      @@ -318,13 +323,14 @@ public synchronized void put(DirListingMetadata meta) throws IOException {
           } else {
             entry.setDirListingMetadata(meta);
           }
      -    put(meta.getListing());
      +    put(meta.getListing(), null);
         }
       
      -  public synchronized void put(Collection metas) throws
      +  public synchronized void put(Collection metas,
      +      final BulkOperationState operationState) throws
             IOException {
           for (PathMetadata meta : metas) {
      -      put(meta);
      +      put(meta, operationState);
           }
         }
       
      diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java
      index a68da964c227f..5d46ba9a956e0 100644
      --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java
      +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java
      @@ -153,14 +153,14 @@ PathMetadata get(Path path, boolean wantEmptyDirectoryFlag)
          *                      source directory tree of the move.
          * @param pathsToCreate Collection of all PathMetadata for the new paths
          *                      that were created at the destination of the rename().
      -   * @param moveState     Any ongoing state supplied to the rename tracker
      +   * @param operationState     Any ongoing state supplied to the rename tracker
          *                      which is to be passed in with each move operation.
          * @throws IOException if there is an error
          */
         void move(
             @Nullable Collection pathsToDelete,
             @Nullable Collection pathsToCreate,
      -      @Nullable Closeable moveState) throws IOException;
      +      @Nullable BulkOperationState operationState) throws IOException;
       
         /**
          * Saves metadata for exactly one path.
      @@ -175,15 +175,33 @@ void move(
         @RetryTranslated
         void put(PathMetadata meta) throws IOException;
       
      +  /**
      +   * Saves metadata for exactly one path, potentially
      +   * using any bulk operation state to eliminate duplicate work.
      +   *
      +   * Implementations may pre-create all the path's ancestors automatically.
      +   * Implementations must update any {@code DirListingMetadata} objects which
      +   * track the immediate parent of this file.
      +   *
      +   * @param meta the metadata to save
      +   * @param operationState operational state for a bulk update
      +   * @throws IOException if there is an error
      +   */
      +  @RetryTranslated
      +  void put(PathMetadata meta,
      +      @Nullable BulkOperationState operationState) throws IOException;
      +
         /**
          * Saves metadata for any number of paths.
          *
          * Semantics are otherwise the same as single-path puts.
          *
          * @param metas the metadata to save
      +   * @param operationState (nullable) operational state for a bulk update
          * @throws IOException if there is an error
          */
      -  void put(Collection metas) throws IOException;
      +  void put(Collection metas,
      +      @Nullable BulkOperationState operationState) throws IOException;
       
         /**
          * Save directory listing metadata. Callers may save a partial directory
      @@ -275,4 +293,16 @@ RenameTracker initiateRenameOperation(
             FileStatus sourceStatus,
             Path dest)
             throws IOException;
      +
      +  /**
      +   * Initiate a bulk update and create an operation state for it.
      +   * This may then be passed into put operations.
      +   * @param dest path under which updates will be explicitly put.
      +   * @return null or a store-specific state to pass into the put operations.
      +   * @throws IOException failure
      +   */
      +  default BulkOperationState initiateBulkWrite(Path dest) throws IOException {
      +    return null;
      +  }
      +
       }
      diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java
      index 82821cad494a3..67d07a3d93a89 100644
      --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java
      +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java
      @@ -24,7 +24,6 @@
       import org.apache.hadoop.fs.Path;
       import org.apache.hadoop.fs.s3a.impl.StoreContext;
       
      -import java.io.Closeable;
       import java.io.IOException;
       import java.util.Collection;
       import java.util.HashMap;
      @@ -80,15 +79,21 @@ public DirListingMetadata listChildren(Path path) throws IOException {
         @Override
         public void move(Collection pathsToDelete,
             Collection pathsToCreate,
      -      final Closeable moveState) throws IOException {
      +      final BulkOperationState operationState) throws IOException {
         }
       
         @Override
      -  public void put(PathMetadata meta) throws IOException {
      +  public void put(final PathMetadata meta) throws IOException {
         }
       
         @Override
      -  public void put(Collection meta) throws IOException {
      +  public void put(PathMetadata meta,
      +      final BulkOperationState operationState) throws IOException {
      +  }
      +
      +  @Override
      +  public void put(Collection meta,
      +      final BulkOperationState operationState) throws IOException {
         }
       
         @Override
      diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ProgressiveRenameTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ProgressiveRenameTracker.java
      index 69052986c71ad..93a39fc332c6f 100644
      --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ProgressiveRenameTracker.java
      +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ProgressiveRenameTracker.java
      @@ -18,7 +18,6 @@
       
       package org.apache.hadoop.fs.s3a.s3guard;
       
      -import java.io.Closeable;
       import java.io.IOException;
       import java.util.ArrayList;
       import java.util.Collection;
      @@ -82,9 +81,9 @@ public ProgressiveRenameTracker(
             final MetadataStore metadataStore,
             final Path sourceRoot,
             final Path dest,
      -      final Closeable moveState) {
      +      final BulkOperationState operationState) {
           super("ProgressiveRenameTracker",
      -        storeContext, metadataStore, sourceRoot, dest, moveState);
      +        storeContext, metadataStore, sourceRoot, dest, operationState);
         }
       
         /**
      @@ -141,7 +140,7 @@ public void fileCopied(
       
           // outside the lock, the entriesToAdd list has all new files to create.
           // ...so update the store.
      -    store.move(null, entriesToAdd, getMoveState());
      +    store.move(null, entriesToAdd, getOperationState());
         }
       
         /**
      @@ -181,7 +180,7 @@ public void directoryMarkerCopied(final FileStatus sourceStatus,
           // ...so update the store.
           try (DurationInfo ignored = new DurationInfo(LOG, false,
               "adding %s metastore entries", entriesToAdd.size())) {
      -      store.move(null, entriesToAdd, getMoveState());
      +      store.move(null, entriesToAdd, getOperationState());
           }
         }
       
      @@ -207,7 +206,7 @@ public void sourceObjectsDeleted(
           // delete the paths from the metastore
           try (DurationInfo ignored = new DurationInfo(LOG, false,
               "delete %s metastore entries", paths.size())) {
      -      getMetadataStore().move(paths, null, getMoveState());
      +      getMetadataStore().move(paths, null, getOperationState());
           }
         }
       
      @@ -215,7 +214,7 @@ public void sourceObjectsDeleted(
         public void completeRename() throws IOException {
           // this should all have happened.
           LOG.debug("Rename completed for {}", this);
      -    getMetadataStore().move(pathsToDelete, destMetas, getMoveState());
      +    getMetadataStore().move(pathsToDelete, destMetas, getOperationState());
           super.completeRename();
         }
       
      diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RenameTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RenameTracker.java
      index 03b6d43c6716a..fac1784d43d0a 100644
      --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RenameTracker.java
      +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RenameTracker.java
      @@ -18,7 +18,6 @@
       
       package org.apache.hadoop.fs.s3a.s3guard;
       
      -import java.io.Closeable;
       import java.io.IOException;
       import java.util.List;
       
      @@ -41,7 +40,7 @@
        * as initiated in the S3AFilesystem rename.
        * Subclasses must provide an implementation and return it in
        * {@link MetadataStore#initiateRenameOperation(StoreContext, Path, FileStatus, Path)}.
      - * The {@code moveState} field/constructor argument is an opaque state to
      + * The {@link #operationState} field/constructor argument is an opaque state to
        * be passed down to the metastore in its move operations; this allows the
        * stores to manage ongoing state -while still being able to share
        * rename tracker implementations.
      @@ -75,7 +74,7 @@ public abstract class RenameTracker extends StoreOperation {
          * which is to be passed in with each move operation.
          * This must be closed at the end of the tracker's life.
          */
      -  private final Closeable moveState;
      +  private final BulkOperationState operationState;
       
         /**
          * The metadata store for this tracker.
      @@ -93,7 +92,7 @@ public abstract class RenameTracker extends StoreOperation {
          * @param metadataStore the stopre
          * @param sourceRoot source path.
          * @param dest destination path.
      -   * @param moveState ongoing move state.
      +   * @param operationState ongoing move state.
          */
         protected RenameTracker(
             final String name,
      @@ -101,13 +100,13 @@ protected RenameTracker(
             final MetadataStore metadataStore,
             final Path sourceRoot,
             final Path dest,
      -      Closeable moveState) {
      +      final BulkOperationState operationState) {
           super(checkNotNull(storeContext));
           checkNotNull(storeContext.getUsername(), "No username");
           this.metadataStore = checkNotNull(metadataStore);
           this.sourceRoot = checkNotNull(sourceRoot);
           this.dest = checkNotNull(dest);
      -    this.moveState = moveState;
      +    this.operationState = operationState;
           this.name = String.format("%s (%s, %s)", name, sourceRoot, dest);
           durationInfo = new DurationInfo(LOG, false,
               name +" (%s, %s)", sourceRoot, dest);
      @@ -130,8 +129,8 @@ public String getOwner() {
           return getStoreContext().getUsername();
         }
       
      -  public Closeable getMoveState() {
      -    return moveState;
      +  public BulkOperationState getOperationState() {
      +    return operationState;
         }
       
         /**
      @@ -214,7 +213,7 @@ public void sourceObjectsDeleted(
          * @throws IOException failure.
          */
         public void completeRename() throws IOException {
      -    IOUtils.cleanupWithLogger(LOG, moveState);
      +    IOUtils.cleanupWithLogger(LOG, operationState);
           noteRenameFinished();
         }
       
      @@ -242,7 +241,7 @@ protected void noteRenameFinished() {
          */
         public IOException renameFailed(Exception ex) {
           LOG.debug("Rename has failed", ex);
      -    IOUtils.cleanupWithLogger(LOG, moveState);
      +    IOUtils.cleanupWithLogger(LOG, operationState);
           noteRenameFinished();
           return convertToIOException(ex);
         }
      diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java
      index 5b614a6a82992..a1e35e493e22f 100644
      --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java
      +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java
      @@ -18,6 +18,7 @@
       
       package org.apache.hadoop.fs.s3a.s3guard;
       
      +import javax.annotation.Nullable;
       import java.io.FileNotFoundException;
       import java.io.IOException;
       import java.net.URI;
      @@ -135,7 +136,6 @@ static Class getMetadataStoreClass(
           return aClass;
         }
       
      -
         /**
          * Helper function which puts a given S3AFileStatus into the MetadataStore and
          * returns the same S3AFileStatus. Instrumentation monitors the put operation.
      @@ -149,18 +149,55 @@ static Class getMetadataStoreClass(
         public static S3AFileStatus putAndReturn(MetadataStore ms,
             S3AFileStatus status,
             S3AInstrumentation instrumentation) throws IOException {
      +    return putAndReturn(ms, status, instrumentation, null);
      +  }
      +
      +  /**
      +   * Helper function which puts a given S3AFileStatus into the MetadataStore and
      +   * returns the same S3AFileStatus. Instrumentation monitors the put operation.
      +   * @param ms MetadataStore to {@code put()} into.
      +   * @param status status to store
      +   * @param instrumentation instrumentation of the s3a file system
      +   * @param operationState possibly-null metastore state tracker.
      +   * @return The same status as passed in
      +   * @throws IOException if metadata store update failed
      +   */
      +  @RetryTranslated
      +  public static S3AFileStatus putAndReturn(
      +      final MetadataStore ms,
      +      final S3AFileStatus status,
      +      final S3AInstrumentation instrumentation,
      +      @Nullable final BulkOperationState operationState) throws IOException {
           long startTimeNano = System.nanoTime();
           try {
      -      ms.put(new PathMetadata(status));
      +      ms.put(new PathMetadata(status), operationState);
           } finally {
             instrumentation.addValueToQuantiles(S3GUARD_METADATASTORE_PUT_PATH_LATENCY,
                 (System.nanoTime() - startTimeNano));
             instrumentation.incrementCounter(S3GUARD_METADATASTORE_PUT_PATH_REQUEST, 1);
           }
      -
           return status;
         }
       
      +  /**
      +   * Initiate a bulk update and create an operation state for it.
      +   * This may then be passed into put operations.
      +   * @param metastore store
      +   * @param path path under which updates will be explicitly put.
      +   * @return a store-specific state to pass into the put operations, or null
      +   * @throws IOException failure
      +   */
      +  public static BulkOperationState initiateBulkWrite(
      +      @Nullable final MetadataStore metastore,
      +      final Path path) throws IOException {
      +    if (metastore == null || isNullMetadataStore(metastore)) {
      +      return null;
      +    } else {
      +      return metastore.initiateBulkWrite(path);
      +    }
      +  }
      +
      +
         /**
          * Convert the data of a directory listing to an array of {@link FileStatus}
          * entries. Tombstones are filtered out at this point. If the listing is null
      @@ -246,7 +283,7 @@ public static FileStatus[] dirListingUnion(MetadataStore ms, Path path,
               if (status != null
                   && s.getModificationTime() > status.getModificationTime()) {
                 LOG.debug("Update ms with newer metadata of: {}", status);
      -          ms.put(new PathMetadata(s));
      +          ms.put(new PathMetadata(s), null);
               }
             }
       
      @@ -361,7 +398,7 @@ public static void makeDirsOrdered(MetadataStore ms, List dirs,
             }
       
             // Batched put
      -      ms.put(pathMetas);
      +      ms.put(pathMetas, null);
           } catch (IOException ioe) {
             LOG.error("MetadataStore#put() failure:", ioe);
           }
      @@ -462,8 +499,19 @@ public static void addMoveAncestors(MetadataStore ms,
           }
         }
       
      +  /**
      +   * This adds all new ancestors of a path as directories
      +   * @param metadataStore store
      +   * @param qualifiedPath path to update
      +   * @param username username to use in all new FileStatus entries.
      +   * @param operationState (nullable) operational state for a bulk update
      +   * @throws IOException failure
      +   */
      +  @Retries.RetryTranslated
         public static void addAncestors(MetadataStore metadataStore,
      -      Path qualifiedPath, String username) throws IOException {
      +      Path qualifiedPath,
      +      String username,
      +      @Nullable final BulkOperationState operationState) throws IOException {
           Collection newDirs = new ArrayList<>();
           Path parent = qualifiedPath.getParent();
           while (!parent.isRoot()) {
      @@ -479,7 +527,7 @@ public static void addAncestors(MetadataStore metadataStore,
             parent = parent.getParent();
           }
           if (!newDirs.isEmpty()) {
      -      metadataStore.put(newDirs);
      +      metadataStore.put(newDirs, operationState);
           }
         }
       
      diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java
      index 1ac167f5a6dc6..c7700f2d065d9 100644
      --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java
      +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java
      @@ -18,6 +18,7 @@
       
       package org.apache.hadoop.fs.s3a.s3guard;
       
      +import javax.annotation.Nullable;
       import java.io.FileNotFoundException;
       import java.io.IOException;
       import java.io.PrintStream;
      @@ -694,9 +695,11 @@ public String getUsage() {
            * Put parents into MS and cache if the parents are not presented.
            *
            * @param f the file or an empty directory.
      +     * @param operationState store's bulk update state.
            * @throws IOException on I/O errors.
            */
      -    private void putParentsIfNotPresent(FileStatus f) throws IOException {
      +    private void putParentsIfNotPresent(FileStatus f,
      +        @Nullable BulkOperationState operationState) throws IOException {
             Preconditions.checkNotNull(f);
             Path parent = f.getPath().getParent();
             while (parent != null) {
      @@ -705,7 +708,7 @@ private void putParentsIfNotPresent(FileStatus f) throws IOException {
               }
               FileStatus dir = DynamoDBMetadataStore.makeDirStatus(parent,
                   f.getOwner());
      -        getStore().put(new PathMetadata(dir));
      +        getStore().put(new PathMetadata(dir), operationState);
               dirCache.add(parent);
               parent = parent.getParent();
             }
      @@ -718,10 +721,11 @@ private void putParentsIfNotPresent(FileStatus f) throws IOException {
            */
           private long importDir(FileStatus status) throws IOException {
             Preconditions.checkArgument(status.isDirectory());
      +      BulkOperationState operationState = getStore().initiateBulkWrite(
      +          status.getPath());
             RemoteIterator it = getFilesystem()
                 .listFilesAndEmptyDirectories(status.getPath(), true);
             long items = 0;
      -
             while (it.hasNext()) {
               LocatedFileStatus located = it.next();
               FileStatus child;
      @@ -736,8 +740,8 @@ private long importDir(FileStatus status) throws IOException {
                     located.getBlockSize(),
                     located.getOwner());
               }
      -        putParentsIfNotPresent(child);
      -        getStore().put(new PathMetadata(child));
      +        putParentsIfNotPresent(child, operationState);
      +        getStore().put(new PathMetadata(child), operationState);
               items++;
             }
             return items;
      @@ -772,7 +776,7 @@ public int run(String[] args, PrintStream out) throws Exception {
             long items = 1;
             if (status.isFile()) {
               PathMetadata meta = new PathMetadata(status);
      -        getStore().put(meta);
      +        getStore().put(meta, null);
             } else {
               items = importDir(status);
             }
      diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetadataPersistenceException.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetadataPersistenceException.java
      index 26661a36090ed..3662194d34b50 100644
      --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetadataPersistenceException.java
      +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetadataPersistenceException.java
      @@ -33,6 +33,7 @@
       import org.apache.hadoop.fs.FileSystem;
       import org.apache.hadoop.fs.Path;
       import org.apache.hadoop.fs.s3a.S3ATestUtils.MetricDiff;
      +import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
       import org.apache.hadoop.fs.s3a.s3guard.LocalMetadataStore;
       import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
       import org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore;
      @@ -129,8 +130,15 @@ private IOExceptionMetadataStore(IOException ioException) {
           }
       
           @Override
      -    public void put(PathMetadata meta) throws IOException {
      +    public void put(PathMetadata meta,
      +        final BulkOperationState operationState) throws IOException {
             throw ioException;
           }
      +
      +    @Override
      +    public void put(final PathMetadata meta) throws IOException {
      +      put(meta, null);
      +    }
      +
         }
       }
      diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java
      index 51ff299e7be08..fbabfa138bbf8 100644
      --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java
      +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java
      @@ -39,6 +39,7 @@
       import org.apache.hadoop.fs.permission.FsPermission;
       import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets;
       import org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase;
      +import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
       import org.apache.hadoop.util.Progressable;
       
       /**
      @@ -177,8 +178,9 @@ public boolean exists(Path f) throws IOException {
         }
       
         @Override
      -  void finishedWrite(String key, long length) {
      -
      +  void finishedWrite(String key,
      +      long length,
      +      final BulkOperationState operationState) {
         }
       
         @Override
      diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java
      index 4a146b117c383..72613538a42f6 100644
      --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java
      +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java
      @@ -608,12 +608,16 @@ public void testRestrictedCommitActions() throws Throwable {
             // all those commits must fail
             List commits = pendingCommits.getLeft().getCommits();
             assertEquals(range, commits.size());
      -      commits.parallelStream().forEach(
      -          (c) -> {
      -            CommitOperations.MaybeIOE maybeIOE = operations.commit(c, "origin");
      -            Path path = c.destinationPath();
      -            assertCommitAccessDenied(path, maybeIOE);
      -          });
      +      try(CommitOperations.CommitContext commitContext
      +              = operations.initiateCommitOperation(uploadDest)) {
      +        commits.parallelStream().forEach(
      +            (c) -> {
      +              CommitOperations.MaybeIOE maybeIOE =
      +                  commitContext.commit(c, "origin");
      +              Path path = c.destinationPath();
      +              assertCommitAccessDenied(path, maybeIOE);
      +            });
      +      }
       
             // fail of all list and abort of .pending files.
             LOG.info("abortAllSinglePendingCommits({})", readOnlyDir);
      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 2886a998b0373..60e9cf5165bca 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
      @@ -45,6 +45,7 @@
       import org.apache.hadoop.mapreduce.lib.output.PathOutputCommitter;
       import org.apache.hadoop.mapreduce.lib.output.PathOutputCommitterFactory;
       import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
      +import org.apache.hadoop.util.DurationInfo;
       
       import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
       import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
      @@ -332,13 +333,21 @@ private void commit(String filename,
           validateIntermediateAndFinalPaths(magicFile, destFile);
           SinglePendingCommit commit = SinglePendingCommit.load(getFileSystem(),
               validatePendingCommitData(filename, magicFile));
      -    CommitOperations actions = newCommitOperations();
           setThrottling(throttle, failures);
      -    actions.commitOrFail(commit);
      +    commitOrFail(destFile, commit, newCommitOperations());
           resetFailures();
           verifyCommitExists(commit);
         }
       
      +  private void commitOrFail(final Path destFile,
      +      final SinglePendingCommit commit, final CommitOperations actions)
      +      throws IOException {
      +    try (CommitOperations.CommitContext commitContext
      +             = actions.initiateCommitOperation(destFile)) {
      +      commitContext.commitOrFail(commit);
      +    }
      +  }
      +
         /**
          * Perform any validation of paths.
          * @param magicFilePath path to magic file
      @@ -439,7 +448,7 @@ public void testUploadEmptyFile() throws Throwable {
           resetFailures();
           assertPathDoesNotExist("pending commit", dest);
           fullThrottle();
      -    actions.commitOrFail(pendingCommit);
      +    commitOrFail(dest, pendingCommit, actions);
           resetFailures();
           FileStatus status = verifyPathExists(fs,
               "uploaded file commit", dest);
      @@ -462,7 +471,7 @@ public void testUploadSmallFile() throws Throwable {
           resetFailures();
           assertPathDoesNotExist("pending commit", dest);
           fullThrottle();
      -    actions.commitOrFail(pendingCommit);
      +    commitOrFail(dest, pendingCommit, actions);
           resetFailures();
           String s = readUTF8(fs, dest, -1);
           assertEquals(text, s);
      diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestS3AHugeMagicCommits.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestS3AHugeMagicCommits.java
      index 072295962ce36..9a2ad0ee9da10 100644
      --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestS3AHugeMagicCommits.java
      +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestS3AHugeMagicCommits.java
      @@ -143,9 +143,12 @@ public void test_030_postCreationAssertions() throws Throwable {
           assertNotNull("jobDir", jobDir);
           Pair>>
               results = operations.loadSinglePendingCommits(jobDir, false);
      -    for (SinglePendingCommit singlePendingCommit :
      -        results.getKey().getCommits()) {
      -      operations.commitOrFail(singlePendingCommit);
      +    try(CommitOperations.CommitContext commitContext
      +            = operations.initiateCommitOperation(jobDir)) {
      +      for (SinglePendingCommit singlePendingCommit :
      +          results.getKey().getCommits()) {
      +        commitContext.commitOrFail(singlePendingCommit);
      +      }
           }
           timer.end("time to commit %s", pendingDataFile);
           // upload is no longer pending
      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 7e3524d8aa702..470a32d010c6f 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
      @@ -342,7 +342,10 @@ private Path uniquePath() throws IOException {
         public void testCannotTouchUnderRODir() throws Throwable {
           forbidden("touching the empty child " + readOnlyChild,
               "",
      -        () -> touch(roleFS, readOnlyChild));
      +        () -> {
      +          touch(roleFS, readOnlyChild);
      +          return readOnlyChild;
      +        });
         }
         @Test
         public void testCannotReadUnderNoReadDir() throws Throwable {
      diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java
      index 73d954d043941..abc98e91a0810 100644
      --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java
      +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java
      @@ -19,7 +19,6 @@
       package org.apache.hadoop.fs.s3a.impl;
       
       import javax.annotation.Nullable;
      -import java.io.Closeable;
       import java.io.IOException;
       import java.net.URI;
       import java.net.URISyntaxException;
      @@ -47,9 +46,11 @@
       import org.apache.hadoop.fs.Path;
       import org.apache.hadoop.fs.s3a.Constants;
       import org.apache.hadoop.fs.s3a.Invoker;
      +import org.apache.hadoop.fs.s3a.Retries;
       import org.apache.hadoop.fs.s3a.S3AInputPolicy;
       import org.apache.hadoop.fs.s3a.S3AInstrumentation;
       import org.apache.hadoop.fs.s3a.S3AStorageStatistics;
      +import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
       import org.apache.hadoop.fs.s3a.s3guard.DirListingMetadata;
       import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
       import org.apache.hadoop.fs.s3a.s3guard.PathMetadata;
      @@ -274,18 +275,25 @@ public DirListingMetadata listChildren(final Path path) {
           public void move(
               @Nullable final Collection pathsToDelete,
               @Nullable final Collection pathsToCreate,
      -        @Nullable final Closeable moveState) {
      +        @Nullable final BulkOperationState moveState) {
       
           }
       
           @Override
      -    public void put(final PathMetadata meta) {
      +    public void put(final PathMetadata meta) throws IOException {
      +      put(meta, null);
      +    }
      +
      +    @Override
      +    public void put(final PathMetadata meta,
      +        final BulkOperationState operationState) {
             created.add(meta.getFileStatus().getPath());
           }
       
           @Override
      -    public void put(final Collection metas) {
      -      metas.stream().forEach(this::put);
      +    public void put(final Collection metas,
      +        final BulkOperationState operationState) {
      +      metas.stream().forEach(meta -> put(meta, null));
           }
       
           @Override
      diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java
      index a3ad91b8c33b7..6f9b3851c8cea 100644
      --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java
      +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java
      @@ -59,7 +59,6 @@
       import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_DDB_TABLE_NAME_KEY;
       import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_METASTORE_NULL;
       import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL;
      -import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestDynamoTablePrefix;
       import static org.apache.hadoop.fs.s3a.S3AUtils.clearBucketOption;
       import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.E_BAD_STATE;
       import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.SUCCESS;
      @@ -198,18 +197,18 @@ protected void mkdirs(Path path, boolean onS3, boolean onMetadataStore)
             fs.mkdirs(path);
           } else if (onMetadataStore) {
             S3AFileStatus status = new S3AFileStatus(true, path, OWNER);
      -      ms.put(new PathMetadata(status));
      +      ms.put(new PathMetadata(status), null);
           }
         }
       
         protected static void putFile(MetadataStore ms, S3AFileStatus f)
             throws IOException {
           assertNotNull(f);
      -    ms.put(new PathMetadata(f));
      +    ms.put(new PathMetadata(f), null);
           Path parent = f.getPath().getParent();
           while (parent != null) {
             S3AFileStatus dir = new S3AFileStatus(false, parent, f.getOwner());
      -      ms.put(new PathMetadata(dir));
      +      ms.put(new PathMetadata(dir), null);
             parent = parent.getParent();
           }
         }
      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 f42d7dcfa5f1a..24af0bf039830 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
      @@ -357,8 +357,8 @@ private void doTestBatchWrite(int numDelete, int numPut,
           final Path newDir = new Path(root, "newDir");
           LOG.info("doTestBatchWrite: oldDir={}, newDir={}", oldDir, newDir);
       
      -    ms.put(new PathMetadata(basicFileStatus(oldDir, 0, true)));
      -    ms.put(new PathMetadata(basicFileStatus(newDir, 0, true)));
      +    ms.put(new PathMetadata(basicFileStatus(oldDir, 0, true)), null);
      +    ms.put(new PathMetadata(basicFileStatus(newDir, 0, true)), null);
       
           final List oldMetas = numDelete < 0 ? null :
               new ArrayList<>(numDelete);
      @@ -511,7 +511,7 @@ public void testRootDirectory() throws IOException {
       
           ddbms.put(new PathMetadata(new S3AFileStatus(true,
               new Path(rootPath, "foo"),
      -        UserGroupInformation.getCurrentUser().getShortUserName())));
      +        UserGroupInformation.getCurrentUser().getShortUserName())), null);
           verifyRootDirectory(ddbms.get(rootPath), false);
         }
       
      @@ -562,9 +562,9 @@ public void testMovePopulatesAncestors() throws IOException {
           final String destRoot = testRoot + "/c/d/e/dest";
       
           final Path nestedPath1 = strToPath(srcRoot + "/file1.txt");
      -    ddbms.put(new PathMetadata(basicFileStatus(nestedPath1, 1024, false)));
      +    ddbms.put(new PathMetadata(basicFileStatus(nestedPath1, 1024, false)), null);
           final Path nestedPath2 = strToPath(srcRoot + "/dir1/dir2");
      -    ddbms.put(new PathMetadata(basicFileStatus(nestedPath2, 0, true)));
      +    ddbms.put(new PathMetadata(basicFileStatus(nestedPath2, 0, true)), null);
       
           // We don't put the destRoot path here, since put() would create ancestor
           // entries, and we want to ensure that move() does it, instead.
      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 408c72eb8f521..615eadcc3e290 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
      @@ -238,7 +238,7 @@ public void test_030_BatchedWrite() throws Exception {
                       toCleanup.add(pm);
                     }
       
      -              ddbms.put(pm);
      +              ddbms.put(pm, null);
       
                     pruneItems++;
       
      @@ -274,7 +274,7 @@ public void test_040_get() throws Throwable {
           Path path = new Path("s3a://example.org/get");
           S3AFileStatus status = new S3AFileStatus(true, path, "alice");
           PathMetadata metadata = new PathMetadata(status);
      -    ddbms.put(metadata);
      +    ddbms.put(metadata, null);
           try {
             execute("get",
                 OPERATIONS_PER_THREAD,
      @@ -318,7 +318,7 @@ public void test_060_list() throws Throwable {
           Path path = new Path("s3a://example.org/list");
           S3AFileStatus status = new S3AFileStatus(true, path, "alice");
           PathMetadata metadata = new PathMetadata(status);
      -    ddbms.put(metadata);
      +    ddbms.put(metadata, null);
           try {
             Path parent = path.getParent();
             execute("list",
      @@ -337,7 +337,7 @@ public void test_070_putDirMarker() throws Throwable {
           Path path = new Path("s3a://example.org/putDirMarker");
           S3AFileStatus status = new S3AFileStatus(true, path, "alice");
           PathMetadata metadata = new PathMetadata(status);
      -    ddbms.put(metadata);
      +    ddbms.put(metadata, null);
           DirListingMetadata children = ddbms.listChildren(path.getParent());
           try {
             execute("list",
      @@ -356,12 +356,14 @@ public void test_080_fullPathsToPut() throws Throwable {
           Path base = new Path("s3a://example.org/test_080_fullPathsToPut");
           Path child = new Path(base, "child");
           List pms = new ArrayList<>();
      -    ddbms.put(new PathMetadata(makeDirStatus(base)));
      -    ddbms.put(new PathMetadata(makeDirStatus(child)));
      +    BulkOperationState bulkUpdate
      +        = ddbms.initiateBulkWrite(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));
      +        () -> ddbms.put(pms, bulkUpdate));
           try {
             DDBPathMetadata dirData = ddbms.get(child, true);
             execute("list",
      diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreTestBase.java
      index 1acce2ddc8700..7e49874ea54cd 100644
      --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreTestBase.java
      +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreTestBase.java
      @@ -150,7 +150,7 @@ private void doTestDescendantsIterator(
             final FileStatus status = pathStr.contains("file")
                 ? basicFileStatus(strToPath(pathStr), 100, false)
                 : basicFileStatus(strToPath(pathStr), 0, true);
      -      ms.put(new PathMetadata(status));
      +      ms.put(new PathMetadata(status), null);
           }
       
           final PathMetadata rootMeta = new PathMetadata(makeDirStatus("/"));
      @@ -237,7 +237,7 @@ public void testPutNew() throws Exception {
            * containing directory.  We only track direct children of the directory.
            * Thus this will not affect entry for /da1.
            */
      -    ms.put(new PathMetadata(makeFileStatus("/da1/db1/fc1", 100)));
      +    ms.put(new PathMetadata(makeFileStatus("/da1/db1/fc1", 100)), null);
       
           assertEmptyDirs("/da2", "/da3");
           assertDirectorySize("/da1/db1", 1);
      @@ -249,7 +249,7 @@ public void testPutNew() throws Exception {
           }
       
           /* This already exists, and should silently replace it. */
      -    ms.put(new PathMetadata(makeDirStatus("/da1/db1")));
      +    ms.put(new PathMetadata(makeDirStatus("/da1/db1")), null);
       
           /* If we had putNew(), and used it above, this would be empty again. */
           assertDirectorySize("/da1", 1);
      @@ -257,8 +257,8 @@ public void testPutNew() throws Exception {
           assertEmptyDirs("/da2", "/da3");
       
           /* Ensure new files update correct parent dirs. */
      -    ms.put(new PathMetadata(makeFileStatus("/da1/db1/fc1", 100)));
      -    ms.put(new PathMetadata(makeFileStatus("/da1/db1/fc2", 200)));
      +    ms.put(new PathMetadata(makeFileStatus("/da1/db1/fc1", 100)), null);
      +    ms.put(new PathMetadata(makeFileStatus("/da1/db1/fc2", 200)), null);
           assertDirectorySize("/da1", 1);
           assertDirectorySize("/da1/db1", 2);
           assertEmptyDirs("/da2", "/da3");
      @@ -273,14 +273,15 @@ public void testPutNew() throws Exception {
         public void testPutOverwrite() throws Exception {
           final String filePath = "/a1/b1/c1/some_file";
           final String dirPath = "/a1/b1/c1/d1";
      -    ms.put(new PathMetadata(makeFileStatus(filePath, 100)));
      -    ms.put(new PathMetadata(makeDirStatus(dirPath)));
      +    ms.put(new PathMetadata(makeFileStatus(filePath, 100)), null);
      +    ms.put(new PathMetadata(makeDirStatus(dirPath)), null);
           PathMetadata meta = ms.get(strToPath(filePath));
           if (!allowMissing() || meta != null) {
             verifyFileStatus(meta.getFileStatus(), 100);
           }
       
      -    ms.put(new PathMetadata(basicFileStatus(strToPath(filePath), 9999, false)));
      +    ms.put(new PathMetadata(basicFileStatus(strToPath(filePath), 9999, false)),
      +        null);
           meta = ms.get(strToPath(filePath));
           if (!allowMissing() || meta != null) {
             verifyFileStatus(meta.getFileStatus(), 9999);
      @@ -291,7 +292,7 @@ public void testPutOverwrite() throws Exception {
         public void testRootDirPutNew() throws Exception {
           Path rootPath = strToPath("/");
       
      -    ms.put(new PathMetadata(makeFileStatus("/file1", 100)));
      +    ms.put(new PathMetadata(makeFileStatus("/file1", 100)), null);
           DirListingMetadata dir = ms.listChildren(rootPath);
           if (!allowMissing() || dir != null) {
             assertNotNull("Root dir cached", dir);
      @@ -331,7 +332,7 @@ private void deleteSubtreeHelper(String pathPrefix) throws Exception {
           setUpDeleteTest(p);
           createNewDirs(p + "/ADirectory1/db1/dc1", p + "/ADirectory1/db1/dc1/dd1");
           ms.put(new PathMetadata(
      -        makeFileStatus(p + "/ADirectory1/db1/dc1/dd1/deepFile", 100)));
      +        makeFileStatus(p + "/ADirectory1/db1/dc1/dd1/deepFile", 100)), null);
           if (!allowMissing()) {
             assertCached(p + "/ADirectory1/db1");
           }
      @@ -381,9 +382,9 @@ private void setUpDeleteTest(String prefix) throws IOException {
           createNewDirs(prefix + "/ADirectory1", prefix + "/ADirectory2",
               prefix + "/ADirectory1/db1");
           ms.put(new PathMetadata(makeFileStatus(prefix + "/ADirectory1/db1/file1",
      -        100)));
      +        100)), null);
           ms.put(new PathMetadata(makeFileStatus(prefix + "/ADirectory1/db1/file2",
      -        100)));
      +        100)), null);
       
           PathMetadata meta = ms.get(strToPath(prefix + "/ADirectory1/db1/file2"));
           if (!allowMissing() || meta != null) {
      @@ -396,8 +397,8 @@ private void setUpDeleteTest(String prefix) throws IOException {
         public void testGet() throws Exception {
           final String filePath = "/a1/b1/c1/some_file";
           final String dirPath = "/a1/b1/c1/d1";
      -    ms.put(new PathMetadata(makeFileStatus(filePath, 100)));
      -    ms.put(new PathMetadata(makeDirStatus(dirPath)));
      +    ms.put(new PathMetadata(makeFileStatus(filePath, 100)), null);
      +    ms.put(new PathMetadata(makeDirStatus(dirPath)), null);
           PathMetadata meta = ms.get(strToPath(filePath));
           if (!allowMissing() || meta != null) {
             assertNotNull("Get found file", meta);
      @@ -627,7 +628,7 @@ public void testMultiBucketPaths() throws Exception {
           assertNull("Path2 should not be present yet.", meta);
       
           // Put p1, assert p2 doesn't match
      -    ms.put(new PathMetadata(makeFileStatus(p1, 100)));
      +    ms.put(new PathMetadata(makeFileStatus(p1, 100)), null);
           meta = ms.get(new Path(p2));
           assertNull("Path 2 should not match path 1.", meta);
       
      @@ -647,7 +648,7 @@ public void testPruneFiles() throws Exception {
       
           long oldTime = getTime();
           ms.put(new PathMetadata(makeFileStatus("/pruneFiles/old", 1, oldTime,
      -        oldTime)));
      +        oldTime)), null);
           DirListingMetadata ls2 = ms.listChildren(strToPath("/pruneFiles"));
           if (!allowMissing()) {
             assertListingsEqual(ls2.getListing(), "/pruneFiles/old");
      @@ -659,7 +660,7 @@ public void testPruneFiles() throws Exception {
           long cutoff = System.currentTimeMillis();
           long newTime = getTime();
           ms.put(new PathMetadata(makeFileStatus("/pruneFiles/new", 1, newTime,
      -        newTime)));
      +        newTime)), null);
       
           DirListingMetadata ls;
           ls = ms.listChildren(strToPath("/pruneFiles"));
      @@ -690,7 +691,7 @@ public void testPruneDirs() throws Exception {
       
           long oldTime = getTime();
           ms.put(new PathMetadata(makeFileStatus("/pruneDirs/dir/file",
      -        1, oldTime, oldTime)));
      +        1, oldTime, oldTime)), null);
       
           // It's possible for the Local implementation to get from the old
           // modification time to here in under 1ms, causing it to not get pruned
      @@ -715,10 +716,10 @@ public void testPruneUnsetsAuthoritative() throws Exception {
           long time = System.currentTimeMillis();
           ms.put(new PathMetadata(
               new FileStatus(0, false, 0, 0, time - 1, strToPath(staleFile)),
      -        Tristate.FALSE, false));
      +        Tristate.FALSE, false), null);
           ms.put(new PathMetadata(
               new FileStatus(0, false, 0, 0, time + 1, strToPath(freshFile)),
      -        Tristate.FALSE, false));
      +        Tristate.FALSE, false), null);
       
           // set parent dir as authoritative
           if (!allowMissing()) {
      @@ -752,10 +753,10 @@ public void testPrunePreservesAuthoritative() throws Exception {
           long time = System.currentTimeMillis();
           ms.put(new PathMetadata(
               new FileStatus(0, false, 0, 0, time + 1, strToPath(staleFile)),
      -        Tristate.FALSE, false));
      +        Tristate.FALSE, false), null);
           ms.put(new PathMetadata(
               new FileStatus(0, false, 0, 0, time + 1, strToPath(freshFile)),
      -        Tristate.FALSE, false));
      +        Tristate.FALSE, false), null);
       
           if (!allowMissing()) {
             // set parent dir as authoritative
      @@ -813,7 +814,7 @@ public void testPutRetainsIsDeletedInParentListing() throws Exception {
           final FileStatus fileStatus = basicFileStatus(path, 0, false);
           PathMetadata pm = new PathMetadata(fileStatus);
           pm.setIsDeleted(true);
      -    ms.put(pm);
      +    ms.put(pm, null);
           if(!allowMissing()) {
             final PathMetadata pathMetadata =
                 ms.listChildren(path.getParent()).get(path);
      @@ -853,8 +854,8 @@ private void commonTestPutListStatus(final String parent) throws IOException {
         private void setupListStatus() throws IOException {
           createNewDirs("/a1", "/a2", "/a1/b1", "/a1/b2", "/a1/b1/c1",
               "/a1/b1/c1/d1");
      -    ms.put(new PathMetadata(makeFileStatus("/a1/b1/file1", 100)));
      -    ms.put(new PathMetadata(makeFileStatus("/a1/b1/file2", 100)));
      +    ms.put(new PathMetadata(makeFileStatus("/a1/b1/file1", 100)), null);
      +    ms.put(new PathMetadata(makeFileStatus("/a1/b1/file2", 100)), null);
         }
       
         private void assertListingsEqual(Collection listing,
      @@ -885,7 +886,7 @@ private void putListStatusFiles(String dirPath, boolean authoritative,
         private void createNewDirs(String... dirs)
             throws IOException {
           for (String pathStr : dirs) {
      -      ms.put(new PathMetadata(makeDirStatus(pathStr)));
      +      ms.put(new PathMetadata(makeDirStatus(pathStr)), null);
           }
         }
       
      diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractITestS3AMetadataStoreScale.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractITestS3AMetadataStoreScale.java
      index d015161275391..b71a8d2ee3ac9 100644
      --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractITestS3AMetadataStoreScale.java
      +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractITestS3AMetadataStoreScale.java
      @@ -20,6 +20,7 @@
       
       import org.apache.hadoop.fs.Path;
       import org.apache.hadoop.fs.s3a.S3AFileStatus;
      +import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
       import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
       import org.apache.hadoop.fs.s3a.s3guard.PathMetadata;
       
      @@ -180,8 +181,9 @@ private long populateMetadataStore(Collection paths,
           long count = 0;
           NanoTimer putTimer = new NanoTimer();
           describe("Inserting into MetadataStore");
      +    BulkOperationState operationState = ms.initiateBulkWrite(BUCKET_ROOT);
           for (PathMetadata p : paths) {
      -      ms.put(p);
      +      ms.put(p, operationState);
             count++;
           }
           putTimer.end();
      
      From 301a7d4c0853daa29bf225b114dd10c8a9a943f5 Mon Sep 17 00:00:00 2001
      From: Steve Loughran 
      Date: Thu, 16 May 2019 16:31:52 +0200
      Subject: [PATCH 18/22] HADOOP-15183: address github review and yetus comments
      
      Change-Id: Ib25d5bb7735132745dab150208f5fafad600bdb6
      ---
       .../hadoop-common/src/main/resources/core-default.xml       | 2 +-
       .../src/main/java/org/apache/hadoop/fs/s3a/Constants.java   | 6 +++---
       .../src/site/markdown/tools/hadoop-aws/s3guard.md           | 4 ++--
       .../org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java   | 2 +-
       4 files changed, 7 insertions(+), 7 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 e5e71c59ae7d9..ba976036f0e1b 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
      @@ -1213,7 +1213,7 @@
       
       
         fs.s3a.connection.maximum
      -  48
      +  72
         Controls the maximum number of simultaneous connections to S3.
           This must be bigger than the value of fs.s3a.threads.max so as to stop
           threads being blocked waiting for new HTTPS connections.
      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 6bd89b19835aa..92cefbbed410b 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
      @@ -143,11 +143,11 @@ private Constants() {
         // the maximum number of tasks cached if all threads are already uploading
         public static final String MAX_TOTAL_TASKS = "fs.s3a.max.total.tasks";
       
      -  public static final int DEFAULT_MAX_TOTAL_TASKS = 5;
      +  public static final int DEFAULT_MAX_TOTAL_TASKS = 32;
       
         // number of simultaneous connections to s3
         public static final String MAXIMUM_CONNECTIONS = "fs.s3a.connection.maximum";
      -  public static final int DEFAULT_MAXIMUM_CONNECTIONS = DEFAULT_MAX_TOTAL_TASKS * 2;
      +  public static final int DEFAULT_MAXIMUM_CONNECTIONS = 48;
       
         // connect to s3 over ssl?
         public static final String SECURE_CONNECTIONS =
      @@ -305,7 +305,7 @@ private Constants() {
          * upload, where {@link #FAST_UPLOAD_ACTIVE_BLOCKS} is used instead.
          * Value: {@value}
          */
      -  public static final int DEFAULT_EXECUTOR_CAPACITY = 10;
      +  public static final int DEFAULT_EXECUTOR_CAPACITY = 16;
       
         // Private | PublicRead | PublicReadWrite | AuthenticatedRead |
         // LogDeliveryWrite | BucketOwnerRead | BucketOwnerFullControl
      diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md
      index 2f0c5fa5500d6..352be731bb296 100644
      --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md
      +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md
      @@ -1219,8 +1219,8 @@ See [Fail on Error](#fail-on-error) for more detail.
       
       ```
       com.amazonaws.services.dynamodbv2.model.ResourceInUseException:
      -  Attempt to change a resource which is still in use: Table is being deleted: s
      -    3guard.test.testDynamoDBInitDestroy351245027 
      +  Attempt to change a resource which is still in use: Table is being deleted:
      +   s3guard.test.testDynamoDBInitDestroy351245027
           (Service: AmazonDynamoDBv2; Status Code: 400; Error Code: ResourceInUseException;)
       ```
       
      diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java
      index f708012570e2c..3a5786f458ae3 100644
      --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java
      +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java
      @@ -151,7 +151,7 @@ public void testMultiObjectDeleteNoPermissions() throws Throwable {
       
         /**
          * See what happens when you delete two entries which do not exist.
      -   * The call succeeds; if
      +   * It must not raise an exception.
          */
         @Test
         public void testMultiObjectDeleteMissingEntriesSucceeds() throws Throwable {
      
      From 1be6883332b4f6e4c5e8813ca3f63f117431092d Mon Sep 17 00:00:00 2001
      From: Steve Loughran 
      Date: Sat, 18 May 2019 00:03:42 +0100
      Subject: [PATCH 19/22] HADOOP-15183: full end-to-end test to verify that >1
       commit will only update parent directories once.
      
      +ancestor checking does this
      +instrumentation update in DDB => optional
      +ITestPartialRenamesDeletes prunes on teardown properly. This stops the store being full of tombstone markers, and
       during development of the code, many invalid entries
      +DDB prune will only sleep at the end of each batch, so if there is only work done at cleanup, no need to sleep at all.
      
      Change-Id: Ibdf63af1f5563489d15166f06b8c8e7c9ebfaec0
      ---
       .../fs/s3a/commit/CommitOperations.java       |  13 ++
       .../fs/s3a/s3guard/DynamoDBMetadataStore.java | 132 ++++++++++++------
       .../PathMetadataDynamoDBTranslation.java      |   6 +-
       .../fs/s3a/commit/ITestCommitOperations.java  |  80 ++++++++++-
       .../s3a/impl/ITestPartialRenamesDeletes.java  |  48 +++----
       .../s3guard/ITestDynamoDBMetadataStore.java   |  37 +++--
       .../AbstractITestS3AMetadataStoreScale.java   |   9 +-
       7 files changed, 242 insertions(+), 83 deletions(-)
      
      diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java
      index 7bee5c9f48927..ce80945b25a29 100644
      --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java
      +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java
      @@ -559,6 +559,9 @@ public CommitContext initiateCommitOperation(Path path) throws IOException {
          */
         public final class CommitContext implements Closeable {
       
      +    /**
      +     * State of any metastore.
      +     */
           private final BulkOperationState operationState;
       
           private CommitContext(@Nullable final BulkOperationState operationState) {
      @@ -627,6 +630,16 @@ public void abortMultipartCommit(
           public void close() throws IOException {
             IOUtils.cleanupWithLogger(LOG, operationState);
           }
      +
      +    @Override
      +    public String toString() {
      +      final StringBuilder sb = new StringBuilder(
      +          "CommitContext{");
      +      sb.append("operationState=").append(operationState);
      +      sb.append('}');
      +      return sb.toString();
      +    }
      +
         }
       
         /**
      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 0c84aaae900d5..aca16cd38694d 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
      @@ -34,6 +34,7 @@
       import java.util.List;
       import java.util.Map;
       import java.util.Objects;
      +import java.util.Optional;
       import java.util.Set;
       import java.util.TreeMap;
       import java.util.concurrent.CompletableFuture;
      @@ -187,7 +188,8 @@
        * sub-tree.
        *
        * Some mutating operations, notably {@link #deleteSubtree(Path)} and
      - * {@link MetadataStore#move(Collection, Collection, AutoCloseable)}, are less efficient with this schema.
      + * {@link MetadataStore#move(Collection, Collection, BulkOperationState)}
      + * are less efficient with this schema.
        * They require mutating multiple items in the DynamoDB table.
        *
        * By default, DynamoDB access is performed within the same AWS region as
      @@ -279,7 +281,7 @@ public class DynamoDBMetadataStore implements MetadataStore,
          */
         private RetryPolicy batchWriteRetryPolicy;
       
      -  private S3AInstrumentation.S3GuardInstrumentation instrumentation;
      +  private Optional instrumentation;
       
         /** Owner FS: only valid if configured with an owner FS. */
         private S3AFileSystem owner;
      @@ -389,7 +391,8 @@ public void initialize(FileSystem fs) throws IOException {
       
           initTable();
       
      -    instrumentation.initialized();
      +    instrumentation.ifPresent(
      +        S3AInstrumentation.S3GuardInstrumentation::initialized);
         }
       
         /**
      @@ -403,7 +406,8 @@ void bindToOwnerFilesystem(final S3AFileSystem fs) {
           owner = fs;
           conf = owner.getConf();
           StoreContext context = owner.createStoreContext();
      -    instrumentation = context.getInstrumentation().getS3GuardInstrumentation();
      +    instrumentation = Optional.of(
      +        context.getInstrumentation().getS3GuardInstrumentation());
           username = context.getUsername();
           executor = context.createThrottledExecutor();
         }
      @@ -589,9 +593,7 @@ private Item getConsistentItem(final Path path) throws IOException {
               path.toString(),
               true,
               () -> table.getItem(spec));
      -    if (instrumentation != null) {
      -      instrumentation.recordsRead(1);
      -    }
      +    recordsRead(1);
           return item;
         }
       
      @@ -736,8 +738,8 @@ DirListingMetadata getDirListingMetadataFromDirMetaAndList(Path path,
       
         /**
          * Build the list of all parent entries.
      -   * Thread safety: none. if access the move state is to be synchronized,
      -   * callers must do this.
      +   * Thread safety: none. Callers are expected to
      +   * synchronize on ancestorState as required.
          * @param pathsToCreate paths to create
          * @param ancestorState ongoing ancestor state.
          * @return the full ancestry paths
      @@ -747,6 +749,7 @@ private Collection completeAncestry(
             final AncestorState ancestorState) {
           // Key on path to allow fast lookup
           Map ancestry = ancestorState.getAncestry();
      +    List ancestorsToAdd = new ArrayList<>(0);
       
           for (DDBPathMetadata meta : pathsToCreate) {
             Preconditions.checkArgument(meta != null);
      @@ -754,18 +757,30 @@ private Collection completeAncestry(
             if (path.isRoot()) {
               break;
             }
      -      ancestry.put(path, new DDBPathMetadata(meta));
      +      // add the new entry
      +      DDBPathMetadata entry = new DDBPathMetadata(meta);
      +      DDBPathMetadata oldEntry = ancestry.put(path, entry);
      +      if (oldEntry != null) {
      +        // check for and warn if the existing bulk operation overwrote it.
      +        // this should never occur outside tests.
      +        LOG.warn("Overwriting a S3Guard entry created in the operation: {}",
      +            oldEntry);
      +        continue;
      +      }
      +      ancestorsToAdd.add(entry);
             Path parent = path.getParent();
             while (!parent.isRoot() && !ancestry.containsKey(parent)) {
               LOG.debug("auto-create ancestor path {} for child path {}",
                   parent, path);
               final FileStatus status = makeDirStatus(parent, username);
      -        ancestry.put(parent, new DDBPathMetadata(status, Tristate.FALSE,
      -            false));
      +        DDBPathMetadata md = new DDBPathMetadata(status, Tristate.FALSE,
      +            false);
      +        ancestry.put(parent, md);
      +        ancestorsToAdd.add(md);
               parent = parent.getParent();
             }
           }
      -    return ancestry.values();
      +    return ancestorsToAdd;
         }
       
         /**
      @@ -889,9 +904,7 @@ private int processBatchWriteRequest(PrimaryKey[] keysToDelete,
                 () -> dynamoDB.batchWriteItem(writeItems));
             // Check for unprocessed keys in case of exceeding provisioned throughput
             Map> unprocessed = res.getUnprocessedItems();
      -      if (instrumentation != null) {
      -        instrumentation.recordsWritten(batchSize - unprocessed.size());
      -      }
      +      recordsWritten(batchSize - unprocessed.size());
             int retryCount = 0;
             while (!unprocessed.isEmpty()) {
               batchWriteCapacityExceededEvents.incrementAndGet();
      @@ -906,9 +919,7 @@ private int processBatchWriteRequest(PrimaryKey[] keysToDelete,
                   true,
                   () -> dynamoDB.batchWriteItemUnprocessed(upx));
               unprocessed = res.getUnprocessedItems();
      -        if (instrumentation != null) {
      -          instrumentation.recordsWritten(batchSize - unprocessed.size());
      -        }
      +        recordsWritten(batchSize - unprocessed.size());
             }
           }
           return batches;
      @@ -1097,9 +1108,8 @@ public void put(DirListingMetadata meta) throws IOException {
       
         @Override
         public synchronized void close() {
      -    if (instrumentation != null) {
      -      instrumentation.storeClosed();
      -    }
      +    instrumentation.ifPresent(
      +        S3AInstrumentation.S3GuardInstrumentation::storeClosed);
           try {
             if (dynamoDB != null) {
               LOG.debug("Shutting down {}", this);
      @@ -1196,9 +1206,6 @@ public void prune(long modTime, String keyPrefix) throws IOException {
       
               itemCount++;
               if (deletionBatch.size() == S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT) {
      -          if (delay > 0) {
      -            Thread.sleep(delay);
      -          }
                 processBatchWriteRequest(pathToKey(deletionBatch), null);
       
                 // set authoritative false for each pruned dir listing
      @@ -1208,11 +1215,13 @@ public void prune(long modTime, String keyPrefix) throws IOException {
                 parentPathSet.clear();
       
                 deletionBatch.clear();
      +          if (delay > 0) {
      +            Thread.sleep(delay);
      +          }
               }
             }
             // final batch of deletes
             if (!deletionBatch.isEmpty()) {
      -        Thread.sleep(delay);
               processBatchWriteRequest(pathToKey(deletionBatch), null);
       
               // set authoritative false for each pruned dir listing
      @@ -1783,9 +1792,8 @@ void retryEvent(
             boolean idempotent) {
           if (S3AUtils.isThrottleException(ex)) {
             // throttled
      -      if (instrumentation != null) {
      -        instrumentation.throttled();
      -      }
      +      instrumentation.ifPresent(
      +          S3AInstrumentation.S3GuardInstrumentation::throttled);
             int eventCount = throttleEventCount.addAndGet(1);
             if (attempts == 1 && eventCount < THROTTLE_EVENT_LOG_LIMIT) {
               LOG.warn("DynamoDB IO limits reached in {};"
      @@ -1801,11 +1809,8 @@ void retryEvent(
             LOG.info("Retrying {}: {}", text, ex.toString());
             LOG.debug("Retrying {}", text, ex);
           }
      -
      -    if (instrumentation != null) {
      -      // note a retry
      -      instrumentation.retrying();
      -    }
      +    instrumentation.ifPresent(
      +        S3AInstrumentation.S3GuardInstrumentation::retrying);
           if (owner != null) {
             owner.metastoreOperationRetried(ex, attempts, idempotent);
           }
      @@ -1839,6 +1844,22 @@ public Invoker getInvoker() {
           return invoker;
         }
       
      +  /**
      +   * Record the number of records written.
      +   * @param count count of records.
      +   */
      +  private void recordsWritten(final int count) {
      +    instrumentation.ifPresent(i -> i.recordsWritten(count));
      +  }
      +
      +  /**
      +   * Record the number of records read.
      +   * @param count count of records.
      +   */
      +  private void recordsRead(final int count) {
      +    instrumentation.ifPresent(i -> i.recordsRead(count));
      +  }
      +
         /**
          * Initiate the rename operation by creating the tracker and the ongoing
          * move state.
      @@ -1851,9 +1872,16 @@ public Invoker getInvoker() {
         @Override
         public RenameTracker initiateRenameOperation(final StoreContext storeContext,
             final Path source,
      -      final FileStatus sourceStatus, final Path dest) {
      +      final FileStatus sourceStatus,
      +      final Path dest) {
           return new ProgressiveRenameTracker(storeContext, this, source, dest,
      -        new AncestorState());
      +        new AncestorState(dest));
      +  }
      +
      +  @Override
      +  public AncestorState initiateBulkWrite(final Path dest)
      +      throws IOException {
      +    return new AncestorState(dest);
         }
       
         /**
      @@ -1908,24 +1936,48 @@ static AncestorState extractOrCreate(@Nullable BulkOperationState state) {
           if (state != null) {
             return (AncestorState) state;
           } else {
      -      return new AncestorState();
      +      return new AncestorState(null);
           }
         }
       
         /**
          * This tracks all the ancestors created,
          * across multiple move/write operations.
      -   * This is to avoid duplicate creation of ancestors even during
      -   * rename operations managed by a rename tracker.
      +   * This is to avoid duplicate creation of ancestors during bulk commits
      +   * and rename operations managed by a rename tracker.
          */
         @VisibleForTesting
         static final class AncestorState extends BulkOperationState {
       
           private final Map ancestry = new HashMap<>();
       
      -    public Map getAncestry() {
      +    private final Path dest;
      +
      +    AncestorState(final Path dest) {
      +      super();
      +      this.dest = dest;
      +    }
      +
      +    private Map getAncestry() {
             return ancestry;
           }
       
      +    int size() {
      +      return ancestry.size();
      +    }
      +
      +    public Path getDest() {
      +      return dest;
      +    }
      +
      +    @Override
      +    public String toString() {
      +      final StringBuilder sb = new StringBuilder(
      +          "AncestorState{");
      +      sb.append("dest=").append(dest);
      +      sb.append("size=").append(size());
      +      sb.append('}');
      +      return sb.toString();
      +    }
         }
       }
      diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadataDynamoDBTranslation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadataDynamoDBTranslation.java
      index c6f70bf277f44..5df54f562f0f8 100644
      --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadataDynamoDBTranslation.java
      +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadataDynamoDBTranslation.java
      @@ -51,7 +51,8 @@
        */
       @InterfaceAudience.Private
       @InterfaceStability.Evolving
      -final class PathMetadataDynamoDBTranslation {
      +@VisibleForTesting
      +public final class PathMetadataDynamoDBTranslation {
       
         /** The HASH key name of each item. */
         @VisibleForTesting
      @@ -279,7 +280,8 @@ static KeyAttribute pathToParentKeyAttribute(Path path) {
          * @param path path to convert
          * @return string for parent key
          */
      -  static String pathToParentKey(Path path) {
      +  @VisibleForTesting
      +  public static String pathToParentKey(Path path) {
           Preconditions.checkNotNull(path);
           Preconditions.checkArgument(path.isUriPathAbsolute(), "Path not absolute");
           URI uri = path.toUri();
      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 60e9cf5165bca..56adab75808f3 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
      @@ -21,9 +21,11 @@
       import java.io.File;
       import java.io.FileNotFoundException;
       import java.io.IOException;
      +import java.util.ArrayList;
       import java.util.List;
       
       import com.amazonaws.services.s3.model.PartETag;
      +import com.google.common.collect.Lists;
       import org.junit.Assume;
       import org.junit.Test;
       import org.slf4j.Logger;
      @@ -36,6 +38,7 @@
       import org.apache.hadoop.fs.FileSystem;
       import org.apache.hadoop.fs.Path;
       import org.apache.hadoop.fs.s3a.S3AFileSystem;
      +import org.apache.hadoop.fs.s3a.Statistic;
       import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit;
       import org.apache.hadoop.fs.s3a.commit.magic.MagicCommitTracker;
       import org.apache.hadoop.fs.s3a.commit.magic.MagicS3GuardCommitter;
      @@ -45,7 +48,6 @@
       import org.apache.hadoop.mapreduce.lib.output.PathOutputCommitter;
       import org.apache.hadoop.mapreduce.lib.output.PathOutputCommitterFactory;
       import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
      -import org.apache.hadoop.util.DurationInfo;
       
       import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
       import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
      @@ -553,4 +555,80 @@ public void testWriteNormalStream() throws Throwable {
           assertTrue("Empty marker file: " + status, status.getLen() > 0);
         }
       
      +  /**
      +   * Creates a bulk commit and commits multiple files.
      +   * If the DDB metastore is in use, use the instrumentation to
      +   * verify that the write count is as expected.
      +   * This is done without actually looking into the store -just monitoring
      +   * changes in the filesystem's instrumentation counters.
      +   */
      +  @Test
      +  public void testBulkCommitFiles() throws Throwable {
      +    describe("verify bulk commit including metastore update count");
      +    File localFile = File.createTempFile("commit", ".txt");
      +    CommitOperations actions = newCommitOperations();
      +    Path destDir = methodPath("testBulkCommitFiles");
      +    S3AFileSystem fs = getFileSystem();
      +    fs.delete(destDir, false);
      +    fullThrottle();
      +
      +    Path destFile1 = new Path(destDir, "file1");
      +    // this subdir will only be created in the commit of file 2
      +    Path subdir = new Path(destDir, "subdir");
      +    // file 2
      +    Path destFile2 = new Path(subdir, "file2");
      +    Path destFile3 = new Path(subdir, "file3");
      +    List destinations = Lists.newArrayList(destFile1, destFile2,
      +        destFile3);
      +    List commits = new ArrayList<>(3);
      +
      +    for (Path destination : destinations) {
      +      SinglePendingCommit commit1 =
      +          actions.uploadFileToPendingCommit(localFile,
      +              destination, null,
      +              DEFAULT_MULTIPART_SIZE);
      +      commits.add(commit1);
      +    }
      +    resetFailures();
      +    assertPathDoesNotExist("destination dir", destDir);
      +    assertPathDoesNotExist("subdirectory", subdir);
      +    // how many records have been written
      +    try (CommitOperations.CommitContext commitContext
      +             = actions.initiateCommitOperation(destDir)) {
      +      MetricDiff writes = new MetricDiff(fs,
      +          Statistic.S3GUARD_METADATASTORE_RECORD_WRITES);
      +      commitContext.commitOrFail(commits.get(0));
      +      long writesOnFirstCommit = writes.diff();
      +      writes.reset();
      +      assertPathExists("destFile1", destFile1);
      +      assertPathExists("destination dir", destDir);
      +
      +      commitContext.commitOrFail(commits.get(1));
      +      assertPathExists("subdirectory", subdir);
      +      assertPathExists("destFile2", destFile2);
      +      if (writesOnFirstCommit != 0) {
      +        // S3Guard is in use against DDB, so the metrics can be checked
      +        // to see how many records were updated.
      +        // there should only be two new entries: one for the file and
      +        // one for the parent.
      +        writes.assertDiffEquals(
      +            "Number of records written after second commit; "
      +                + "first commit had " + writesOnFirstCommit,
      +            2);
      +      }
      +      writes.reset();
      +      commitContext.commitOrFail(commits.get(2));
      +      assertPathExists("destFile3", destFile3);
      +      if (writesOnFirstCommit != 0) {
      +        // this file is in the same dir as destFile2, so only its entry
      +        // is added
      +        writes.assertDiffEquals(
      +            "Number of records written after third commit; "
      +                + "first commit had " + writesOnFirstCommit,
      +            1);
      +      }
      +    }
      +    resetFailures();
      +  }
      +
       }
      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 470a32d010c6f..01294b374033e 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
      @@ -50,24 +50,14 @@
       import org.apache.hadoop.fs.s3a.S3AFileSystem;
       import org.apache.hadoop.fs.s3a.S3AUtils;
       import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
      +import org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation;
       import org.apache.hadoop.util.BlockingThreadPoolExecutorService;
       import org.apache.hadoop.util.DurationInfo;
       
      -import static org.apache.hadoop.fs.contract.ContractTestUtils.assertRenameOutcome;
      -import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
      -import static org.apache.hadoop.fs.contract.ContractTestUtils.readUTF8;
      -import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
      -import static org.apache.hadoop.fs.s3a.Constants.ASSUMED_ROLE_ARN;
      -import static org.apache.hadoop.fs.s3a.Constants.ENABLE_MULTI_DELETE;
      -import static org.apache.hadoop.fs.s3a.Constants.MAXIMUM_CONNECTIONS;
      -import static org.apache.hadoop.fs.s3a.Constants.MAX_THREADS;
      -import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY;
      +import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
      +import static org.apache.hadoop.fs.s3a.Constants.*;
       import static org.apache.hadoop.fs.s3a.S3ATestUtils.MetricDiff;
      -import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume;
      -import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName;
      -import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestPropertyBool;
      -import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBucketOverrides;
      -import static org.apache.hadoop.fs.s3a.S3ATestUtils.reset;
      +import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
       import static org.apache.hadoop.fs.s3a.S3AUtils.applyLocatedFiles;
       import static org.apache.hadoop.fs.s3a.Statistic.FILES_DELETE_REJECTED;
       import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_DELETE_REQUESTS;
      @@ -148,7 +138,7 @@ public class ITestPartialRenamesDeletes extends AbstractS3ATestBase {
         public static final int DIR_COUNT_SCALED = 4;
         public static final int DEPTH = 2;
         public static final int DEPTH_SCALED = 2;
      -  
      +
         /**
          * A role FS; if non-null it is closed in teardown.
          */
      @@ -252,7 +242,16 @@ public void setup() throws Exception {
         @Override
         public void teardown() throws Exception {
           S3AUtils.closeAll(LOG, roleFS);
      -    super.teardown();
      +    try(DurationInfo ignored =
      +            new DurationInfo(LOG, "Delete and purge store")) {
      +      S3AFileSystem fs = getFileSystem();
      +      Path testCasePath = fs.qualify(getContract().getTestPath());
      +      rm(fs, testCasePath, true, false);
      +      fs.getMetadataStore().prune(0,
      +          PathMetadataDynamoDBTranslation.pathToParentKey((testCasePath)));
      +    } finally {
      +      super.teardown();
      +    }
         }
       
         /**
      @@ -437,8 +436,8 @@ public void testRenameSingleFileFailsInDelete() throws Throwable {
          * data is read only to the client calling rename().
          * This will cause the inner delete() operations to fail, whose outcomes
          * are explored.
      -   * Multiple files are created (in parallel) for some renames, so exploring
      -   * the outcome on bulk delete calls, including verifying that a
      +   * Multiple files are created (in parallel) for some renames, so the test
      +   * explores the outcome on bulk delete calls, including verifying that a
          * MultiObjectDeleteException is translated to an AccessDeniedException.
          * 
        *
      1. The exception raised is AccessDeniedException, @@ -481,7 +480,7 @@ public void testRenameDirFailsInDelete() throws Throwable { .containsAll(createdFiles) .containsExactlyInAnyOrderElementsOf(createdFiles); } - LOG.info("Result of renaming read-only files is AccessDeniedException", + LOG.info("Result of renaming read-only files is (Correctly) AccessDeniedException", deniedException); assertFileCount("files in the source directory", roleFS, readOnlyDir, expectedFileCount); @@ -515,8 +514,8 @@ public void testRenameFileFailsNoWrite() throws Throwable { } @Test - public void testCopyDirFailsInNoWrite() throws Throwable { - describe("Try to copy to a write-only destination"); + public void testCopyDirFailsToReadOnlyDir() throws Throwable { + describe("Try to copy to a read-only destination"); roleFS.mkdirs(writableDir); S3AFileSystem fs = getFileSystem(); List files = createFiles(fs, writableDir, dirDepth, fileCount, @@ -699,7 +698,8 @@ private void prune(Path path) throws IOException { MetadataStore store = fs.getMetadataStore(); try(DurationInfo ignored = new DurationInfo(LOG, true, "prune %s", path)) { - store.prune(System.currentTimeMillis(), fs.pathToKey(path)); + store.prune(System.currentTimeMillis(), + PathMetadataDynamoDBTranslation.pathToParentKey(fs.qualify(path))); } } } @@ -716,9 +716,7 @@ private Set listFilesUnderPath(Path path, boolean recursive) throws IOExce try (DurationInfo ignore = new DurationInfo(LOG, "ls -R %s", path)) { applyLocatedFiles(getFileSystem().listFiles(path, recursive), - (status) -> { - files.add(status.getPath()); - }); + (status) -> files.add(status.getPath())); } return files; } 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 24af0bf039830..e19cf25a39f12 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 @@ -36,16 +36,17 @@ import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughputDescription; import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException; import com.amazonaws.services.dynamodbv2.model.TableDescription; - import com.amazonaws.services.dynamodbv2.model.Tag; import com.google.common.collect.Lists; +import org.assertj.core.api.Assertions; + import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.fs.contract.s3a.S3AContract; import org.apache.hadoop.fs.s3a.Constants; import org.apache.hadoop.fs.s3a.Tristate; - import org.apache.hadoop.io.IOUtils; + import org.junit.AfterClass; import org.junit.Assert; import org.junit.Assume; @@ -62,6 +63,7 @@ import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.security.UserGroupInformation; +import static com.google.common.base.Preconditions.checkNotNull; import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.*; @@ -150,22 +152,24 @@ public static void beforeClassSetup() throws IOException { // be configured to use this test. testDynamoDBTableName = conf.get(S3GUARD_DDB_TEST_TABLE_NAME_KEY); String dynamoDbTableName = conf.getTrimmed(S3GUARD_DDB_TABLE_NAME_KEY); - Assume.assumeTrue("No DynamoDB table name configured", !StringUtils + Assume.assumeTrue("No DynamoDB table name configured in " + S3GUARD_DDB_TABLE_NAME_KEY, + !StringUtils .isEmpty(dynamoDbTableName)); // We should assert that the table name is configured, so the test should // fail if it's not configured. - assertTrue("Test DynamoDB table name '" + assertNotNull("Test DynamoDB table name '" + S3GUARD_DDB_TEST_TABLE_NAME_KEY + "' should be set to run " - + "integration tests.", testDynamoDBTableName != null); + + "integration tests.", testDynamoDBTableName); // We should assert that the test table is not the same as the production // table, as the test table could be modified and destroyed multiple // times during the test. - assertTrue("Test DynamoDB table name: '" + assertNotEquals("Test DynamoDB table name: '" + S3GUARD_DDB_TEST_TABLE_NAME_KEY + "' and production table name: '" + S3GUARD_DDB_TABLE_NAME_KEY + "' can not be the same.", - !conf.get(S3GUARD_DDB_TABLE_NAME_KEY).equals(testDynamoDBTableName)); + testDynamoDBTableName, + conf.get(S3GUARD_DDB_TABLE_NAME_KEY)); // We can use that table in the test if these assertions are valid conf.set(S3GUARD_DDB_TABLE_NAME_KEY, testDynamoDBTableName); @@ -388,11 +392,22 @@ private void doTestBatchWrite(int numDelete, int numPut, } // move the old paths to new paths and verify - ms.move(pathsToDelete, newMetas, null); - assertEquals(0, ms.listChildren(oldDir).withoutTombstones().numEntries()); + AncestorState state = checkNotNull(ms.initiateBulkWrite(newDir), + "No state from initiateBulkWrite()"); + assertEquals(newDir, state.getDest()); + + ms.move(pathsToDelete, newMetas, state); + assertEquals("Number of children in source directory", + 0, ms.listChildren(oldDir).withoutTombstones().numEntries()); if (newMetas != null) { - assertTrue(CollectionUtils - .isEqualCollection(newMetas, ms.listChildren(newDir).getListing())); + Assertions.assertThat(ms.listChildren(newDir).getListing()) + .describedAs("Directory listing") + .containsAll(newMetas); + if (!newMetas.isEmpty()) { + Assertions.assertThat(state.size()) + .describedAs("Size of ancestor state") + .isGreaterThan(newMetas.size()); + } } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractITestS3AMetadataStoreScale.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractITestS3AMetadataStoreScale.java index b71a8d2ee3ac9..57c543ffc9905 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractITestS3AMetadataStoreScale.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractITestS3AMetadataStoreScale.java @@ -181,10 +181,11 @@ private long populateMetadataStore(Collection paths, long count = 0; NanoTimer putTimer = new NanoTimer(); describe("Inserting into MetadataStore"); - BulkOperationState operationState = ms.initiateBulkWrite(BUCKET_ROOT); - for (PathMetadata p : paths) { - ms.put(p, operationState); - count++; + try(BulkOperationState operationState = ms.initiateBulkWrite(BUCKET_ROOT)) { + for (PathMetadata p : paths) { + ms.put(p, operationState); + count++; + } } putTimer.end(); printTiming(LOG, "put", putTimer, count); From 37d597b5bfd1d18089e3fd07cdb7c189c8848ea9 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Sat, 18 May 2019 19:39:36 +0100 Subject: [PATCH 20/22] HADOOP-15183: teardown improvements in ITestPartialRenamesDeletes Change-Id: Iee6c82070a3c050051ff80dcc03f877d0115c7e0 --- .../fs/s3a/impl/ITestPartialRenamesDeletes.java | 11 +---------- 1 file changed, 1 insertion(+), 10 deletions(-) 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 01294b374033e..a6a1d64a512d0 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 @@ -242,16 +242,7 @@ public void setup() throws Exception { @Override public void teardown() throws Exception { S3AUtils.closeAll(LOG, roleFS); - try(DurationInfo ignored = - new DurationInfo(LOG, "Delete and purge store")) { - S3AFileSystem fs = getFileSystem(); - Path testCasePath = fs.qualify(getContract().getTestPath()); - rm(fs, testCasePath, true, false); - fs.getMetadataStore().prune(0, - PathMetadataDynamoDBTranslation.pathToParentKey((testCasePath))); - } finally { - super.teardown(); - } + super.teardown(); } /** From 6e5a5a15eb8177b6d62e45a5f8d61bc161ff8eb0 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Sat, 18 May 2019 21:04:02 +0100 Subject: [PATCH 21/22] HADOOP-15183 checkstyle Change-Id: I4cc830515ce9cce300727d5d73c5c74160951be1 --- .../hadoop/fs/impl/FutureIOSupport.java | 2 +- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 12 ++--- .../hadoop/fs/s3a/WriteOperationHelper.java | 5 +- .../fs/s3a/impl/FunctionsRaisingIOE.java | 8 ++-- .../fs/s3a/impl/MultiObjectDeleteSupport.java | 2 +- .../hadoop/fs/s3a/impl/PathToBucketKeys.java | 22 --------- .../hadoop/fs/s3a/impl/StoreContext.java | 19 +++++--- .../s3guard/DelayedUpdateRenameTracker.java | 4 +- .../fs/s3a/s3guard/DynamoDBMetadataStore.java | 3 +- .../fs/s3a/s3guard/PathOrderComparators.java | 23 +++++----- .../s3a/s3guard/ProgressiveRenameTracker.java | 3 +- .../apache/hadoop/fs/s3a/s3guard/S3Guard.java | 2 +- .../apache/hadoop/fs/s3a/S3ATestUtils.java | 4 -- .../hadoop/fs/s3a/auth/RoleTestUtils.java | 6 --- .../s3a/impl/ITestPartialRenamesDeletes.java | 46 ++++++++++++------- .../s3a/impl/TestPartialDeleteFailures.java | 11 +++-- .../s3guard/AbstractS3GuardToolTestBase.java | 14 +++--- .../s3guard/ITestDynamoDBMetadataStore.java | 19 ++++---- .../hadoop/fs/s3a/test/ExtraAssertions.java | 6 ++- 19 files changed, 103 insertions(+), 108 deletions(-) delete mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/PathToBucketKeys.java diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureIOSupport.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureIOSupport.java index 72f5f91ac84b5..9fe402366c5df 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureIOSupport.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureIOSupport.java @@ -114,7 +114,7 @@ public static T raiseInnerCause(final ExecutionException e) /** * Extract the cause of a completion failure and rethrow it if an IOE - * or RTE + * or RTE. * @param e exception. * @param type of return value. * @return nothing, ever. 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 42e644bff5480..7f3c87479cb49 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 @@ -1272,12 +1272,12 @@ private long innerRename(Path source, Path dest) // the list. final FunctionsRaisingIOE.FunctionRaisingIOE completeActiveCopies = (String reason) -> { - LOG.debug("Waiting for {} active copies to complete: {}", - activeCopies.size(), reason); - waitForCompletion(activeCopies); - activeCopies.clear(); - return null; - }; + LOG.debug("Waiting for {} active copies to complete: {}", + activeCopies.size(), reason); + waitForCompletion(activeCopies); + activeCopies.clear(); + return null; + }; try { if (srcStatus.isFile()) { 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 080b5713f671d..7a45a883f7251 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 @@ -42,15 +42,14 @@ import com.amazonaws.services.s3.model.UploadPartResult; import com.amazonaws.services.s3.transfer.model.UploadResult; import com.google.common.base.Preconditions; -import org.apache.hadoop.conf.Configuration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; -import org.apache.hadoop.fs.s3a.s3guard.MetadataStore; import org.apache.hadoop.fs.s3a.s3guard.S3Guard; import org.apache.hadoop.fs.s3a.select.SelectBinding; import org.apache.hadoop.util.DurationInfo; @@ -502,8 +501,6 @@ public void revertCommit(String destKey) throws IOException { * @param uploadId multipart operation Id * @param partETags list of partial uploads * @param length length of the upload - * @param errorCount a counter incremented by 1 on every error; for - * use in statistics * @param operationState operational state for a bulk update * @return the result of the operation. * @throws IOException if problems arose which could not be retried, or diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/FunctionsRaisingIOE.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/FunctionsRaisingIOE.java index 92741aebb8b5b..e7e533b082fc3 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/FunctionsRaisingIOE.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/FunctionsRaisingIOE.java @@ -21,12 +21,10 @@ import java.io.IOException; /** - * Function of arity 1 which may raise an IOException. - * @param - * @param

        + * Evolving support for functional programming/lambda-expressions. + * Currently in the s3a module for experimentation and evolution; at + * some point it could be pulled into hadoop-common. */ - - public final class FunctionsRaisingIOE { private FunctionsRaisingIOE() { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteSupport.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteSupport.java index e6d2417c21aac..c43f71460338e 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteSupport.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteSupport.java @@ -163,7 +163,7 @@ public static List convertToPaths( * @return a tuple of (undeleted, deleted, failures) */ public Triple, List, List>> - processDeleteFailure( + processDeleteFailure( final MultiObjectDeleteException deleteException, final List keysToDelete) { final MetadataStore metadataStore = diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/PathToBucketKeys.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/PathToBucketKeys.java deleted file mode 100644 index a444400a6def3..0000000000000 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/PathToBucketKeys.java +++ /dev/null @@ -1,22 +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; - -public class PathToBucketKeys { -} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java index 9bfa198506cbe..92e2ffd5b0700 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java @@ -58,7 +58,7 @@ public class StoreContext { /** Filesystem URI. */ private final URI fsURI; - /** Bucket name */ + /** Bucket name. */ private final String bucket; /** FS configuration after all per-bucket overrides applied. */ @@ -75,7 +75,7 @@ public class StoreContext { * Optional as the AWS call to evaluate this may fail from a permissions * or other IOE. */ - public final Optional bucketLocation; + private final Optional bucketLocation; /** * Bounded thread pool for async operations. @@ -300,7 +300,7 @@ public ListeningExecutorService createThrottledExecutor(int capacity) { /** * Create a new executor with the capacity defined in - * {@link #executorCapacity} + * {@link #executorCapacity}. * @return a new executor for exclusive use by the caller. */ public ListeningExecutorService createThrottledExecutor() { @@ -311,8 +311,15 @@ public UserGroupInformation getOwner() { return owner; } - public File createTempFile(String pathStr, long size) throws IOException { - return tempFileFactory.createTempFile(pathStr, size); + /** + * Create a temporary file somewhere. + * @param prefix prefix for the temporary file + * @param size expected size. + * @return a file reference. + * @throws IOException failure. + */ + public File createTempFile(String prefix, long size) throws IOException { + return tempFileFactory.createTempFile(prefix, size); } /** @@ -322,6 +329,6 @@ public File createTempFile(String pathStr, long size) throws IOException { */ @FunctionalInterface public interface TempFileFactory { - File createTempFile(String pathStr, long size) throws IOException; + File createTempFile(String prefix, long size) throws IOException; } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameTracker.java index 92381fc467d7b..5c5f52377948a 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameTracker.java @@ -57,8 +57,8 @@ public DelayedUpdateRenameTracker( final Path sourceRoot, final Path dest, final BulkOperationState operationState) { - super("DelayedUpdateRenameTracker", storeContext, metadataStore, sourceRoot, dest, - operationState); + super("DelayedUpdateRenameTracker", storeContext, metadataStore, + sourceRoot, dest, operationState); this.metadataStore = storeContext.getMetadataStore(); } 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 aca16cd38694d..3e1270d6c8e6b 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 @@ -281,7 +281,8 @@ public class DynamoDBMetadataStore implements MetadataStore, */ private RetryPolicy batchWriteRetryPolicy; - private Optional instrumentation; + private Optional instrumentation = + Optional.empty(); /** Owner FS: only valid if configured with an owner FS. */ private S3AFileSystem owner; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathOrderComparators.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathOrderComparators.java index 770842ea01927..62542d55375c7 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathOrderComparators.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathOrderComparators.java @@ -18,25 +18,27 @@ package org.apache.hadoop.fs.s3a.s3guard; +import java.io.Serializable; import java.util.Comparator; import org.apache.hadoop.fs.Path; /** - * A comparator of path ordering where those paths which are higher up - * the tree come first. - * This can be used to ensure the sort order of changes. + * Comparator of path ordering for sorting collections. * - * Policy + * The definition of "topmost" is: *

          - *
        1. higher up entries come first
        2. - *
        3. Root is topmost
        4. + *
        5. The depth of a path is the primary comparator.
        6. + *
        7. Root is topmost, "0"
        8. + *
        9. If two paths are of equal depth, {@link Path#compareTo(Path)}
        10. + * is used. This delegates to URI compareTo. *
        11. repeated sorts do not change the order
        12. *
        */ -@SuppressWarnings("ComparatorNotSerializable") -class PathOrderComparators { +final class PathOrderComparators { + private PathOrderComparators() { + } /** * The shallowest paths come first. @@ -66,7 +68,7 @@ class PathOrderComparators { static final Comparator TOPMOST_PM_LAST = new PathMetadataComparator(TOPMOST_PATH_LAST); - private static class TopmostFirst implements Comparator { + private static class TopmostFirst implements Comparator, Serializable { @Override public int compare(Path pathL, Path pathR) { @@ -108,14 +110,13 @@ public int compare(final Path pathL, final Path pathR) { } return 0; } - } /** * Compare on path status. */ private static final class PathMetadataComparator implements - Comparator { + Comparator, Serializable { private final Comparator inner; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ProgressiveRenameTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ProgressiveRenameTracker.java index 93a39fc332c6f..184e85919550d 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ProgressiveRenameTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ProgressiveRenameTracker.java @@ -57,7 +57,8 @@ * When deletes are executed, the store is also updated. *
      2. *
      3. - * And at the completion of a successful rename, the source directory is also removed. + * And at the completion of a successful rename, the source directory + * is also removed. *
      4. *
      *
      diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java
      index a1e35e493e22f..783f3ed1bfe66 100644
      --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java
      +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java
      @@ -500,7 +500,7 @@ public static void addMoveAncestors(MetadataStore ms,
         }
       
         /**
      -   * This adds all new ancestors of a path as directories
      +   * This adds all new ancestors of a path as directories.
          * @param metadataStore store
          * @param qualifiedPath path to update
          * @param username username to use in all new FileStatus entries.
      diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
      index 3d39259f39804..260ecdd71d25e 100644
      --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
      +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
      @@ -28,7 +28,6 @@
       import org.apache.hadoop.fs.FileStatus;
       import org.apache.hadoop.fs.FileSystem;
       import org.apache.hadoop.fs.Path;
      -import org.apache.hadoop.fs.contract.ContractTestUtils;
       import org.apache.hadoop.fs.permission.FsPermission;
       import org.apache.hadoop.fs.s3a.auth.MarshalledCredentialBinding;
       import org.apache.hadoop.fs.s3a.auth.MarshalledCredentials;
      @@ -45,7 +44,6 @@
       import org.apache.hadoop.util.ReflectionUtils;
       
       import com.amazonaws.auth.AWSCredentialsProvider;
      -import com.amazonaws.services.s3.model.DeleteObjectsRequest;
       import org.hamcrest.core.Is;
       import org.junit.Assert;
       import org.junit.Assume;
      @@ -62,7 +60,6 @@
       import java.util.List;
       import java.util.Map;
       import java.util.concurrent.Callable;
      -import java.util.stream.Collectors;
       
       import static com.google.common.base.Preconditions.checkNotNull;
       import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CREDENTIAL_PROVIDER_PATH;
      @@ -1224,5 +1221,4 @@ public static boolean metadataStorePersistsAuthoritativeBit(MetadataStore ms)
           }
           return Boolean.valueOf(persists);
         }
      -
       }
      diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/RoleTestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/RoleTestUtils.java
      index b7e1edacb22d9..8942d70daa698 100644
      --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/RoleTestUtils.java
      +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/RoleTestUtils.java
      @@ -21,12 +21,9 @@
       import java.io.IOException;
       import java.nio.file.AccessDeniedException;
       import java.util.List;
      -import java.util.Optional;
       import java.util.concurrent.Callable;
      -import java.util.function.Function;
       import java.util.stream.Collectors;
       import java.util.stream.IntStream;
      -import java.util.stream.Stream;
       
       import com.fasterxml.jackson.core.JsonProcessingException;
       import org.junit.Assume;
      @@ -39,15 +36,12 @@
       import org.apache.hadoop.fs.FileSystem;
       import org.apache.hadoop.fs.Path;
       import org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants;
      -import org.apache.hadoop.util.LambdaUtils;
       
      -import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
       import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
       import static org.apache.hadoop.fs.s3a.Constants.*;
       import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching;
       import static org.apache.hadoop.fs.s3a.auth.RoleModel.*;
       import static org.apache.hadoop.fs.s3a.auth.RolePolicies.*;
      -import static org.apache.hadoop.test.LambdaTestUtils.eval;
       import static org.apache.hadoop.test.LambdaTestUtils.intercept;
       import static org.junit.Assert.assertEquals;
       import static org.junit.Assert.assertTrue;
      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 a6a1d64a512d0..9aa8a57edace4 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
      @@ -114,7 +114,7 @@ public class ITestPartialRenamesDeletes extends AbstractS3ATestBase {
         /**
          * For submitting work.
          */
      -  private static final ListeningExecutorService executor =
      +  private static final ListeningExecutorService EXECUTOR =
             BlockingThreadPoolExecutorService.newInstance(
                 EXECUTOR_THREAD_COUNT,
                 EXECUTOR_THREAD_COUNT * 2,
      @@ -145,7 +145,7 @@ public class ITestPartialRenamesDeletes extends AbstractS3ATestBase {
         private S3AFileSystem roleFS;
       
         /**
      -   * Base path for this test run; it's all
      +   * Base path for this test run.
          */
         private Path basePath;
       
      @@ -160,6 +160,9 @@ public class ITestPartialRenamesDeletes extends AbstractS3ATestBase {
         /** delete policy: single or multi? */
         private final boolean multiDelete;
       
      +  /**
      +   * Configuration for the assume role FS.
      +   */
         private Configuration assumedRoleConfig;
       
         private int fileCount;
      @@ -185,6 +188,10 @@ public static Collection params() {
           });
         }
       
      +  /**
      +   * Constructor.
      +   * @param multiDelete single vs multi delete in the role FS?
      +   */
         public ITestPartialRenamesDeletes(final boolean multiDelete) {
           this.multiDelete = multiDelete;
         }
      @@ -301,10 +308,12 @@ protected Configuration createConfiguration() {
       
           // ramp up the number of connections we can have for maximum PUT
           // performance
      -    removeBucketOverrides(bucketName, conf, MAX_THREADS, MAXIMUM_CONNECTIONS);
      +    removeBucketOverrides(bucketName, conf,
      +        MAX_THREADS,
      +        MAXIMUM_CONNECTIONS,
      +        S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY);
           conf.setInt(MAX_THREADS, EXECUTOR_THREAD_COUNT);
           conf.setInt(MAXIMUM_CONNECTIONS, EXECUTOR_THREAD_COUNT * 2);
      -    conf.setBoolean(ENABLE_MULTI_DELETE, multiDelete);
           // turn off prune delays, so as to stop scale tests creating
           // so much cruft that future CLI prune commands take forever
           conf.setInt(S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY, 0);
      @@ -448,7 +457,8 @@ public void testRenameDirFailsInDelete() throws Throwable {
       
           // create a set of files
           // this is done in parallel as it is 10x faster on a long-haul test run.
      -    List createdFiles = createFiles(fs, readOnlyDir, dirDepth, fileCount, dirCount);
      +    List createdFiles = createFiles(fs, readOnlyDir, dirDepth, fileCount,
      +        dirCount);
           // are they all there?
           int expectedFileCount = createdFiles.size();
           assertFileCount("files ready to rename", roleFS,
      @@ -471,7 +481,7 @@ public void testRenameDirFailsInDelete() throws Throwable {
                 .containsAll(createdFiles)
                 .containsExactlyInAnyOrderElementsOf(createdFiles);
           }
      -    LOG.info("Result of renaming read-only files is (Correctly) AccessDeniedException",
      +    LOG.info("Result of renaming read-only files is as expected",
               deniedException);
           assertFileCount("files in the source directory", roleFS,
               readOnlyDir, expectedFileCount);
      @@ -558,8 +568,10 @@ public void testPartialDirDelete() throws Throwable {
           // the full FS
           S3AFileSystem fs = getFileSystem();
       
      -    List readOnlyFiles = createFiles(fs, readOnlyDir, dirDepth, fileCount, dirCount);
      -    List deletableFiles = createFiles(fs, writableDir, dirDepth, fileCount, dirCount);
      +    List readOnlyFiles = createFiles(fs, readOnlyDir, dirDepth, fileCount,
      +        dirCount);
      +    List deletableFiles = createFiles(fs, writableDir, dirDepth, fileCount,
      +        dirCount);
       
           // as a safety check, verify that one of the deletable files can be deleted
           Path head = deletableFiles.remove(0);
      @@ -630,7 +642,8 @@ public void testPartialDirDelete() throws Throwable {
          * @return the expected exception.
          * @throws Exception any other failure.
          */
      -  private AccessDeniedException expectDeleteForbidden(Path path) throws Exception {
      +  private AccessDeniedException expectDeleteForbidden(Path path)
      +      throws Exception {
           try(DurationInfo ignored =
                   new DurationInfo(LOG, true, "delete %s", path)) {
             return forbidden("Expected an error deleting "  + path,
      @@ -702,13 +715,14 @@ private void prune(Path path) throws IOException {
          * @return an unordered set of the paths.
          * @throws IOException failure
          */
      -  private Set listFilesUnderPath(Path path, boolean recursive) throws IOException {
      +  private Set listFilesUnderPath(Path path, boolean recursive)
      +      throws IOException {
           Set files = new TreeSet<>();
      -      try (DurationInfo ignore =
      -               new DurationInfo(LOG, "ls -R %s", path)) {
      -        applyLocatedFiles(getFileSystem().listFiles(path, recursive),
      -            (status) -> files.add(status.getPath()));
      -      }
      +    try (DurationInfo ignore =
      +             new DurationInfo(LOG, "ls -R %s", path)) {
      +      applyLocatedFiles(getFileSystem().listFiles(path, recursive),
      +          (status) -> files.add(status.getPath()));
      +    }
           return files;
         }
       
      @@ -720,7 +734,7 @@ private Set listFilesUnderPath(Path path, boolean recursive) throws IOExce
          */
         private static CompletableFuture put(FileSystem fs,
             Path path, String text) {
      -    return submit(executor, () -> {
      +    return submit(EXECUTOR, () -> {
             try (DurationInfo ignore =
                      new DurationInfo(LOG, false, "Creating %s", path)) {
               createFile(fs, path, true, text.getBytes(Charsets.UTF_8));
      diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java
      index abc98e91a0810..ad7fd2cc3c459 100644
      --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java
      +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java
      @@ -46,7 +46,6 @@
       import org.apache.hadoop.fs.Path;
       import org.apache.hadoop.fs.s3a.Constants;
       import org.apache.hadoop.fs.s3a.Invoker;
      -import org.apache.hadoop.fs.s3a.Retries;
       import org.apache.hadoop.fs.s3a.S3AInputPolicy;
       import org.apache.hadoop.fs.s3a.S3AInstrumentation;
       import org.apache.hadoop.fs.s3a.S3AStorageStatistics;
      @@ -111,8 +110,8 @@ public void testSplitKeysFromResults() throws Throwable {
         }
       
         /**
      -   * Build a list of paths from a vararg stream list.DistributedFileSystem
      -   * @param paths paths in
      +   * Build a list of qualified paths from vararg parameters.
      +   * @param paths paths to qualify and then convert to a lst.
          * @return same paths as a list.
          */
         private List pathList(String... paths) {
      @@ -339,9 +338,11 @@ public List getCreated() {
           }
       
           @Override
      -    public RenameTracker initiateRenameOperation(final StoreContext storeContext,
      +    public RenameTracker initiateRenameOperation(
      +        final StoreContext storeContext,
               final Path source,
      -        final FileStatus sourceStatus, final Path dest) throws IOException {
      +        final FileStatus sourceStatus,
      +        final Path dest) throws IOException {
             throw new UnsupportedOperationException("unsupported");
           }
         }
      diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java
      index 6f9b3851c8cea..e263c99d4f0c4 100644
      --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java
      +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java
      @@ -204,12 +204,14 @@ protected void mkdirs(Path path, boolean onS3, boolean onMetadataStore)
         protected static void putFile(MetadataStore ms, S3AFileStatus f)
             throws IOException {
           assertNotNull(f);
      -    ms.put(new PathMetadata(f), null);
      -    Path parent = f.getPath().getParent();
      -    while (parent != null) {
      -      S3AFileStatus dir = new S3AFileStatus(false, parent, f.getOwner());
      -      ms.put(new PathMetadata(dir), null);
      -      parent = parent.getParent();
      +    try(BulkOperationState bulkWrite = ms.initiateBulkWrite(f.getPath())) {
      +      ms.put(new PathMetadata(f), bulkWrite);
      +      Path parent = f.getPath().getParent();
      +      while (parent != null) {
      +        S3AFileStatus dir = new S3AFileStatus(false, parent, f.getOwner());
      +        ms.put(new PathMetadata(dir), bulkWrite);
      +        parent = parent.getParent();
      +      }
           }
         }
       
      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 e19cf25a39f12..d2176d941ca20 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
      @@ -152,9 +152,9 @@ public static void beforeClassSetup() throws IOException {
           // be configured to use this test.
           testDynamoDBTableName = conf.get(S3GUARD_DDB_TEST_TABLE_NAME_KEY);
           String dynamoDbTableName = conf.getTrimmed(S3GUARD_DDB_TABLE_NAME_KEY);
      -    Assume.assumeTrue("No DynamoDB table name configured in " + S3GUARD_DDB_TABLE_NAME_KEY,
      -        !StringUtils
      -            .isEmpty(dynamoDbTableName));
      +    Assume.assumeTrue("No DynamoDB table name configured in "
      +            + S3GUARD_DDB_TABLE_NAME_KEY,
      +        !StringUtils.isEmpty(dynamoDbTableName));
       
           // We should assert that the table name is configured, so the test should
           // fail if it's not configured.
      @@ -576,10 +576,13 @@ public void testMovePopulatesAncestors() throws IOException {
           final String srcRoot = testRoot + "/a/b/src";
           final String destRoot = testRoot + "/c/d/e/dest";
       
      +    AncestorState bulkWrite = ddbms.initiateBulkWrite(null);
           final Path nestedPath1 = strToPath(srcRoot + "/file1.txt");
      -    ddbms.put(new PathMetadata(basicFileStatus(nestedPath1, 1024, false)), null);
      +    ddbms.put(new PathMetadata(basicFileStatus(nestedPath1, 1024, false)),
      +        bulkWrite);
           final Path nestedPath2 = strToPath(srcRoot + "/dir1/dir2");
      -    ddbms.put(new PathMetadata(basicFileStatus(nestedPath2, 0, true)), null);
      +    ddbms.put(new PathMetadata(basicFileStatus(nestedPath2, 0, true)),
      +        bulkWrite);
       
           // We don't put the destRoot path here, since put() would create ancestor
           // entries, and we want to ensure that move() does it, instead.
      @@ -589,8 +592,7 @@ public void testMovePopulatesAncestors() throws IOException {
               strToPath(srcRoot),
               strToPath(srcRoot + "/dir1"),
               strToPath(srcRoot + "/dir1/dir2"),
      -        strToPath(srcRoot + "/file1.txt")
      -    );
      +        strToPath(srcRoot + "/file1.txt"));
           final Collection pathsToCreate = Lists.newArrayList(
               new PathMetadata(basicFileStatus(strToPath(destRoot),
                   0, true)),
      @@ -602,8 +604,9 @@ public void testMovePopulatesAncestors() throws IOException {
                   1024, false))
           );
       
      -    ddbms.move(fullSourcePaths, pathsToCreate, null);
      +    ddbms.move(fullSourcePaths, pathsToCreate, bulkWrite);
       
      +    bulkWrite.close();
           // assert that all the ancestors should have been populated automatically
           assertCached(testRoot + "/c");
           assertCached(testRoot + "/c/d");
      diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/ExtraAssertions.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/ExtraAssertions.java
      index 78a5ee5d1a3b3..2b93d72face4e 100644
      --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/ExtraAssertions.java
      +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/ExtraAssertions.java
      @@ -34,18 +34,20 @@
       import org.apache.hadoop.util.DurationInfo;
       
       import static org.apache.hadoop.fs.s3a.S3AUtils.applyLocatedFiles;
      -import static org.junit.Assert.assertNotNull;
       import static org.junit.Assert.assertTrue;
       
       /**
        * Some extra assertions for tests.
        */
       @InterfaceAudience.Private
      -public class ExtraAssertions {
      +public final class ExtraAssertions {
       
         private static final Logger LOG = LoggerFactory.getLogger(
             ExtraAssertions.class);
       
      +  private ExtraAssertions() {
      +  }
      +
         /**
          * Assert that the number of files in a destination matches that expected.
          * @param message text to use in the message
      
      From 3982cebd60bc98a9f707f8b271c6d9f913976db7 Mon Sep 17 00:00:00 2001
      From: Steve Loughran 
      Date: Sun, 19 May 2019 20:53:51 +0100
      Subject: [PATCH 22/22] HADOOP-15183 remove diff between this branch and trunk;
       checkstyle
      
      Change-Id: I518f427a287e72ac5d617d08ec71e24fbd683a14
      ---
       .../apache/hadoop/fs/s3a/S3AFileSystem.java   | 31 +++++++-------
       .../fs/s3a/impl/FunctionsRaisingIOE.java      |  4 ++
       .../fs/s3a/impl/MultiObjectDeleteSupport.java |  3 +-
       .../fs/s3a/s3guard/DynamoDBMetadataStore.java | 42 +++++++++++--------
       .../apache/hadoop/fs/s3a/s3guard/S3Guard.java |  7 +++-
       .../s3a/impl/ITestPartialRenamesDeletes.java  |  8 ++--
       6 files changed, 55 insertions(+), 40 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 7f3c87479cb49..733d76e61ffce 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
      @@ -1272,12 +1272,12 @@ private long innerRename(Path source, Path dest)
           // the list.
           final FunctionsRaisingIOE.FunctionRaisingIOE
               completeActiveCopies = (String reason) -> {
      -            LOG.debug("Waiting for {} active copies to complete: {}",
      -                activeCopies.size(), reason);
      -            waitForCompletion(activeCopies);
      -            activeCopies.clear();
      -            return null;
      -          };
      +          LOG.debug("Waiting for {} active copies to complete: {}",
      +              activeCopies.size(), reason);
      +          waitForCompletion(activeCopies);
      +          activeCopies.clear();
      +          return null;
      +        };
       
           try {
             if (srcStatus.isFile()) {
      @@ -1322,7 +1322,8 @@ private long innerRename(Path source, Path dest)
       
               // These are the lists of keys to delete and of their paths, the
               // latter being used to update the rename tracker.
      -        final List keysToDelete = new ArrayList<>();
      +        final List keysToDelete =
      +            new ArrayList<>();
               final List pathsToDelete = new ArrayList<>();
               // to update the lists of keys and paths.
               final BiFunction queueToDelete =
      @@ -1335,14 +1336,14 @@ private long innerRename(Path source, Path dest)
               // then delete all queued keys + paths to delete.
               final FunctionsRaisingIOE.FunctionRaisingIOE
                   completeActiveCopiesAndDeleteSources =
      -            (String reason) -> {
      -              completeActiveCopies.apply(reason);
      -              removeSourceObjects(renameTracker, keysToDelete, pathsToDelete);
      -              // now reset the lists.
      -              keysToDelete.clear();
      -              pathsToDelete.clear();
      -              return null;
      -            };
      +                (reason) -> {
      +                  completeActiveCopies.apply(reason);
      +                  removeSourceObjects(renameTracker, keysToDelete, pathsToDelete);
      +                  // now reset the lists.
      +                  keysToDelete.clear();
      +                  pathsToDelete.clear();
      +                  return null;
      +                };
       
               if (dstStatus != null && dstStatus.isEmptyDirectory() == Tristate.TRUE) {
                 // delete unnecessary fake directory at the destination.
      diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/FunctionsRaisingIOE.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/FunctionsRaisingIOE.java
      index e7e533b082fc3..e2f7caab5330c 100644
      --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/FunctionsRaisingIOE.java
      +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/FunctionsRaisingIOE.java
      @@ -41,6 +41,10 @@ public interface FunctionRaisingIOE {
           R apply(T p) throws IOException;
         }
       
      +  /**
      +   * This is a callable which only raises an IOException.
      +   * @param  return type
      +   */
         @FunctionalInterface
         public interface CallableRaisingIOE {
       
      diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteSupport.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteSupport.java
      index c43f71460338e..fac0779cc6a12 100644
      --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteSupport.java
      +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteSupport.java
      @@ -163,7 +163,8 @@ public static List convertToPaths(
          * @return a tuple of (undeleted, deleted, failures)
          */
         public Triple, List, List>>
      -     processDeleteFailure(
      +      processDeleteFailure(
      +
             final MultiObjectDeleteException deleteException,
             final List keysToDelete) {
           final MetadataStore metadataStore =
      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 3e1270d6c8e6b..696c128c94475 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
      @@ -34,7 +34,6 @@
       import java.util.List;
       import java.util.Map;
       import java.util.Objects;
      -import java.util.Optional;
       import java.util.Set;
       import java.util.TreeMap;
       import java.util.concurrent.CompletableFuture;
      @@ -281,8 +280,7 @@ public class DynamoDBMetadataStore implements MetadataStore,
          */
         private RetryPolicy batchWriteRetryPolicy;
       
      -  private Optional instrumentation =
      -      Optional.empty();
      +  private S3AInstrumentation.S3GuardInstrumentation instrumentation;
       
         /** Owner FS: only valid if configured with an owner FS. */
         private S3AFileSystem owner;
      @@ -392,8 +390,7 @@ public void initialize(FileSystem fs) throws IOException {
       
           initTable();
       
      -    instrumentation.ifPresent(
      -        S3AInstrumentation.S3GuardInstrumentation::initialized);
      +    instrumentation.initialized();
         }
       
         /**
      @@ -407,8 +404,7 @@ void bindToOwnerFilesystem(final S3AFileSystem fs) {
           owner = fs;
           conf = owner.getConf();
           StoreContext context = owner.createStoreContext();
      -    instrumentation = Optional.of(
      -        context.getInstrumentation().getS3GuardInstrumentation());
      +    instrumentation = context.getInstrumentation().getS3GuardInstrumentation();
           username = context.getUsername();
           executor = context.createThrottledExecutor();
         }
      @@ -1101,16 +1097,17 @@ public void put(DirListingMetadata meta) throws IOException {
           // next add all children of the directory
           metasToPut.addAll(pathMetaToDDBPathMeta(meta.getListing()));
       
      -    // and sort
      +    // sort so highest-level entries are written to the store first.
      +    // if a sequence fails, no orphan entries will have been written.
           metasToPut.sort(PathOrderComparators.TOPMOST_PM_FIRST);
      -
           processBatchWriteRequest(null, pathMetadataToItem(metasToPut));
         }
       
         @Override
         public synchronized void close() {
      -    instrumentation.ifPresent(
      -        S3AInstrumentation.S3GuardInstrumentation::storeClosed);
      +    if (instrumentation != null) {
      +      instrumentation.storeClosed();
      +    }
           try {
             if (dynamoDB != null) {
               LOG.debug("Shutting down {}", this);
      @@ -1793,8 +1790,9 @@ void retryEvent(
             boolean idempotent) {
           if (S3AUtils.isThrottleException(ex)) {
             // throttled
      -      instrumentation.ifPresent(
      -          S3AInstrumentation.S3GuardInstrumentation::throttled);
      +      if (instrumentation != null) {
      +        instrumentation.throttled();
      +      }
             int eventCount = throttleEventCount.addAndGet(1);
             if (attempts == 1 && eventCount < THROTTLE_EVENT_LOG_LIMIT) {
               LOG.warn("DynamoDB IO limits reached in {};"
      @@ -1810,8 +1808,11 @@ void retryEvent(
             LOG.info("Retrying {}: {}", text, ex.toString());
             LOG.debug("Retrying {}", text, ex);
           }
      -    instrumentation.ifPresent(
      -        S3AInstrumentation.S3GuardInstrumentation::retrying);
      +
      +    if (instrumentation != null) {
      +      // note a retry
      +      instrumentation.retrying();
      +    }
           if (owner != null) {
             owner.metastoreOperationRetried(ex, attempts, idempotent);
           }
      @@ -1850,7 +1851,9 @@ public Invoker getInvoker() {
          * @param count count of records.
          */
         private void recordsWritten(final int count) {
      -    instrumentation.ifPresent(i -> i.recordsWritten(count));
      +    if (instrumentation != null) {
      +      instrumentation.recordsWritten(count);
      +    }
         }
       
         /**
      @@ -1858,7 +1861,9 @@ private void recordsWritten(final int count) {
          * @param count count of records.
          */
         private void recordsRead(final int count) {
      -    instrumentation.ifPresent(i -> i.recordsRead(count));
      +    if (instrumentation != null) {
      +      instrumentation.recordsRead(count);
      +    }
         }
       
         /**
      @@ -1871,7 +1876,8 @@ private void recordsRead(final int count) {
          * @return the rename tracker
          */
         @Override
      -  public RenameTracker initiateRenameOperation(final StoreContext storeContext,
      +  public RenameTracker initiateRenameOperation(
      +      final StoreContext storeContext,
             final Path source,
             final FileStatus sourceStatus,
             final Path dest) {
      diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java
      index 783f3ed1bfe66..791b58aa3669d 100644
      --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java
      +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java
      @@ -172,9 +172,12 @@ public static S3AFileStatus putAndReturn(
           try {
             ms.put(new PathMetadata(status), operationState);
           } finally {
      -      instrumentation.addValueToQuantiles(S3GUARD_METADATASTORE_PUT_PATH_LATENCY,
      +      instrumentation.addValueToQuantiles(
      +          S3GUARD_METADATASTORE_PUT_PATH_LATENCY,
                 (System.nanoTime() - startTimeNano));
      -      instrumentation.incrementCounter(S3GUARD_METADATASTORE_PUT_PATH_REQUEST, 1);
      +      instrumentation.incrementCounter(
      +          S3GUARD_METADATASTORE_PUT_PATH_REQUEST,
      +          1);
           }
           return status;
         }
      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 9aa8a57edace4..bec8e1fa054b9 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
      @@ -568,10 +568,10 @@ public void testPartialDirDelete() throws Throwable {
           // the full FS
           S3AFileSystem fs = getFileSystem();
       
      -    List readOnlyFiles = createFiles(fs, readOnlyDir, dirDepth, fileCount,
      -        dirCount);
      -    List deletableFiles = createFiles(fs, writableDir, dirDepth, fileCount,
      -        dirCount);
      +    List readOnlyFiles = createFiles(fs, readOnlyDir,
      +        dirDepth, fileCount, dirCount);
      +    List deletableFiles = createFiles(fs,
      +        writableDir, dirDepth, fileCount, dirCount);
       
           // as a safety check, verify that one of the deletable files can be deleted
           Path head = deletableFiles.remove(0);