You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by st...@apache.org on 2019/06/20 08:57:58 UTC

[hadoop] branch trunk updated: HADOOP-15183. S3Guard store becomes inconsistent after partial failure of rename.

This is an automated email from the ASF dual-hosted git repository.

stevel pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git


The following commit(s) were added to refs/heads/trunk by this push:
     new e02eb24  HADOOP-15183. S3Guard store becomes inconsistent after partial failure of rename.
e02eb24 is described below

commit e02eb24e0a9139418120027b694492e0738df20a
Author: Steve Loughran <st...@cloudera.com>
AuthorDate: Thu Jun 20 09:56:40 2019 +0100

    HADOOP-15183. S3Guard store becomes inconsistent after partial failure of rename.
    
    Contributed by Steve Loughran.
    
    Change-Id: I825b0bc36be960475d2d259b1cdab45ae1bb78eb
---
 .../apache/hadoop/fs/impl/FunctionsRaisingIOE.java |  69 ++
 .../org/apache/hadoop/fs/impl/FutureIOSupport.java |  48 +-
 .../src/main/resources/core-default.xml            |  31 +-
 .../fs/contract/AbstractContractRenameTest.java    |   2 +-
 hadoop-tools/hadoop-aws/pom.xml                    |   5 +
 .../java/org/apache/hadoop/fs/s3a/Constants.java   |  28 +-
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java    | 685 ++++++++++------
 .../org/apache/hadoop/fs/s3a/S3AInputStream.java   |   9 +-
 .../apache/hadoop/fs/s3a/S3AInstrumentation.java   |  31 +-
 .../java/org/apache/hadoop/fs/s3a/S3AUtils.java    |  45 +-
 .../apache/hadoop/fs/s3a/S3ObjectAttributes.java   |  52 +-
 .../java/org/apache/hadoop/fs/s3a/Statistic.java   |  11 +
 .../apache/hadoop/fs/s3a/WriteOperationHelper.java |  62 +-
 .../hadoop/fs/s3a/commit/AbstractS3ACommitter.java |  42 +-
 .../hadoop/fs/s3a/commit/CommitOperations.java     | 155 +++-
 .../fs/s3a/commit/magic/MagicS3GuardCommitter.java |   7 +-
 .../fs/s3a/commit/staging/StagingCommitter.java    |  12 +-
 .../hadoop/fs/s3a/impl/AbstractStoreOperation.java |  49 ++
 .../hadoop/fs/s3a/impl/CallableSupplier.java       | 126 +++
 .../hadoop/fs/s3a/impl/ContextAccessors.java       |  74 ++
 .../hadoop/fs/s3a/impl/InternalConstants.java      |  60 ++
 .../fs/s3a/impl/MultiObjectDeleteSupport.java      | 244 ++++++
 .../apache/hadoop/fs/s3a/impl/RenameOperation.java | 634 +++++++++++++++
 .../apache/hadoop/fs/s3a/impl/StoreContext.java    | 335 ++++++++
 .../hadoop/fs/s3a/s3guard/BulkOperationState.java  |  82 ++
 .../fs/s3a/s3guard/DelayedUpdateRenameTracker.java | 192 +++++
 .../fs/s3a/s3guard/DynamoDBMetadataStore.java      | 688 ++++++++++++++--
 .../hadoop/fs/s3a/s3guard/LocalMetadataStore.java  |  79 +-
 .../hadoop/fs/s3a/s3guard/MetadataStore.java       |  89 ++-
 .../hadoop/fs/s3a/s3guard/NullMetadataStore.java   |  57 +-
 .../s3guard/PathMetadataDynamoDBTranslation.java   |  22 +-
 .../fs/s3a/s3guard/PathOrderComparators.java       | 133 ++++
 .../fs/s3a/s3guard/ProgressiveRenameTracker.java   | 252 ++++++
 .../hadoop/fs/s3a/s3guard/RenameTracker.java       | 275 +++++++
 .../org/apache/hadoop/fs/s3a/s3guard/S3Guard.java  | 197 ++++-
 .../apache/hadoop/fs/s3a/s3guard/S3GuardTool.java  |  37 +-
 .../s3a/s3guard/TableDeleteTimeoutException.java   |  34 +
 .../src/site/markdown/tools/hadoop-aws/s3guard.md  |  12 +
 .../fs/contract/s3a/ITestS3AContractRename.java    |  71 +-
 .../hadoop/fs/s3a/ITestS3AFailureHandling.java     | 102 ++-
 .../s3a/ITestS3AMetadataPersistenceException.java  |  10 +-
 .../hadoop/fs/s3a/ITestS3GuardListConsistency.java |  13 +-
 .../apache/hadoop/fs/s3a/MockS3AFileSystem.java    |   4 +-
 .../hadoop/fs/s3a/TestStreamChangeTracker.java     |   7 +-
 .../apache/hadoop/fs/s3a/auth/ITestAssumeRole.java | 219 +-----
 .../apache/hadoop/fs/s3a/auth/RoleTestUtils.java   |  46 +-
 .../hadoop/fs/s3a/commit/AbstractCommitITest.java  |   2 +
 .../fs/s3a/commit/AbstractITCommitMRJob.java       |   4 +
 .../fs/s3a/commit/AbstractYarnClusterITest.java    |   8 +-
 .../fs/s3a/commit/ITestCommitOperations.java       | 127 ++-
 .../fs/s3a/commit/magic/ITestMagicCommitMRJob.java |  26 +-
 .../s3a/commit/magic/ITestS3AHugeMagicCommits.java |   9 +-
 .../integration/ITestDirectoryCommitMRJob.java     |   2 +-
 .../integration/ITestPartitionCommitMRJob.java     |   2 +-
 .../integration/ITestStagingCommitMRJob.java       |   2 +-
 .../ITestStagingCommitMRJobBadDest.java            |   2 +-
 .../commit/terasort/AbstractCommitTerasortIT.java  |   5 +
 .../fs/s3a/impl/ITestPartialRenamesDeletes.java    | 871 +++++++++++++++++++++
 .../fs/s3a/impl/TestPartialDeleteFailures.java     | 393 ++++++++++
 .../s3a/s3guard/AbstractS3GuardToolTestBase.java   |  44 +-
 .../fs/s3a/s3guard/ITestDynamoDBMetadataStore.java | 631 +++++++++++----
 .../s3guard/ITestDynamoDBMetadataStoreScale.java   | 227 +++---
 .../fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java   |   5 +-
 .../fs/s3a/s3guard/ITestS3GuardToolLocal.java      |   3 +-
 .../fs/s3a/s3guard/MetadataStoreTestBase.java      | 246 ++++--
 .../fs/s3a/s3guard/TestDynamoDBMiscOperations.java |  69 ++
 .../fs/s3a/s3guard/TestPathOrderComparators.java   | 197 +++++
 .../apache/hadoop/fs/s3a/s3guard/TestS3Guard.java  |  12 +-
 .../hadoop/fs/s3a/s3guard/ThrottleTracker.java     | 134 ++++
 .../scale/AbstractITestS3AMetadataStoreScale.java  |  19 +-
 .../apache/hadoop/fs/s3a/test/ExtraAssertions.java | 138 ++++
 71 files changed, 7564 insertions(+), 1051 deletions(-)

diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FunctionsRaisingIOE.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FunctionsRaisingIOE.java
new file mode 100644
index 0000000..7bbb346
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FunctionsRaisingIOE.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.impl;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Evolving support for functional programming/lambda-expressions.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public final class FunctionsRaisingIOE {
+
+  private FunctionsRaisingIOE() {
+  }
+
+  /**
+   * Function of arity 1 which may raise an IOException.
+   * @param <T> type of arg1
+   * @param <R> type of return value.
+   */
+  @FunctionalInterface
+  public interface FunctionRaisingIOE<T, R> {
+
+    R apply(T t) throws IOException;
+  }
+
+  /**
+   * Function of arity 2 which may raise an IOException.
+   * @param <T> type of arg1
+   * @param <U> type of arg2
+   * @param <R> type of return value.
+   */
+  @FunctionalInterface
+  public interface BiFunctionRaisingIOE<T, U, R> {
+
+    R apply(T t, U u) throws IOException;
+  }
+
+  /**
+   * This is a callable which only raises an IOException.
+   * @param <R> return type
+   */
+  @FunctionalInterface
+  public interface CallableRaisingIOE<R> {
+
+    R apply() throws IOException;
+  }
+
+}
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 9d5f2bf..9fe4023 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 @@ package org.apache.hadoop.fs.impl;
 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 final class FutureIOSupport {
    */
   public static <T> 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 <T> 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> 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-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
index 7ffc2ad..4e22e0a 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 @@
 
 <property>
   <name>fs.s3a.connection.maximum</name>
-  <value>15</value>
-  <description>Controls the maximum number of simultaneous connections to S3.</description>
+  <value>48</value>
+  <description>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.
+  </description>
 </property>
 
 <property>
@@ -1312,7 +1316,7 @@
 
 <property>
   <name>fs.s3a.threads.max</name>
-  <value>10</value>
+  <value>64</value>
   <description>The total number of threads available in the filesystem for data
     uploads *or any other queued filesystem operation*.</description>
 </property>
@@ -1326,8 +1330,25 @@
 
 <property>
   <name>fs.s3a.max.total.tasks</name>
-  <value>5</value>
-  <description>The number of operations which can be queued for execution</description>
+  <value>32</value>
+  <description>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.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.executor.capacity</name>
+  <value>16</value>
+  <description>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.
+  </description>
 </property>
 
 <property>
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 5b76a75..2751294 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 abstract class AbstractContractRenameTest extends
    * @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/pom.xml b/hadoop-tools/hadoop-aws/pom.xml
index 9dc0acc..dbe593d 100644
--- a/hadoop-tools/hadoop-aws/pom.xml
+++ b/hadoop-tools/hadoop-aws/pom.xml
@@ -418,6 +418,11 @@
       <scope>compile</scope>
     </dependency>
     <dependency>
+      <groupId>org.assertj</groupId>
+      <artifactId>assertj-core</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>
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 7334506..7dc38db 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
@@ -138,9 +138,15 @@ public final class 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 = 32;
+
   // 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 = 48;
 
   // connect to s3 over ssl?
   public static final String SECURE_CONNECTIONS =
@@ -194,10 +200,6 @@ public final class 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
@@ -283,6 +285,22 @@ public final class 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 = 16;
+
   // 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 4bd58d5..874e2db 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,12 +27,14 @@ import java.net.URI;
 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;
 import java.util.Date;
 import java.util.EnumSet;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
@@ -84,6 +86,8 @@ import com.google.common.util.concurrent.ListeningExecutorService;
 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,8 +95,15 @@ import org.apache.hadoop.fs.CreateFlag;
 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.ContextAccessors;
 import org.apache.hadoop.fs.s3a.impl.CopyOutcome;
+import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport;
+import org.apache.hadoop.fs.s3a.impl.RenameOperation;
+import org.apache.hadoop.fs.s3a.impl.StoreContext;
+import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
 import org.apache.hadoop.fs.s3a.select.InternalSelectConstants;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.DurationInfo;
 import org.apache.hadoop.util.LambdaUtils;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileStatus;
@@ -203,6 +214,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 =
@@ -380,6 +392,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       LOG.debug("Using S3ABlockOutputStream with buffer = {}; block={};" +
               " queue limit={}",
           blockOutputBuffer, partSize, blockOutputActiveBlocks);
+      long authDirTtl = conf.getTimeDuration(METADATASTORE_METADATA_TTL,
+          DEFAULT_METADATASTORE_METADATA_TTL, TimeUnit.MILLISECONDS);
+      ttlTimeProvider = new S3Guard.TtlTimeProvider(authDirTtl);
 
       setMetadataStore(S3Guard.getMetadataStore(this));
       allowAuthoritative = conf.getBoolean(METADATASTORE_AUTHORITATIVE,
@@ -389,17 +404,18 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
             getMetadataStore(), allowAuthoritative);
       }
       initMultipartUploads(conf);
-      if (hasMetadataStore()) {
-        long authDirTtl = conf.getTimeDuration(METADATASTORE_METADATA_TTL,
-            DEFAULT_METADATASTORE_METADATA_TTL, TimeUnit.MILLISECONDS);
-        ttlTimeProvider = new S3Guard.TtlTimeProvider(authDirTtl);
-      }
     } catch (AmazonClientException e) {
       throw translateException("initializing ", new Path(name), e);
     }
 
   }
 
+  /**
+   * Initialize the thread pool.
+   * This must be re-invoked after replacing the S3Client during test
+   * runs.
+   * @param conf configuration.
+   */
   private void initThreadPools(Configuration conf) {
     int maxThreads = conf.getInt(MAX_THREADS, DEFAULT_MAX_THREADS);
     if (maxThreads < 2) {
@@ -418,9 +434,11 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     unboundedThreadPool = new ThreadPoolExecutor(
         maxThreads, Integer.MAX_VALUE,
         keepAliveTime, TimeUnit.SECONDS,
-        new LinkedBlockingQueue<Runnable>(),
+        new LinkedBlockingQueue<>(),
         BlockingThreadPoolExecutorService.newDaemonThreadFactory(
             "s3a-transfer-unbounded"));
+    executorCapacity = intOption(conf,
+        EXECUTOR_CAPACITY, DEFAULT_EXECUTOR_CAPACITY, 1);
   }
 
   /**
@@ -689,6 +707,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * @return the region in which a bucket is located
    * @throws IOException on any failure.
    */
+  @VisibleForTesting
   @Retries.RetryTranslated
   public String getBucketLocation(String bucketName) throws IOException {
     return invoker.retry("getBucketLocation()", bucketName, true,
@@ -733,21 +752,29 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
 
   /**
    * Demand create the directory allocator, then create a temporary file.
+   * This does not mark the file for deletion when a process exits.
    * {@link LocalDirAllocator#createTmpFileForWrite(String, long, Configuration)}.
-   *  @param pathStr prefix for the temporary file
-   *  @param size the size of the file that is going to be written
-   *  @param conf the Configuration object
-   *  @return a unique temporary file
-   *  @throws IOException IO problems
+   * @param pathStr prefix for the temporary file
+   * @param size the size of the file that is going to be written
+   * @param conf the Configuration object
+   * @return a unique temporary file
+   * @throws IOException IO problems
    */
-  synchronized File createTmpFileForWrite(String pathStr, long size,
+  File createTmpFileForWrite(String pathStr, long size,
       Configuration conf) throws IOException {
     if (directoryAllocator == null) {
-      String bufferDir = conf.get(BUFFER_DIR) != null
-          ? BUFFER_DIR : HADOOP_TMP_DIR;
-      directoryAllocator = new LocalDirAllocator(bufferDir);
+      synchronized (this) {
+        String bufferDir = conf.get(BUFFER_DIR) != null
+            ? BUFFER_DIR : HADOOP_TMP_DIR;
+        directoryAllocator = new LocalDirAllocator(bufferDir);
+      }
     }
-    return directoryAllocator.createTmpFileForWrite(pathStr, size, conf);
+    Path path = directoryAllocator.getLocalPathForWrite(pathStr,
+        size, conf);
+    File dir = new File(path.getParent().toUri().getPath());
+    String prefix = path.getName();
+    // create a temp file on this directory
+    return File.createTempFile(prefix, null, dir);
   }
 
   /**
@@ -929,11 +956,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     return new FSDataInputStream(
         new S3AInputStream(
             readContext,
-            createObjectAttributes(
-                path,
-                fileStatus.getETag(),
-                fileStatus.getVersionId()),
-            fileStatus.getLen(),
+            createObjectAttributes(fileStatus),
             s3));
   }
 
@@ -963,22 +986,40 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   }
 
   /**
-   * Create the attributes of an object for a get/select request.
+   * Create the attributes of an object for subsequent use.
    * @param f path path of the request.
    * @param eTag the eTag of the S3 object
    * @param versionId S3 object version ID
+   * @param len length of the file
    * @return attributes to use when building the query.
    */
   private S3ObjectAttributes createObjectAttributes(
       final Path f,
       final String eTag,
-      final String versionId) {
+      final String versionId,
+      final long len) {
     return new S3ObjectAttributes(bucket,
+        f,
         pathToKey(f),
         getServerSideEncryptionAlgorithm(),
         encryptionSecrets.getEncryptionKey(),
         eTag,
-        versionId);
+        versionId,
+        len);
+  }
+
+  /**
+   * Create the attributes of an object for subsequent use.
+   * @param fileStatus file status to build from.
+   * @return attributes to use when building the query.
+   */
+  private S3ObjectAttributes createObjectAttributes(
+      final S3AFileStatus fileStatus) {
+    return createObjectAttributes(
+        fileStatus.getPath(),
+        fileStatus.getETag(),
+        fileStatus.getVersionId(),
+        fileStatus.getLen());
   }
 
   /**
@@ -1117,9 +1158,13 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * @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) {
@@ -1132,33 +1177,22 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   }
 
   /**
-   * The inner rename operation. See {@link #rename(Path, Path)} for
-   * the description of the operation.
-   * This operation throws an exception on any failure which needs to be
-   * reported and downgraded to a failure.
-   * Retries: retry translated, assuming all operations it is called do
-   * so. For safely, consider catch and handle AmazonClientException
-   * because this is such a complex method there's a risk it could surface.
-   * @param source path to be renamed
-   * @param dest new path after rename
+   * Validate the rename parameters and status of the filesystem;
+   * returns the source and any destination File Status.
+   * @param src qualified path to be renamed
+   * @param dst qualified path after rename
+   * @return the source and (possibly null) destination status entries.
    * @throws RenameFailedException if some criteria for a state changing
    * 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.
    * @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)
-      throws RenameFailedException, FileNotFoundException, IOException,
-        AmazonClientException {
-    Path src = qualify(source);
-    Path dst = qualify(dest);
-
-    LOG.debug("Rename path {} to {}", src, dst);
-    entryPoint(INVOCATION_RENAME);
-
+  @Retries.RetryTranslated
+  private Pair<S3AFileStatus, S3AFileStatus> initiateRename(
+      final Path src,
+      final Path dst) throws IOException {
     String srcKey = pathToKey(src);
     String dstKey = pathToKey(dst);
 
@@ -1227,131 +1261,126 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
         }
       }
     }
+    return Pair.of(srcStatus, dstStatus);
+  }
 
-    // If we have a MetadataStore, track deletions/creations.
-    Collection<Path> srcPaths = null;
-    List<PathMetadata> 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();
-      S3ObjectAttributes objectAttributes =
-          createObjectAttributes(srcStatus.getPath(),
-              srcStatus.getETag(), srcStatus.getVersionId());
-      S3AReadOpContext readContext = createReadContext(srcStatus, inputPolicy,
-          changeDetectionPolicy, readAhead);
-      if (dstStatus != null && dstStatus.isDirectory()) {
-        String newDstKey = maybeAddTrailingSlash(dstKey);
-        String filename =
-            srcKey.substring(pathToKey(src.getParent()).length()+1);
-        newDstKey = newDstKey + filename;
-        CopyResult copyResult = copyFile(srcKey, newDstKey, length,
-            objectAttributes, readContext);
-        S3Guard.addMoveFile(metadataStore, srcPaths, dstMetas, src,
-            keyToQualifiedPath(newDstKey), length, getDefaultBlockSize(dst),
-            username, copyResult.getETag(), copyResult.getVersionId());
-      } else {
-        CopyResult copyResult = copyFile(srcKey, dstKey, srcStatus.getLen(),
-            objectAttributes, readContext);
-        S3Guard.addMoveFile(metadataStore, srcPaths, dstMetas, src, dst,
-            length, getDefaultBlockSize(dst), username,
-            copyResult.getETag(), copyResult.getVersionId());
-      }
-      innerDelete(srcStatus, false);
-    } else {
-      LOG.debug("rename: renaming directory {} to {}", src, dst);
+  /**
+   * The inner rename operation. See {@link #rename(Path, Path)} for
+   * the description of the operation.
+   * This operation throws an exception on any failure which needs to be
+   * reported and downgraded to a failure.
+   * Retries: retry translated, assuming all operations it is called do
+   * so. For safely, consider catch and handle AmazonClientException
+   * because this is such a complex method there's a risk it could surface.
+   * @param source path to be renamed
+   * @param dest new path after rename
+   * @throws RenameFailedException if some criteria for a state changing
+   * 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 long innerRename(Path source, Path dest)
+      throws RenameFailedException, FileNotFoundException, IOException,
+        AmazonClientException {
+    Path src = qualify(source);
+    Path dst = qualify(dest);
 
-      // This is a directory to directory copy
-      dstKey = maybeAddTrailingSlash(dstKey);
-      srcKey = maybeAddTrailingSlash(srcKey);
+    LOG.debug("Rename path {} to {}", src, dst);
+    entryPoint(INVOCATION_RENAME);
 
-      //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 ");
-      }
+    String srcKey = pathToKey(src);
+    String dstKey = pathToKey(dst);
 
-      List<DeleteObjectsRequest.KeyVersion> keysToDelete = new ArrayList<>();
-      if (dstStatus != null && dstStatus.isEmptyDirectory() == Tristate.TRUE) {
-        // delete unnecessary fake directory.
-        keysToDelete.add(new DeleteObjectsRequest.KeyVersion(dstKey));
-      }
+    Pair<S3AFileStatus, S3AFileStatus> p = initiateRename(src, dst);
 
-      Path parentPath = keyToQualifiedPath(srcKey);
-      RemoteIterator<S3ALocatedFileStatus> iterator =
-          listFilesAndEmptyDirectories(parentPath, true);
-      while (iterator.hasNext()) {
-        S3ALocatedFileStatus status = iterator.next();
-        long length = status.getLen();
-        String key = pathToKey(status.getPath());
-        if (status.isDirectory() && !key.endsWith("/")) {
-          key += "/";
-        }
-        keysToDelete
-            .add(new DeleteObjectsRequest.KeyVersion(key));
-        String newDstKey =
-            dstKey + key.substring(srcKey.length());
-        S3ObjectAttributes objectAttributes =
-            createObjectAttributes(status.getPath(),
-                status.getETag(), status.getVersionId());
-        S3AReadOpContext readContext = createReadContext(status, inputPolicy,
-            changeDetectionPolicy, readAhead);
-        CopyResult copyResult = copyFile(key, newDstKey, length,
-            objectAttributes, readContext);
-
-        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);
-          } else {
-            S3Guard.addMoveFile(metadataStore, srcPaths, dstMetas, childSrc,
-                childDst, length, getDefaultBlockSize(childDst), username,
-                copyResult.getETag(), copyResult.getVersionId());
-          }
-          // Ancestor directories may not be listed, so we explicitly add them
-          S3Guard.addMoveAncestors(metadataStore, srcPaths, dstMetas,
-              keyToQualifiedPath(srcKey), childSrc, childDst, username);
-        }
+    // Initiate the rename.
+    // this will call back into this class via the rename callbacks
+    // and interact directly with any metastore.
+    RenameOperation renameOperation = new RenameOperation(
+        createStoreContext(),
+        src, srcKey, p.getLeft(),
+        dst, dstKey, p.getRight(),
+        new RenameOperationCallbacksImpl());
+    return renameOperation.executeRename();
+  }
 
-        if (keysToDelete.size() == MAX_ENTRIES_TO_DELETE) {
-          removeKeys(keysToDelete, true, false);
-        }
-      }
-      if (!keysToDelete.isEmpty()) {
-        removeKeys(keysToDelete, false, false);
-      }
+  /**
+   * All the callbacks made by the rename operation of the filesystem.
+   * This separation allows the operation to be factored out and
+   * still avoid knowledge of the S3AFilesystem implementation.
+   */
+  private class RenameOperationCallbacksImpl implements
+      RenameOperation.RenameOperationCallbacks {
 
-      // 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);
-      }
+    @Override
+    public S3ObjectAttributes createObjectAttributes(final Path path,
+        final String eTag,
+        final String versionId,
+        final long len) {
+      return S3AFileSystem.this.createObjectAttributes(path, eTag, versionId,
+          len);
     }
 
-    metadataStore.move(srcPaths, dstMetas, ttlTimeProvider);
+    @Override
+    public S3ObjectAttributes createObjectAttributes(final S3AFileStatus fileStatus) {
+      return S3AFileSystem.this.createObjectAttributes(fileStatus);
+    }
 
-    if (!src.getParent().equals(dst.getParent())) {
-      LOG.debug("source & dest parents are different; fix up dir markers");
-      deleteUnnecessaryFakeDirectories(dst.getParent());
-      maybeCreateFakeParentDirectory(src);
+    @Override
+    public S3AReadOpContext createReadContext(final FileStatus fileStatus) {
+      return S3AFileSystem.this.createReadContext(fileStatus,
+          inputPolicy,
+          changeDetectionPolicy, readAhead);
+    }
+
+    @Override
+    public void deleteObjectAtPath(final Path path,
+        final String key,
+        final boolean isFile)
+        throws IOException {
+      S3AFileSystem.this.deleteObjectAtPath(path, key, isFile);
+    }
+
+    @Override
+    @Retries.RetryTranslated
+    public RemoteIterator<S3ALocatedFileStatus> listFilesAndEmptyDirectories(
+        final Path path) throws IOException {
+      return S3AFileSystem.this.listFilesAndEmptyDirectories(path, true);
+    }
+
+    @Override
+    public CopyResult copyFile(final String srcKey,
+        final String destKey,
+        final S3ObjectAttributes srcAttributes,
+        final S3AReadOpContext readContext) throws IOException {
+      return S3AFileSystem.this.copyFile(srcKey, destKey,
+          srcAttributes.getLen(), srcAttributes, readContext);
+    }
+
+    @Override
+    public void removeKeys(final List<DeleteObjectsRequest.KeyVersion> keysToDelete,
+        final boolean deleteFakeDir,
+        final List<Path> undeletedObjectsOnFailure)
+        throws MultiObjectDeleteException, AmazonClientException, IOException {
+      S3AFileSystem.this.removeKeys(keysToDelete, deleteFakeDir,
+          undeletedObjectsOnFailure);
+    }
+
+    @Override
+    public void finishRename(final Path sourceRenamed, final Path destCreated)
+        throws IOException {
+      Path destParent = destCreated.getParent();
+      if (!sourceRenamed.getParent().equals(destParent)) {
+        LOG.debug("source & dest parents are different; fix up dir markers");
+        deleteUnnecessaryFakeDirectories(destParent);
+        maybeCreateFakeParentDirectory(sourceRenamed);
+      }
     }
-    return true;
   }
 
   /**
@@ -1380,6 +1409,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   public ObjectMetadata getObjectMetadata(Path path,
       ChangeTracker changeTracker, Invoker changeInvoker, String operation)
       throws IOException {
+    checkNotClosed();
     return once("getObjectMetadata", path.toString(),
         () ->
             // this always does a full HEAD to the object
@@ -1609,16 +1639,19 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     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()));
+            }
+          });
+    }
   }
 
   /**
@@ -1646,20 +1679,23 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       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()));
+            }
+          });
+    }
   }
 
   /**
@@ -1697,6 +1733,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       throws AmazonClientException, IOException {
     blockRootDelete(key);
     incrementWriteOperations();
+    LOG.debug("DELETE {}", key);
     invoker.retryUntranslated("Delete "+ bucket + ":/" + key,
         DELETE_CONSIDERED_IDEMPOTENT,
         ()-> {
@@ -1714,9 +1751,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * @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) {
@@ -1755,7 +1792,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   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,
           () -> {
@@ -1892,7 +1931,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       incrementPutCompletedStatistics(true, len);
       // update metadata
       finishedWrite(putObjectRequest.getKey(), len,
-          result.getETag(), result.getVersionId());
+          result.getETag(), result.getVersionId(), null);
       return result;
     } catch (AmazonClientException e) {
       incrementPutCompletedStatistics(false, len);
@@ -1993,23 +2032,23 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   }
 
   /**
-   * A helper method to delete a list of keys on a s3-backend.
+   * Delete a list of keys on a s3-backend.
+   * This does <i>not</i> 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.
    * @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<DeleteObjectsRequest.KeyVersion> keysToDelete,
-      boolean clearKeys, boolean deleteFakeDir)
+  private void removeKeysS3(List<DeleteObjectsRequest.KeyVersion> keysToDelete,
+      boolean deleteFakeDir)
       throws MultiObjectDeleteException, AmazonClientException,
       IOException {
     if (keysToDelete.isEmpty()) {
@@ -2019,22 +2058,118 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     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);
+  }
+
+  /**
+   * 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)} with handling of
+   * {@code MultiObjectDeleteException}.
+   *
+   * @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
+   * @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<DeleteObjectsRequest.KeyVersion> keysToDelete,
+      final boolean deleteFakeDir)
+      throws MultiObjectDeleteException, AmazonClientException,
+      IOException {
+    removeKeys(keysToDelete, deleteFakeDir, new ArrayList<>());
+  }
+
+  /**
+   * Invoke {@link #removeKeysS3(List, boolean)} with handling of
+   * {@code MultiObjectDeleteException} before the exception is rethrown.
+   * Specifically:
+   * <ol>
+   *   <li>Failure and !deleteFakeDir: S3Guard is updated with all
+   *    deleted entries</li>
+   *   <li>Failure where deleteFakeDir == true: do nothing with S3Guard</li>
+   *   <li>Success: do nothing with S3Guard</li>
+   * </ol>
+   * @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<DeleteObjectsRequest.KeyVersion> keysToDelete,
+      final boolean deleteFakeDir,
+      final List<Path> undeletedObjectsOnFailure)
+      throws MultiObjectDeleteException, AmazonClientException,
+      IOException {
+    undeletedObjectsOnFailure.clear();
+    try(DurationInfo ignored = new DurationInfo(LOG, false, "Deleting")) {
+      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.
+      if (!deleteFakeDir) {
+        // when deleting fake directories we don't want to delete metastore
+        // entries so we only process these failures on "real" deletes.
+        Triple<List<Path>, List<Path>, List<Pair<Path, IOException>>> results =
+            new MultiObjectDeleteSupport(createStoreContext())
+                .processDeleteFailure(ex, keysToDelete);
+        undeletedObjectsOnFailure.addAll(results.getMiddle());
+      }
+      throw ex;
+    } catch (AmazonClientException | IOException ex) {
+      List<Path> paths = new MultiObjectDeleteSupport(createStoreContext())
+          .processDeleteFailureGenericException(ex, keysToDelete);
+      // other failures. Assume nothing was deleted
+      undeletedObjectsOnFailure.addAll(paths);
+      throw ex;
     }
   }
 
@@ -2067,7 +2202,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       }
       return outcome;
     } catch (FileNotFoundException e) {
-      LOG.debug("Couldn't delete {} - does not exist", f);
+      LOG.debug("Couldn't delete {} - does not exist: {}", f, e.toString());
       instrumentation.errorIgnored();
       return false;
     } catch (AmazonClientException e) {
@@ -2131,22 +2266,25 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
             LOG.debug("Got object to delete {}", summary.getKey());
 
             if (keys.size() == MAX_ENTRIES_TO_DELETE) {
-              removeKeys(keys, true, false);
+              // delete a single page of keys
+              removeKeys(keys, false);
+              keys.clear();
             }
           }
 
           if (objects.isTruncated()) {
             objects = continueListObjects(request, objects);
           } else {
-            if (!keys.isEmpty()) {
-              // TODO: HADOOP-13761 S3Guard: retries
-              removeKeys(keys, false, false);
-            }
+            // there is no more data: delete the final set of entries.
+            removeKeys(keys, false);
             break;
           }
         }
       }
-      metadataStore.deleteSubtree(f, ttlTimeProvider);
+      try(DurationInfo ignored =
+              new DurationInfo(LOG, false, "Delete metastore")) {
+        metadataStore.deleteSubtree(f, ttlTimeProvider);
+      }
     } else {
       LOG.debug("delete: Path is a file");
       deleteObjectAtPath(f, key, true);
@@ -2276,6 +2414,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       while (files.hasNext()) {
         result.add(files.next());
       }
+      // merge the results. This will update the store as needed
       return S3Guard.dirListingUnion(metadataStore, path, result, dirMeta,
           allowAuthoritative, ttlTimeProvider);
     } else {
@@ -2464,6 +2603,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   S3AFileStatus innerGetFileStatus(final Path f,
       boolean needEmptyDirectoryFlag) throws IOException {
     entryPoint(INVOCATION_GET_FILE_STATUS);
+    checkNotClosed();
     final Path path = qualify(f);
     String key = pathToKey(path);
     LOG.debug("Getting path status for {}  ({})", path, key);
@@ -2476,8 +2616,11 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     Set<Path> 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);
       }
 
       // if ms is not authoritative, check S3 if there's any recent
@@ -2504,8 +2647,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
           final long s3ModTime = s3AFileStatus.getModificationTime();
 
           if(s3ModTime > msModTime) {
-            LOG.debug("S3Guard metadata for {} is outdated, updating it",
-                path);
+            LOG.debug("S3Guard metadata for {} is outdated;"
+                + " s3modtime={}; msModTime={} updating metastore",
+                path, s3ModTime, msModTime);
             return S3Guard.putAndReturn(metadataStore, s3AFileStatus,
                 instrumentation, ttlTimeProvider);
           }
@@ -2835,7 +2979,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     listener.uploadCompleted();
     // post-write actions
     finishedWrite(key, info.getLength(),
-        result.getETag(), result.getVersionId());
+        result.getETag(), result.getVersionId(), null);
     return result;
   }
 
@@ -3168,7 +3312,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   /**
    * Perform post-write actions.
    * Calls {@link #deleteUnnecessaryFakeDirectories(Path)} and then
-   * {@link S3Guard#addAncestors(MetadataStore, Path, String)}}.
+   * updates any metastore.
    * This operation MUST be called after any PUT/multipart PUT completes
    * successfully.
    *
@@ -3182,6 +3326,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * @param length  total length of file written
    * @param eTag eTag of the written object
    * @param versionId S3 object versionId of the written object
+   * @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
@@ -3189,22 +3334,39 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   @InterfaceAudience.Private
   @Retries.RetryTranslated("Except if failOnMetadataWriteError=false, in which"
       + " case RetryExceptionsSwallowed")
-  void finishedWrite(String key, long length, String eTag, String versionId)
+  void finishedWrite(String key, long length, String eTag, String versionId,
+      @Nullable final BulkOperationState operationState)
       throws MetadataPersistenceException {
-    LOG.debug("Finished write to {}, len {}", key, length);
+    LOG.debug("Finished write to {}, len {}. etag {}, version {}",
+        key, length, eTag, versionId);
     Path p = keyToQualifiedPath(key);
     Preconditions.checkArgument(length >= 0, "content length is negative");
     deleteUnnecessaryFakeDirectories(p.getParent());
+    // this is only set if there is a metastore to update and the
+    // operationState parameter passed in was null.
+    BulkOperationState stateToClose = null;
 
     // See note about failure semantics in S3Guard documentation
     try {
       if (hasMetadataStore()) {
-        S3Guard.addAncestors(metadataStore, p, username, ttlTimeProvider);
+        BulkOperationState activeState = operationState;
+        if (activeState == null) {
+          // create an operation state if there was none, so that the
+          // information gleaned from addAncestors is preserved into the
+          // subsequent put.
+          stateToClose = S3Guard.initiateBulkWrite(metadataStore,
+              BulkOperationState.OperationType.Put,
+              keyToPath(key));
+          activeState = stateToClose;
+        }
+        S3Guard.addAncestors(metadataStore, p, ttlTimeProvider, activeState);
         S3AFileStatus status = createUploadFileStatus(p,
             S3AUtils.objectRepresentsDirectory(key, length), length,
             getDefaultBlockSize(p), username, eTag, versionId);
-        S3Guard.putAndReturn(metadataStore, status, instrumentation,
-            ttlTimeProvider);
+        S3Guard.putAndReturn(metadataStore, status,
+            instrumentation,
+            ttlTimeProvider,
+            activeState);
       }
     } catch (IOException e) {
       if (failOnMetadataWriteError) {
@@ -3214,6 +3376,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
             p, e);
       }
       instrumentation.errorIgnored();
+    } finally {
+      // if a new operation state was created, close it.
+      IOUtils.cleanupWithLogger(LOG, stateToClose);
     }
   }
 
@@ -3233,7 +3398,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       path = path.getParent();
     }
     try {
-      removeKeys(keysToRemove, false, true);
+      removeKeys(keysToRemove, true);
     } catch(AmazonClientException | IOException e) {
       instrumentation.errorIgnored();
       if (LOG.isDebugEnabled()) {
@@ -3594,11 +3759,17 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     };
   }
 
-  @Retries.OnceTranslated
+  /**
+   * Recursive List of files and empty directories.
+   * @param f path to list from
+   * @return an iterator.
+   * @throws IOException failure
+   */
+  @Retries.RetryTranslated
   public RemoteIterator<S3ALocatedFileStatus> listFilesAndEmptyDirectories(
       Path f, boolean recursive) throws IOException {
-    return innerListFiles(f, recursive,
-        new Listing.AcceptAllButS3nDirs());
+    return invoker.retry("list", f.toString(), true,
+        () -> innerListFiles(f, recursive, new Listing.AcceptAllButS3nDirs()));
   }
 
   @Retries.OnceTranslated
@@ -3904,8 +4075,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
 
     // readahead range can be dynamically set
     long ra = options.getLong(READAHEAD_RANGE, readAhead);
-    S3ObjectAttributes objectAttributes = createObjectAttributes(
-        path, fileStatus.getETag(), fileStatus.getVersionId());
+    S3ObjectAttributes objectAttributes = createObjectAttributes(fileStatus);
     S3AReadOpContext readContext = createReadContext(fileStatus, inputPolicy,
         changeDetectionPolicy, ra);
 
@@ -3998,4 +4168,59 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     return result;
   }
 
+  /**
+   * 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.
+   */
+  @InterfaceAudience.Private
+  public StoreContext createStoreContext() {
+    return new StoreContext(
+        getUri(),
+        getBucket(),
+        getConf(),
+        getUsername(),
+        owner,
+        boundedThreadPool,
+        executorCapacity,
+        invoker,
+        getInstrumentation(),
+        getStorageStatistics(),
+        getInputPolicy(),
+        changeDetectionPolicy,
+        enableMultiObjectsDelete,
+        metadataStore,
+        useListV1,
+        new ContextAccessorsImpl(),
+        getTtlTimeProvider());
+  }
+
+  /**
+   * The implementation of context accessors.
+   */
+  private class ContextAccessorsImpl implements ContextAccessors {
+
+    @Override
+    public Path keyToPath(final String key) {
+      return keyToQualifiedPath(key);
+    }
+
+    @Override
+    public String pathToKey(final Path path) {
+      return S3AFileSystem.this.pathToKey(path);
+    }
+
+    @Override
+    public File createTempFile(final String prefix, final long size)
+        throws IOException {
+      return createTmpFileForWrite(prefix, size, getConf());
+    }
+
+    @Override
+    public String getBucketLocation() throws IOException {
+      return S3AFileSystem.this.getBucketLocation();
+    }
+  }
 }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
index 6022126..c92a85e 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
@@ -128,23 +128,22 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
    * This does not attempt to open it; that is only done on the first
    * actual read() operation.
    * @param ctx operation context
-   * @param s3Attributes object attributes from a HEAD request
-   * @param contentLength length of content
+   * @param s3Attributes object attributes
    * @param client S3 client to use
    */
   public S3AInputStream(S3AReadOpContext ctx,
       S3ObjectAttributes s3Attributes,
-      long contentLength,
       AmazonS3 client) {
     Preconditions.checkArgument(isNotEmpty(s3Attributes.getBucket()),
         "No Bucket");
     Preconditions.checkArgument(isNotEmpty(s3Attributes.getKey()), "No Key");
-    Preconditions.checkArgument(contentLength >= 0, "Negative content length");
+    long l = s3Attributes.getLen();
+    Preconditions.checkArgument(l >= 0, "Negative content length");
     this.context = ctx;
     this.bucket = s3Attributes.getBucket();
     this.key = s3Attributes.getKey();
     this.pathStr = ctx.dstFileStatus.getPath().toString();
-    this.contentLength = contentLength;
+    this.contentLength = l;
     this.client = client;
     this.uri = "s3a://" + this.bucket + "/" + this.key;
     this.streamStatistics = ctx.instrumentation.newInputStreamStatistics();
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 fd7893f..e9ed972 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,10 +183,14 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
       COMMITTER_MAGIC_FILES_CREATED,
       S3GUARD_METADATASTORE_PUT_PATH_REQUEST,
       S3GUARD_METADATASTORE_INITIALIZATION,
+      S3GUARD_METADATASTORE_RECORD_DELETES,
+      S3GUARD_METADATASTORE_RECORD_READS,
+      S3GUARD_METADATASTORE_RECORD_WRITES,
       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 = {
@@ -1144,6 +1148,31 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
     public void retrying() {
       // counters are incremented by owner.
     }
+
+    /**
+     * Records have been read.
+     * @param count the number of records read
+     */
+    public void recordsDeleted(int count) {
+      incrementCounter(S3GUARD_METADATASTORE_RECORD_DELETES, count);
+    }
+
+    /**
+     * 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/S3AUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
index fe4fd0f..7e30fa6 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
@@ -84,6 +84,7 @@ import java.util.concurrent.ExecutionException;
 
 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.translateDeleteException;
 
 /**
  * Utility methods for S3A code.
@@ -284,7 +285,7 @@ public final class S3AUtils {
       case 200:
         if (exception instanceof MultiObjectDeleteException) {
           // failure during a bulk delete
-          return translateMultiObjectDeleteException(message,
+          return translateDeleteException(message,
               (MultiObjectDeleteException) exception);
         }
         // other 200: FALL THROUGH
@@ -449,40 +450,6 @@ public final class S3AUtils {
   }
 
   /**
-   * 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<String> 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
    * @return string details
@@ -1004,7 +971,7 @@ public final class S3AUtils {
    * @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",
@@ -1022,7 +989,7 @@ public final class S3AUtils {
    * @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) {
@@ -1384,7 +1351,7 @@ public final class S3AUtils {
    * @throws IOException anything in the closure, or iteration logic.
    */
   public static long applyLocatedFiles(
-      RemoteIterator<LocatedFileStatus> iterator,
+      RemoteIterator<? extends LocatedFileStatus> iterator,
       CallOnLocatedFileStatus eval) throws IOException {
     long count = 0;
     while (iterator.hasNext()) {
@@ -1404,7 +1371,7 @@ public final class S3AUtils {
    * @throws IOException anything in the closure, or iteration logic.
    */
   public static <T> List<T> mapLocatedFiles(
-      RemoteIterator<LocatedFileStatus> iterator,
+      RemoteIterator<? extends LocatedFileStatus> iterator,
       LocatedFileStatusMap<T> eval) throws IOException {
     final List<T> results = new ArrayList<>();
     applyLocatedFiles(iterator,
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectAttributes.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectAttributes.java
index 2e62ff6..5a8dfc7 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectAttributes.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectAttributes.java
@@ -18,38 +18,74 @@
 
 package org.apache.hadoop.fs.s3a;
 
+import com.amazonaws.services.s3.transfer.model.CopyResult;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.Path;
 
 /**
- * This class is only a holder for bucket, key, SSE Algorithm and SSE key
- * attributes. It is used in {@link S3AInputStream} and the select equivalent.
+ * This class holds attributed of an object independent of the
+ * file status type.
+ * It is used in {@link S3AInputStream} and the select equivalent.
  * as a way to reduce parameters being passed
- * to the constructor of such class.
+ * to the constructor of such class,
+ * and elsewhere to be a source-neutral representation of a file status.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public class S3ObjectAttributes {
   private final String bucket;
+  private final Path path;
   private final String key;
   private final S3AEncryptionMethods serverSideEncryptionAlgorithm;
   private final String serverSideEncryptionKey;
   private final String eTag;
   private final String versionId;
+  private final long len;
 
   public S3ObjectAttributes(
       String bucket,
+      Path path,
       String key,
       S3AEncryptionMethods serverSideEncryptionAlgorithm,
       String serverSideEncryptionKey,
       String eTag,
-      String versionId) {
+      String versionId,
+      long len) {
     this.bucket = bucket;
+    this.path = path;
     this.key = key;
     this.serverSideEncryptionAlgorithm = serverSideEncryptionAlgorithm;
     this.serverSideEncryptionKey = serverSideEncryptionKey;
     this.eTag = eTag;
     this.versionId = versionId;
+    this.len = len;
+  }
+
+  /**
+   * Construct from the result of a copy and those parameters
+   * which aren't included in an AWS SDK response.
+   * @param path
+   * @param copyResult copy result.
+   * @param serverSideEncryptionAlgorithm current encryption algorithm
+   * @param serverSideEncryptionKey any server side encryption key?
+   * @param len
+   */
+  public S3ObjectAttributes(
+      final Path path,
+      final CopyResult copyResult,
+      final S3AEncryptionMethods serverSideEncryptionAlgorithm,
+      final String serverSideEncryptionKey,
+      final long len) {
+    this.bucket = copyResult.getDestinationBucketName();
+    this.key = copyResult.getDestinationKey();
+    this.path = path;
+    this.serverSideEncryptionAlgorithm = serverSideEncryptionAlgorithm;
+    this.serverSideEncryptionKey = serverSideEncryptionKey;
+    this.eTag = copyResult.getETag();
+    this.versionId = copyResult.getVersionId();
+    this.len = len;
   }
 
   public String getBucket() {
@@ -75,4 +111,12 @@ public class S3ObjectAttributes {
   public String getVersionId() {
     return versionId;
   }
+
+  public long getLen() {
+    return len;
+  }
+
+  public Path getPath() {
+    return path;
+  }
 }
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 54a2c60..42322bc 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",
@@ -207,6 +209,15 @@ 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_DELETES(
+      "s3guard_metadatastore_record_deletes",
+      "S3Guard metadata store records deleted"),
+  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/WriteOperationHelper.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java
index 54386ad..8cdce7b 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;
@@ -41,13 +42,15 @@ import com.amazonaws.services.s3.model.UploadPartRequest;
 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.S3Guard;
 import org.apache.hadoop.fs.s3a.select.SelectBinding;
 import org.apache.hadoop.util.DurationInfo;
 
@@ -226,7 +229,7 @@ public class WriteOperationHelper {
   /**
    * Finalize a multipart PUT operation.
    * This completes the upload, and, if that works, calls
-   * {@link S3AFileSystem#finishedWrite(String, long, String, String)}
+   * {@link S3AFileSystem#finishedWrite(String, long, String, String, BulkOperationState)}
    * to update the filesystem.
    * Retry policy: retrying, translated.
    * @param destKey destination of the commit
@@ -234,6 +237,7 @@ public class WriteOperationHelper {
    * @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.
    */
@@ -243,7 +247,8 @@ public class WriteOperationHelper {
       String uploadId,
       List<PartETag> 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);
@@ -263,7 +268,7 @@ public class WriteOperationHelper {
             }
     );
     owner.finishedWrite(destKey, length, uploadResult.getETag(),
-        uploadResult.getVersionId());
+        uploadResult.getVersionId(), operationState);
     return uploadResult;
   }
 
@@ -298,7 +303,8 @@ public class WriteOperationHelper {
         uploadId,
         partETags,
         length,
-        (text, e, r, i) -> errorCount.incrementAndGet());
+        (text, e, r, i) -> errorCount.incrementAndGet(),
+        null);
   }
 
   /**
@@ -490,6 +496,52 @@ public class WriteOperationHelper {
   }
 
   /**
+   * 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 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<PartETag> 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(),
+        BulkOperationState.OperationType.Put, path);
+  }
+
+  /**
    * Upload part of a multi-partition file.
    * @param request request
    * @return the result of the operation.
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 ed608cb..a49ab52 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 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
     }
     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 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
     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<MultipartUpload> pending;
       try {
@@ -544,7 +559,8 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
       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 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
       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 55ace17..7bf263e 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,9 @@ import org.apache.hadoop.fs.s3a.WriteOperationHelper;
 import org.apache.hadoop.fs.s3a.commit.files.PendingSet;
 import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit;
 import org.apache.hadoop.fs.s3a.commit.files.SuccessData;
+import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.DurationInfo;
 
 import static org.apache.hadoop.fs.s3a.S3AUtils.*;
 import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
@@ -128,10 +132,13 @@ public class CommitOperations {
   /**
    * 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(
+      final SinglePendingCommit commit,
+      final BulkOperationState operationState) throws IOException {
+    commit(commit, commit.getFilename(), operationState).maybeRethrow();
   }
 
   /**
@@ -139,16 +146,20 @@ public class CommitOperations {
    * 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 +182,20 @@ public class CommitOperations {
   /**
    * 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 +263,7 @@ public class CommitOperations {
    * @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 +282,7 @@ public class CommitOperations {
    * @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);
@@ -392,7 +406,10 @@ public class CommitOperations {
     Path markerPath = new Path(outputPath, _SUCCESS);
     LOG.debug("Touching success marker for job {}: {}", markerPath,
         successData);
-    successData.save(fs, markerPath, true);
+    try (DurationInfo ignored = new DurationInfo(LOG,
+        "Writing success file %s", markerPath)) {
+      successData.save(fs, markerPath, true);
+    }
   }
 
   /**
@@ -401,7 +418,7 @@ public class CommitOperations {
    * @throws IOException failure
    */
   public void revertCommit(SinglePendingCommit commit) throws IOException {
-    LOG.warn("Revert {}", commit);
+    LOG.info("Revert {}", commit);
     try {
       writeOperations.revertCommit(commit.getDestinationKey());
     } finally {
@@ -521,6 +538,120 @@ public class CommitOperations {
   }
 
   /**
+   * 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 {
+
+    /**
+     * State of any metastore.
+     */
+    private final BulkOperationState operationState;
+
+    /**
+     * Create.
+     * @param operationState any S3Guard bulk state.
+     */
+    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);
+    }
+
+    @Override
+    public String toString() {
+      final StringBuilder sb = new StringBuilder(
+          "CommitContext{");
+      sb.append("operationState=").append(operationState);
+      sb.append('}');
+      return sb.toString();
+    }
+
+  }
+
+  /**
    * A holder for a possible IOException; the call {@link #maybeRethrow()}
    * will throw any exception passed into the constructor, and be a no-op
    * if none was.
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java
index 813b9a7..969286e 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java
@@ -123,8 +123,11 @@ public class MagicS3GuardCommitter extends AbstractS3ACommitter {
    */
   public void cleanupStagingDirs() {
     Path path = magicSubdir(getOutputPath());
-    Invoker.ignoreIOExceptions(LOG, "cleanup magic directory", path.toString(),
-        () -> deleteWithWarning(getDestFS(), path, true));
+    try(DurationInfo ignored = new DurationInfo(LOG, true,
+        "Deleting magic directory %s", path)) {
+      Invoker.ignoreIOExceptions(LOG, "cleanup magic directory", path.toString(),
+          () -> deleteWithWarning(getDestFS(), path, true));
+    }
   }
 
   @Override
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 f26384d..518d789 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.RemoteIterator;
 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 @@ public class StagingCommitter extends AbstractS3ACommitter {
           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/impl/AbstractStoreOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AbstractStoreOperation.java
new file mode 100644
index 0000000..904d5f7
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AbstractStoreOperation.java
@@ -0,0 +1,49 @@
+/*
+ * 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 static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * Base class of operations in the store.
+ * An operation is something which executes against the context to
+ * perform a single function.
+ * It is expected to have a limited lifespan.
+ */
+public abstract class AbstractStoreOperation {
+
+  private final StoreContext storeContext;
+
+  /**
+   * constructor.
+   * @param storeContext store context.
+   */
+  protected AbstractStoreOperation(final StoreContext storeContext) {
+    this.storeContext = checkNotNull(storeContext);
+  }
+
+  /**
+   * Get the store context.
+   * @return the context.
+   */
+  public final StoreContext getStoreContext() {
+    return storeContext;
+  }
+
+}
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 0000000..609eece
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CallableSupplier.java
@@ -0,0 +1,126 @@
+/*
+ * 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.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
+ * raised by the callable and wrapping them as appropriate.
+ * @param <T> return type.
+ */
+public final class CallableSupplier<T> implements Supplier {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(CallableSupplier.class);
+
+  private final Callable<T> call;
+
+  /**
+   * Create.
+   * @param call call to invoke.
+   */
+  public CallableSupplier(final Callable<T> 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 <T> type
+   * @return the future to wait for
+   */
+  @SuppressWarnings("unchecked")
+  public static <T> CompletableFuture<T> submit(
+      final Executor executor,
+      final Callable<T> call) {
+    return CompletableFuture.supplyAsync(
+        new CallableSupplier<T>(call), executor);
+  }
+
+  /**
+   * Wait for a list of futures to complete. If the list is empty,
+   * return immediately.
+   * @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 <T> void waitForCompletion(
+      final List<CompletableFuture<T>> futures)
+      throws IOException {
+    if (futures.isEmpty()) {
+      return;
+    }
+    // await completion
+    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 <T> void waitForCompletion(
+      final CompletableFuture<T> future)
+      throws IOException {
+    try (DurationInfo ignore =
+            new DurationInfo(LOG, false, "Waiting for task completion")) {
+      future.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/impl/ContextAccessors.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ContextAccessors.java
new file mode 100644
index 0000000..1ca3a42
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ContextAccessors.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.impl;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.Retries;
+
+/**
+ * An interface to implement for providing accessors to
+ * S3AFileSystem-level API calls.
+ * <p>
+ * This is used to avoid giving any explicit reference to the owning
+ * FS in the store context; there are enough calls that using lambda-expressions
+ * gets over-complex.
+ * <ol>
+ *   <li>Test suites are free to provide their own implementation, using
+ *  * the S3AFileSystem methods as the normative reference.</li>
+ *  <li>All implementations <i>MUST</i> translate exceptions.</li>
+ * </ol>
+ */
+public interface ContextAccessors {
+
+  /**
+   * Convert a key to a fully qualified path.
+   * @param key input key
+   * @return the fully qualified path including URI scheme and bucket name.
+   */
+  Path keyToPath(String key);
+
+  /**
+   * Turns a path (relative or otherwise) into an S3 key.
+   *
+   * @param path input path, may be relative to the working dir
+   * @return a key excluding the leading "/", or, if it is the root path, ""
+   */
+  String pathToKey(Path path);
+
+  /**
+   * Create a temporary file.
+   * @param prefix prefix for the temporary file
+   * @param size the size of the file that is going to be written
+   * @return a unique temporary file
+   * @throws IOException IO problems
+   */
+  File createTempFile(String prefix, long size) throws IOException;
+
+  /**
+   * Get the region of a bucket. This may be via an S3 API call if not
+   * already cached.
+   * @return the region in which a bucket is located
+   * @throws IOException on any failure.
+   */
+  @Retries.RetryTranslated
+  String getBucketLocation() throws IOException;
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java
new file mode 100644
index 0000000..1b2a430
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java
@@ -0,0 +1,60 @@
+/*
+ * 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;
+
+/**
+ * Internal constants private only to the S3A codebase.
+ */
+public final class InternalConstants {
+
+  private InternalConstants() {
+  }
+
+  /**
+   * 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;
+
+  /**
+   * The maximum number of entries that can be deleted in any bulk delete
+   * call to S3: {@value}.
+   */
+  public static final int MAX_ENTRIES_TO_DELETE = 1000;
+
+  /**
+   * Default blocksize as used in blocksize and FS status queries: {@value}.
+   */
+  public static final int DEFAULT_BLOCKSIZE = 32 * 1024 * 1024;
+}
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 0000000..fce1780
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteSupport.java
@@ -0,0 +1,244 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+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 extends AbstractStoreOperation {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      MultiObjectDeleteSupport.class);
+
+  /**
+   * Initiate with a store context.
+   * @param context store context.
+   */
+  public MultiObjectDeleteSupport(final StoreContext context) {
+    super(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 translateDeleteException(
+      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);
+    }
+  }
+
+  /**
+   * 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<Path>, List<Path>> splitUndeletedKeys(
+      final MultiObjectDeleteException deleteException,
+      final Collection<DeleteObjectsRequest.KeyVersion> keysToDelete) {
+    LOG.debug("Processing delete failure; keys to delete count = {};"
+            + " errors in exception {}; successful deletions = {}",
+        keysToDelete.size(),
+        deleteException.getErrors().size(),
+        deleteException.getDeletedObjects().size());
+    // convert the collection of keys being deleted into paths
+    final List<Path> 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<Path> undeleted = removeUndeletedPaths(deleteException,
+        pathsBeingDeleted,
+        getStoreContext()::keyToPath);
+    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<Path> keysToPaths(
+      final Collection<DeleteObjectsRequest.KeyVersion> keysToDelete) {
+    return convertToPaths(keysToDelete,
+        getStoreContext()::keyToPath);
+  }
+
+  /**
+   * 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<Path> convertToPaths(
+      final Collection<DeleteObjectsRequest.KeyVersion> keysToDelete,
+      final Function<String, Path> 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
+   * 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.
+   * @return a tuple of (undeleted, deleted, failures)
+   */
+  public Triple<List<Path>, List<Path>, List<Pair<Path, IOException>>>
+      processDeleteFailure(
+      final MultiObjectDeleteException deleteException,
+      final List<DeleteObjectsRequest.KeyVersion> keysToDelete) {
+    final MetadataStore metadataStore =
+        checkNotNull(getStoreContext().getMetadataStore(),
+            "context metadatastore");
+    final List<Pair<Path, IOException>> failures = new ArrayList<>();
+    final Pair<List<Path>, List<Path>> outcome =
+        splitUndeletedKeys(deleteException, keysToDelete);
+    List<Path> deleted = outcome.getRight();
+    List<Path> undeleted = outcome.getLeft();
+    // delete the paths but recover
+    // TODO: handle the case where a parent path is deleted but not a child.
+    // TODO: in a fake object delete, we don't actually want to delete
+    //  metastore entries
+    deleted.forEach(path -> {
+      try {
+        metadataStore.delete(path, getStoreContext().getTimeProvider());
+      } 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);
+  }
+
+  /**
+   * 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.
+   */
+  @VisibleForTesting
+  public static List<Path> extractUndeletedPaths(
+      final MultiObjectDeleteException deleteException,
+      final Function<String, Path> 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
+   */
+  @VisibleForTesting
+  static List<Path> removeUndeletedPaths(
+      final MultiObjectDeleteException deleteException,
+      final Collection<Path> pathsBeingDeleted,
+      final Function<String, Path> qualifier) {
+    List<Path> undeleted = extractUndeletedPaths(deleteException, qualifier);
+    pathsBeingDeleted.removeAll(undeleted);
+    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<Path> processDeleteFailureGenericException(Exception ex,
+      final List<DeleteObjectsRequest.KeyVersion> keysToDelete) {
+    return keysToPaths(keysToDelete);
+  }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java
new file mode 100644
index 0000000..04784fb
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java
@@ -0,0 +1,634 @@
+/*
+ * 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.io.InterruptedIOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import com.amazonaws.AmazonClientException;
+import com.amazonaws.services.s3.model.DeleteObjectsRequest;
+import com.amazonaws.services.s3.model.MultiObjectDeleteException;
+import com.amazonaws.services.s3.transfer.model.CopyResult;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.InvalidRequestException;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.s3a.RenameFailedException;
+import org.apache.hadoop.fs.s3a.Retries;
+import org.apache.hadoop.fs.s3a.S3AFileStatus;
+import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus;
+import org.apache.hadoop.fs.s3a.S3AReadOpContext;
+import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
+import org.apache.hadoop.fs.s3a.Tristate;
+import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
+import org.apache.hadoop.fs.s3a.s3guard.RenameTracker;
+import org.apache.hadoop.util.DurationInfo;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.hadoop.fs.s3a.Constants.FS_S3A_BLOCK_SIZE;
+import static org.apache.hadoop.fs.s3a.S3AUtils.objectRepresentsDirectory;
+import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit;
+import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletion;
+import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DEFAULT_BLOCKSIZE;
+import static org.apache.hadoop.fs.s3a.impl.InternalConstants.MAX_ENTRIES_TO_DELETE;
+import static org.apache.hadoop.fs.s3a.impl.InternalConstants.RENAME_PARALLEL_LIMIT;
+
+/**
+ * A parallelized rename operation which updates the metastore in the
+ * process, through whichever {@link RenameTracker} the store provides.
+ * The parallel execution is in groups of size
+ * {@link InternalConstants#RENAME_PARALLEL_LIMIT}; it is only
+ * after one group completes that the next group is initiated.
+ * Once enough files have been copied that they meet the
+ * {@link InternalConstants#MAX_ENTRIES_TO_DELETE} threshold, a delete
+ * is initiated.
+ * If it succeeds, the rename continues with the next group of files.
+ *
+ * The RenameTracker has the task of keeping the metastore up to date
+ * as the rename proceeds.
+ *
+ * The rename operation implements the classic HDFS rename policy of
+ * rename(file, dir) renames the file under the directory.
+ *
+ * There is <i>no</i> validation of input and output paths.
+ * Callers are required to themselves verify that destination is not under
+ * the source, above the source, the source itself, etc, etc.
+ */
+public class RenameOperation extends AbstractStoreOperation {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      RenameOperation.class);
+
+  /**
+   * Used to stop any re-entrancy of the rename.
+   * This is an execute-once operation.
+   */
+  private final AtomicBoolean executed = new AtomicBoolean(false);
+
+  private final Path sourcePath;
+
+  private final String sourceKey;
+
+  private final S3AFileStatus sourceStatus;
+
+  private final Path destPath;
+
+  private final String destKey;
+
+  private final S3AFileStatus destStatus;
+
+  /**
+   * Callbacks into the filesystem.
+   */
+  private final RenameOperationCallbacks callbacks;
+
+  /**
+   * Counter of bytes copied.
+   */
+  private final AtomicLong bytesCopied = new AtomicLong();
+
+  /**
+   * Rename tracker.
+   */
+  private RenameTracker renameTracker;
+
+  /**
+   * List of active copies.
+   */
+  private final List<CompletableFuture<Path>> activeCopies =
+      new ArrayList<>(RENAME_PARALLEL_LIMIT);
+
+  /**
+   * list of keys to delete on the next (bulk) delete call.
+   */
+  private final List<DeleteObjectsRequest.KeyVersion> keysToDelete =
+      new ArrayList<>();
+
+  /**
+   * List of paths to delete, which will be passed to the rename
+   * tracker after the deletion succeeds.
+   */
+  private final List<Path> pathsToDelete = new ArrayList<>();
+
+  private final long blocksize;
+
+  /**
+   * Initiate the rename.
+   *
+   * @param storeContext store context
+   * @param sourcePath source path
+   * @param sourceKey key of source
+   * @param sourceStatus pre-fetched source status
+   * @param destPath destination path.
+   * @param destKey destination key
+   * @param destStatus destination status.
+   * @param callbacks callback provider
+   */
+  public RenameOperation(
+      final StoreContext storeContext,
+      final Path sourcePath,
+      final String sourceKey,
+      final S3AFileStatus sourceStatus,
+      final Path destPath,
+      final String destKey,
+      final S3AFileStatus destStatus,
+      final RenameOperationCallbacks callbacks) {
+    super(storeContext);
+    this.sourcePath = sourcePath;
+    this.sourceKey = sourceKey;
+    this.sourceStatus = sourceStatus;
+    this.destPath = destPath;
+    this.destKey = destKey;
+    this.destStatus = destStatus;
+    this.callbacks = callbacks;
+    blocksize = storeContext.getConfiguration()
+        .getLongBytes(FS_S3A_BLOCK_SIZE, DEFAULT_BLOCKSIZE);
+  }
+
+  /**
+   * Wait for the active copies to complete then reset the list.
+   * @param reason for messages
+   */
+  private void completeActiveCopies(String reason) throws IOException {
+    LOG.debug("Waiting for {} active copies to complete: {}",
+        activeCopies.size(), reason);
+    waitForCompletion(activeCopies);
+    activeCopies.clear();
+  }
+
+  /**
+   * Queue and object for deletion.
+   * @param path path to the object
+   * @param key key of the object.
+   */
+  private void queueToDelete(Path path, String key) {
+    pathsToDelete.add(path);
+    keysToDelete.add(new DeleteObjectsRequest.KeyVersion(key));
+  }
+
+  /**
+   * Block waiting for ay active copies to finish
+   * then delete all queued keys + paths to delete.
+   * @param reason reason for logs
+   * @throws IOException failure.
+   */
+  private void completeActiveCopiesAndDeleteSources(String reason)
+      throws IOException {
+    completeActiveCopies(reason);
+    removeSourceObjects(
+        keysToDelete,
+        pathsToDelete);
+    // now reset the lists.
+    keysToDelete.clear();
+    pathsToDelete.clear();
+  }
+
+  @Retries.RetryMixed
+  public long executeRename() throws IOException {
+    Preconditions.checkState(
+        !executed.getAndSet(true),
+        "Rename attempted twice");
+    final StoreContext storeContext = getStoreContext();
+    final MetadataStore metadataStore = checkNotNull(
+        storeContext.getMetadataStore(),
+        "No metadata store in context");
+
+    // Validation completed: time to begin the operation.
+    // The store-specific rename tracker is used to keep the store
+    // to date with the in-progress operation.
+    // for the null store, these are all no-ops.
+    renameTracker = metadataStore.initiateRenameOperation(
+        storeContext,
+        sourcePath, sourceStatus, destPath);
+
+
+    // Ok! Time to start
+    try {
+      if (sourceStatus.isFile()) {
+        renameFileToDest();
+      } else {
+        recursiveDirectoryRename();
+      }
+    } catch (AmazonClientException | IOException ex) {
+      // rename failed.
+      // block for all ongoing copies to complete, successfully or not
+      try {
+        completeActiveCopies("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);
+      }
+      // notify the rename tracker of the failure
+      throw renameTracker.renameFailed(ex);
+    }
+
+    // At this point the rename has completed successfully in the S3 store.
+    // Tell the metastore this fact and let it complete its changes
+    renameTracker.completeRename();
+
+    callbacks.finishRename(sourcePath, destPath);
+    return bytesCopied.get();
+  }
+
+  /**
+   * The source is a file: rename it to the destination.
+   * @throws IOException failure
+   */
+  protected void renameFileToDest() throws IOException {
+    final StoreContext storeContext = getStoreContext();
+    // the source is a file.
+    Path copyDestinationPath = destPath;
+    String copyDestinationKey = destKey;
+    S3ObjectAttributes sourceAttributes =
+        callbacks.createObjectAttributes(sourceStatus);
+    S3AReadOpContext readContext = callbacks.createReadContext(sourceStatus);
+    if (destStatus != null && destStatus.isDirectory()) {
+      // destination is a directory: build the final destination underneath
+      String newDestKey = maybeAddTrailingSlash(destKey);
+      String filename = sourceKey.substring(
+          storeContext.pathToKey(sourcePath.getParent()).length() + 1);
+      newDestKey = newDestKey + filename;
+      copyDestinationKey = newDestKey;
+      copyDestinationPath = storeContext.keyToPath(newDestKey);
+    }
+    // destination either does not exist or is a file to overwrite.
+    LOG.debug("rename: renaming file {} to {}", sourcePath,
+        copyDestinationPath);
+    copySourceAndUpdateTracker(
+        sourcePath,
+        sourceKey,
+        sourceAttributes,
+        readContext,
+        copyDestinationPath,
+        copyDestinationKey,
+        false);
+    bytesCopied.addAndGet(sourceStatus.getLen());
+    // delete the source
+    callbacks.deleteObjectAtPath(sourcePath, sourceKey, true);
+    // and update the tracker
+    renameTracker.sourceObjectsDeleted(Lists.newArrayList(sourcePath));
+  }
+
+  /**
+   * Execute a full recursive rename.
+   * The source is a file: rename it to the destination.
+   * @throws IOException failure
+   */
+  protected void recursiveDirectoryRename() throws IOException {
+    final StoreContext storeContext = getStoreContext();
+
+    LOG.debug("rename: renaming directory {} to {}", sourcePath, destPath);
+
+    // This is a directory-to-directory copy
+    String dstKey = maybeAddTrailingSlash(destKey);
+    String srcKey = maybeAddTrailingSlash(sourceKey);
+
+    // 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 ");
+    }
+
+    if (destStatus != null
+        && destStatus.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 and insert a tombstone
+      // marker.
+      LOG.debug("Deleting fake directory marker at destination {}",
+          destStatus.getPath());
+      callbacks.deleteObjectAtPath(destStatus.getPath(), dstKey, false);
+    }
+
+    Path parentPath = storeContext.keyToPath(srcKey);
+    final RemoteIterator<S3ALocatedFileStatus> iterator =
+        callbacks.listFilesAndEmptyDirectories(parentPath);
+    while (iterator.hasNext()) {
+      // get the next entry in the listing.
+      S3ALocatedFileStatus child = iterator.next();
+      // convert it to an S3 key.
+      String k = storeContext.pathToKey(child.getPath());
+      // possibly adding a "/" if it represents directory and it does
+      // not have a trailing slash already.
+      String key = (child.isDirectory() && !k.endsWith("/"))
+          ? k + "/"
+          : k;
+      // the source object to copy as a path.
+      Path childSourcePath = storeContext.keyToPath(key);
+
+      // mark for deletion on a successful copy.
+      queueToDelete(childSourcePath, key);
+
+      // the destination key is that of the key under the source tree,
+      // remapped under the new destination path.
+      String newDestKey =
+          dstKey + key.substring(srcKey.length());
+      Path childDestPath = storeContext.keyToPath(newDestKey);
+
+      // now begin the single copy
+      CompletableFuture<Path> copy = initiateCopy(child, key,
+          childSourcePath, newDestKey, childDestPath);
+      activeCopies.add(copy);
+      bytesCopied.addAndGet(sourceStatus.getLen());
+
+      if (activeCopies.size() == RENAME_PARALLEL_LIMIT) {
+        // the limit of active copies has been reached;
+        // wait for completion or errors to surface.
+        LOG.debug("Waiting for active copies to complete");
+        completeActiveCopies("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("paged delete");
+      }
+    } // end of iteration through the list
+
+    // await the final set of copies and their deletion
+    // This will notify the renameTracker that these objects
+    // have been deleted.
+    completeActiveCopiesAndDeleteSources("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.moveSourceDirectory();
+  }
+
+  /**
+   * Initiate a copy operation in the executor.
+   * @param source status of the source object.
+   * @param key source key
+   * @param childSourcePath source as a path.
+   * @param newDestKey destination key
+   * @param childDestPath destination path.
+   * @return the future.
+   */
+  protected CompletableFuture<Path> initiateCopy(
+      final S3ALocatedFileStatus source,
+      final String key,
+      final Path childSourcePath,
+      final String newDestKey,
+      final Path childDestPath) {
+    S3ObjectAttributes sourceAttributes =
+        callbacks.createObjectAttributes(
+            source.getPath(),
+            source.getETag(),
+            source.getVersionId(),
+            source.getLen());
+    // queue the copy operation for execution in the thread pool
+    return submit(getStoreContext().getExecutor(), () ->
+        copySourceAndUpdateTracker(
+            childSourcePath,
+            key,
+            sourceAttributes,
+            callbacks.createReadContext(source),
+            childDestPath,
+            newDestKey,
+            true));
+  }
+
+  /**
+   * 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 sourceFile source path of the copy; may have a trailing / on it.
+   * @param srcKey source key
+   * @param srcAttributes status of the source object
+   * @param destination destination as a qualified path.
+   * @param destinationKey destination key
+   * @param addAncestors should ancestors be added to the metastore?
+   * @return the destination path.
+   * @throws IOException failure
+   */
+  @Retries.RetryTranslated
+  private Path copySourceAndUpdateTracker(
+      final Path sourceFile,
+      final String srcKey,
+      final S3ObjectAttributes srcAttributes,
+      final S3AReadOpContext readContext,
+      final Path destination,
+      final String destinationKey,
+      final boolean addAncestors) throws IOException {
+    long len = srcAttributes.getLen();
+    CopyResult copyResult;
+    try (DurationInfo ignored = new DurationInfo(LOG, false,
+        "Copy file from %s to %s (length=%d)", srcKey, destinationKey, len)) {
+      copyResult = callbacks.copyFile(srcKey, destinationKey,
+          srcAttributes, readContext);
+    }
+    if (objectRepresentsDirectory(srcKey, len)) {
+      renameTracker.directoryMarkerCopied(
+          sourceFile,
+          destination,
+          addAncestors);
+    } else {
+      S3ObjectAttributes destAttributes = new S3ObjectAttributes(
+          destination,
+          copyResult,
+          srcAttributes.getServerSideEncryptionAlgorithm(),
+          srcAttributes.getServerSideEncryptionKey(),
+          len);
+      renameTracker.fileCopied(
+          sourceFile,
+          srcAttributes,
+          destAttributes,
+          destination,
+          blocksize,
+          addAncestors);
+    }
+    return destination;
+  }
+
+  /**
+   * Remove source objects and update the metastore by way of
+   * the rename tracker.
+   * @param keys list of keys to delete
+   * @param paths list of paths matching the keys to delete 1:1.
+   * @throws IOException failure
+   */
+  @Retries.RetryMixed
+  private void removeSourceObjects(
+      final List<DeleteObjectsRequest.KeyVersion> keys,
+      final List<Path> paths)
+      throws IOException {
+    List<Path> undeletedObjects = new ArrayList<>();
+    try {
+      // remove the keys
+      // this will update the metastore on a failure, but on
+      // a successful operation leaves the store as is.
+      callbacks.removeKeys(keys, 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 renameTracker.deleteFailed(e, paths, undeletedObjects);
+    }
+    renameTracker.sourceObjectsDeleted(paths);
+  }
+
+  /**
+   * Turns a path (relative or otherwise) into an S3 key, adding a trailing
+   * "/" if the path is not the root <i>and</i> does not already have a "/"
+   * at the end.
+   *
+   * @param key s3 key or ""
+   * @return the with a trailing "/", or, if it is the root key, "",
+   */
+  private String maybeAddTrailingSlash(String key) {
+    if (!key.isEmpty() && !key.endsWith("/")) {
+      return key + '/';
+    } else {
+      return key;
+    }
+  }
+
+  /**
+   * These are all the callbacks which the rename operation needs,
+   * derived from the appropriate S3AFileSystem methods.
+   */
+  public interface RenameOperationCallbacks {
+
+    /**
+     * Create the attributes of an object for subsequent use.
+     * @param path path path of the request.
+     * @param eTag the eTag of the S3 object
+     * @param versionId S3 object version ID
+     * @param len length of the file
+     * @return attributes to use when building the query.
+     */
+    S3ObjectAttributes createObjectAttributes(
+        Path path,
+        String eTag,
+        String versionId,
+        long len);
+
+    /**
+     * Create the attributes of an object for subsequent use.
+     * @param fileStatus file status to build from.
+     * @return attributes to use when building the query.
+     */
+    S3ObjectAttributes createObjectAttributes(
+        S3AFileStatus fileStatus);
+
+    /**
+     * Create the read context for reading from the referenced file,
+     * using FS state as well as the status.
+     * @param fileStatus file status.
+     * @return a context for read and select operations.
+     */
+    S3AReadOpContext createReadContext(
+        FileStatus fileStatus);
+
+    /**
+     * The rename has finished; perform any store cleanup operations
+     * such as creating/deleting directory markers.
+     * @param sourceRenamed renamed source
+     * @param destCreated destination file created.
+     * @throws IOException failure
+     */
+    void finishRename(Path sourceRenamed, Path destCreated) throws IOException;
+
+    /**
+     * Delete an object, also updating the metastore.
+     * This call does <i>not</i> create any mock parent entries.
+     * Retry policy: retry untranslated; delete considered idempotent.
+     * @param path path to delete
+     * @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 in the metastore
+     */
+    @Retries.RetryMixed
+    void deleteObjectAtPath(Path path, String key, boolean isFile)
+        throws IOException;
+
+    /**
+     * Recursive list of files and empty directories.
+     * @param path path to list from
+     * @return an iterator.
+     * @throws IOException failure
+     */
+    RemoteIterator<S3ALocatedFileStatus> listFilesAndEmptyDirectories(
+        Path path) throws IOException;
+
+    /**
+     * Copy a single object in the bucket via a COPY operation.
+     * There's no update of metadata, directory markers, etc.
+     * Callers must implement.
+     * @param srcKey source object path
+     * @param srcAttributes S3 attributes of the source object
+     * @param readContext the read context
+     * @return the result of the copy
+     * @throws InterruptedIOException the operation was interrupted
+     * @throws IOException Other IO problems
+     */
+    @Retries.RetryTranslated
+    CopyResult copyFile(String srcKey,
+        String destKey,
+        S3ObjectAttributes srcAttributes,
+        S3AReadOpContext readContext)
+        throws IOException;
+
+    /**
+     * Remove keys from the store, updating the metastore on a
+     * partial delete represented as a MultiObjectDeleteException failure by
+     * deleting all those entries successfully deleted and then rethrowing
+     * the MultiObjectDeleteException.
+     * @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.
+     */
+    @Retries.RetryMixed
+    void removeKeys(
+        List<DeleteObjectsRequest.KeyVersion> keysToDelete,
+        boolean deleteFakeDir,
+        List<Path> undeletedObjectsOnFailure)
+        throws MultiObjectDeleteException, AmazonClientException,
+        IOException;
+  }
+}
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 0000000..28300c2
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java
@@ -0,0 +1,335 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.impl;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+
+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.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.Statistic;
+import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider;
+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
+ * components, without exposing the entire parent class.
+ * This is eliminate explicit recursive coupling.
+ *
+ * Where methods on the FS are to be invoked, they are referenced
+ * via the {@link ContextAccessors} interface, so tests can implement
+ * their own.
+ *
+ * <i>Warning:</i> this really is private and unstable. Do not use
+ * outside the org.apache.hadoop.fs.s3a package.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class StoreContext {
+
+  /** 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;
+
+  /**
+   * Bounded thread pool for async operations.
+   */
+  private final ListeningExecutorService executor;
+
+  /**
+   * Capacity of new executors created.
+   */
+  private final int executorCapacity;
+
+  /** Invoker of operations. */
+  private final Invoker invoker;
+
+  /** 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 multiObjectDeleteEnabled;
+
+  /** List algorithm. */
+  private final boolean useListV1;
+
+  /**
+   * To allow this context to be passed down to the metastore, this field
+   * wll be null until initialized.
+   */
+  private final MetadataStore metadataStore;
+
+  private final ContextAccessors contextAccessors;
+
+  /**
+   * Source of time.
+   */
+  private ITtlTimeProvider timeProvider;
+
+  /**
+   * 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,
+      final Configuration configuration,
+      final String username,
+      final UserGroupInformation owner,
+      final ListeningExecutorService executor,
+      final int executorCapacity,
+      final Invoker invoker,
+      final S3AInstrumentation instrumentation,
+      final S3AStorageStatistics storageStatistics,
+      final S3AInputPolicy inputPolicy,
+      final ChangeDetectionPolicy changeDetectionPolicy,
+      final boolean multiObjectDeleteEnabled,
+      final MetadataStore metadataStore,
+      final boolean useListV1,
+      final ContextAccessors contextAccessors,
+      final ITtlTimeProvider timeProvider) {
+    this.fsURI = fsURI;
+    this.bucket = bucket;
+    this.configuration = configuration;
+    this.username = username;
+    this.owner = owner;
+    this.executor = executor;
+    this.executorCapacity = executorCapacity;
+    this.invoker = invoker;
+    this.instrumentation = instrumentation;
+    this.storageStatistics = storageStatistics;
+    this.inputPolicy = inputPolicy;
+    this.changeDetectionPolicy = changeDetectionPolicy;
+    this.multiObjectDeleteEnabled = multiObjectDeleteEnabled;
+    this.metadataStore = metadataStore;
+    this.useListV1 = useListV1;
+    this.contextAccessors = contextAccessors;
+    this.timeProvider = timeProvider;
+  }
+
+  @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 ListeningExecutorService getExecutor() {
+    return executor;
+  }
+
+  public Invoker getInvoker() {
+    return invoker;
+  }
+
+  public S3AInstrumentation getInstrumentation() {
+    return instrumentation;
+  }
+
+  public S3AInputPolicy getInputPolicy() {
+    return inputPolicy;
+  }
+
+  public ChangeDetectionPolicy getChangeDetectionPolicy() {
+    return changeDetectionPolicy;
+  }
+
+  public boolean isMultiObjectDeleteEnabled() {
+    return multiObjectDeleteEnabled;
+  }
+
+  public MetadataStore getMetadataStore() {
+    return metadataStore;
+  }
+
+  public boolean isUseListV1() {
+    return useListV1;
+  }
+
+  /**
+   * Convert a key to a fully qualified path.
+   * @param key input key
+   * @return the fully qualified path including URI scheme and bucket name.
+   */
+  public Path keyToPath(String key) {
+    return contextAccessors.keyToPath(key);
+  }
+
+  /**
+   * Turns a path (relative or otherwise) into an S3 key.
+   *
+   * @param path input path, may be relative to the working dir
+   * @return a key excluding the leading "/", or, if it is the root path, ""
+   */
+  public String pathToKey(Path path) {
+    return contextAccessors.pathToKey(path);
+  }
+
+  /**
+   * 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);
+  }
+
+  /**
+   * 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 <i>without</i> 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);
+  }
+
+  public UserGroupInformation getOwner() {
+    return owner;
+  }
+
+  /**
+   * 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 contextAccessors.createTempFile(prefix, size);
+  }
+
+  /**
+   * Get the location of the bucket.
+   * @return the bucket location.
+   * @throws IOException failure.
+   */
+  public String getBucketLocation() throws IOException {
+    return contextAccessors.getBucketLocation();
+  }
+
+  /**
+   * Get the time provider.
+   * @return the time source.
+   */
+  public ITtlTimeProvider getTimeProvider() {
+    return timeProvider;
+  }
+}
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 0000000..0fe05db
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/BulkOperationState.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.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.
+ * <p>
+ * A bulk operation state <i>MUST</i> only be be used for the single store
+ * from which it was created, and <i>MUST</i>only for the duration of a single
+ * bulk update operation.
+ * <p>
+ * Passing in the state is to allow the stores to maintain state about
+ * updates they have already made to their store during this single operation:
+ * a cache of what has happened. It is not a list of operations to be applied.
+ * If a list of operations to perform is built up (e.g. during rename)
+ * that is the duty of the caller, not this state.
+ * <p>
+ * After the operation has completed, it <i>MUST</i> be closed so
+ * as to guarantee that all state is released.
+ */
+public class BulkOperationState implements Closeable {
+
+  private final OperationType operation;
+
+  /**
+   * Constructor.
+   * @param operation the type of the operation.
+   */
+  public BulkOperationState(final OperationType operation) {
+    this.operation = operation;
+  }
+
+  /**
+   * Get the operation type.
+   * @return the operation type.
+   */
+  public OperationType getOperation() {
+    return operation;
+  }
+
+  @Override
+  public void close() throws IOException {
+
+  }
+
+  /**
+   * Enumeration of operations which can be performed in bulk.
+   * This can be used by the stores however they want.
+   * One special aspect: renames are to be done through a {@link RenameTracker}.
+   * Callers will be blocked from initiating a rename through
+   * {@code S3Guard#initiateBulkWrite()}
+   */
+  public enum OperationType {
+    /** Writing data. */
+    Put,
+    /** Rename: add and delete. */
+    Rename,
+    /** Pruning: deleting entries and updating parents. */
+    Prune,
+  }
+}
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
new file mode 100644
index 0000000..916714b
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameTracker.java
@@ -0,0 +1,192 @@
+/*
+ * 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 java.util.Set;
+
+import com.amazonaws.SdkBaseException;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
+import org.apache.hadoop.fs.s3a.Tristate;
+import org.apache.hadoop.fs.s3a.impl.StoreContext;
+import org.apache.hadoop.util.DurationInfo;
+
+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,
+ * then updated at the end (possibly slow).
+ * <p>
+ * It is not currently instantiated by any of the active trackers,
+ * but is preserved to show that the original rename strategy
+ * can be implemented via the tracker model.
+ */
+public class DelayedUpdateRenameTracker extends RenameTracker {
+
+  private final MetadataStore metadataStore;
+
+  private final Collection<Path> sourcePaths = new HashSet<>();
+
+  private final List<PathMetadata> destMetas = new ArrayList<>();
+
+  private final List<Path> deletedPaths = new ArrayList<>();
+
+  public DelayedUpdateRenameTracker(
+      final StoreContext storeContext,
+      final MetadataStore metadataStore,
+      final Path sourceRoot,
+      final Path dest,
+      final BulkOperationState operationState) {
+    super("DelayedUpdateRenameTracker", storeContext, metadataStore,
+        sourceRoot, dest, operationState);
+    this.metadataStore = storeContext.getMetadataStore();
+  }
+
+  @Override
+  public synchronized void fileCopied(
+      final Path sourcePath,
+      final S3ObjectAttributes sourceAttributes,
+      final S3ObjectAttributes destAttributes,
+      final Path destPath,
+      final long blockSize,
+      final boolean addAncestors) throws IOException {
+
+    addMoveFile(metadataStore,
+        sourcePaths,
+        destMetas,
+        sourcePath,
+        destPath,
+        sourceAttributes.getLen(),
+        blockSize,
+        getOwner(),
+        destAttributes.getETag(),
+        destAttributes.getVersionId());
+    // Ancestor directories may not be listed, so we explicitly add them
+    if (addAncestors) {
+      addMoveAncestors(metadataStore,
+          sourcePaths,
+          destMetas,
+          getSourceRoot(),
+          sourcePath,
+          destPath,
+          getOwner());
+    }
+  }
+
+  @Override
+  public synchronized void directoryMarkerCopied(final Path sourcePath,
+      final Path destPath,
+      final boolean addAncestors) throws IOException {
+    addMoveDir(metadataStore, sourcePaths, destMetas,
+        sourcePath,
+        destPath, getOwner());
+    // Ancestor directories may not be listed, so we explicitly add them
+    if (addAncestors) {
+      addMoveAncestors(metadataStore,
+          sourcePaths,
+          destMetas,
+          getSourceRoot(),
+          sourcePath,
+          destPath,
+          getOwner());
+    }
+  }
+
+  @Override
+  public synchronized void moveSourceDirectory() throws IOException {
+    if (!sourcePaths.contains(getSourceRoot())) {
+      addMoveDir(metadataStore, sourcePaths, destMetas,
+          getSourceRoot(),
+          getDest(), getOwner());
+    }
+  }
+
+  @Override
+  public synchronized void sourceObjectsDeleted(
+      final Collection<Path> paths) throws IOException {
+    // add to the list of deleted paths.
+    deletedPaths.addAll(paths);
+  }
+
+  @Override
+  public void completeRename() throws IOException {
+    metadataStore.move(sourcePaths, destMetas,
+        getStoreContext().getTimeProvider(),
+        getOperationState());
+    super.completeRename();
+  }
+
+  @Override
+  public IOException renameFailed(final Exception ex) {
+    LOG.warn("Rename has failed; updating s3guard with destination state");
+    try (DurationInfo ignored = new DurationInfo(LOG,
+        "Cleaning up deleted paths")) {
+      // the destination paths are updated; the source is left alone.
+      metadataStore.move(new ArrayList<>(0), destMetas,
+          getStoreContext().getTimeProvider(),
+          getOperationState());
+      for (Path deletedPath : deletedPaths) {
+        // this is not ideal in that it may leave parent stuff around.
+        metadataStore.delete(deletedPath, getStoreContext().getTimeProvider());
+      }
+      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<Path> 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<Path> parents = new ArrayList<>(parentPaths);
+    parents.sort(PathOrderComparators.TOPMOST_PATH_LAST);
+    for (Path parent : parents) {
+      PathMetadata md = metadataStore.get(parent, true);
+      if (md != null && md.isEmptyDirectory() == Tristate.TRUE) {
+        // if were confident that this is empty: delete it.
+        metadataStore.delete(parent, getStoreContext().getTimeProvider());
+      }
+    }
+  }
+}
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 f668c6a..9f06313 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;
@@ -35,6 +36,7 @@ import java.util.Map;
 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;
@@ -59,6 +61,7 @@ import com.amazonaws.services.dynamodbv2.document.TableWriteItems;
 import com.amazonaws.services.dynamodbv2.document.spec.GetItemSpec;
 import com.amazonaws.services.dynamodbv2.document.spec.QuerySpec;
 import com.amazonaws.services.dynamodbv2.document.utils.ValueMap;
+import com.amazonaws.services.dynamodbv2.model.AmazonDynamoDBException;
 import com.amazonaws.services.dynamodbv2.model.BillingMode;
 import com.amazonaws.services.dynamodbv2.model.CreateTableRequest;
 import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughput;
@@ -73,6 +76,7 @@ import com.amazonaws.waiters.WaiterTimedOutException;
 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;
 
@@ -83,6 +87,7 @@ 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.PathIOException;
 import org.apache.hadoop.fs.s3a.AWSClientIOException;
 import org.apache.hadoop.fs.s3a.AWSCredentialProviderList;
 import org.apache.hadoop.fs.s3a.AWSServiceThrottledException;
@@ -97,15 +102,19 @@ import org.apache.hadoop.fs.s3a.Tristate;
 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.impl.CallableSupplier.waitForCompletion;
 import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.*;
 import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.*;
 
@@ -191,7 +200,7 @@ import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.*;
  *
  * Some mutating operations, notably
  * {@link MetadataStore#deleteSubtree(Path, ITtlTimeProvider)} and
- * {@link MetadataStore#move(Collection, Collection, ITtlTimeProvider)},
+ * {@link MetadataStore#move(Collection, Collection, ITtlTimeProvider, BulkOperationState)}
  * are less efficient with this schema.
  * They require mutating multiple items in the DynamoDB table.
  *
@@ -256,9 +265,25 @@ public class DynamoDBMetadataStore implements MetadataStore,
   public static final String E_ON_DEMAND_NO_SET_CAPACITY
       = "Neither ReadCapacityUnits nor WriteCapacityUnits can be specified when BillingMode is PAY_PER_REQUEST";
 
+  @VisibleForTesting
+  static final String E_INCONSISTENT_UPDATE
+      = "Duplicate and inconsistent entry in update operation";
+
   private static ValueMap deleteTrackingValueMap =
       new ValueMap().withBoolean(":false", false);
 
+  /**
+   * The maximum number of outstanding operations to submit
+   * before blocking to await completion of all the executors.
+   * Paging work like this is less efficient, but it ensures that
+   * failure (auth, network, etc) are picked up before many more
+   * operations are submitted.
+   *
+   * Arbitrary Choice.
+   * Value: {@value}.
+   */
+  private static final int S3GUARD_DDB_SUBMITTED_TASK_LIMIT = 50;
+
   private AmazonDynamoDB amazonDynamoDB;
   private DynamoDB dynamoDB;
   private AWSCredentialProviderList credentials;
@@ -309,6 +334,17 @@ public class DynamoDBMetadataStore implements MetadataStore,
   private AtomicInteger throttleEventCount = new AtomicInteger(0);
 
   /**
+   * Executor for submitting operations.
+   */
+  private ListeningExecutorService executor;
+
+  /**
+   * Time source. This is used during writes when parent
+   * entries need to be created.
+   */
+  private ITtlTimeProvider timeProvider;
+
+  /**
    * A utility function to create DynamoDB instance.
    * @param conf the file system configuration
    * @param s3Region region of the associated S3 bucket (if any).
@@ -383,6 +419,7 @@ public class DynamoDBMetadataStore implements MetadataStore,
         this::retryEvent
     );
 
+    timeProvider = new S3Guard.TtlTimeProvider(conf);
     initTable();
 
     instrumentation.initialized();
@@ -398,8 +435,13 @@ public class DynamoDBMetadataStore implements MetadataStore,
   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();
+    timeProvider = Preconditions.checkNotNull(
+        context.getTimeProvider(),
+        "ttlTimeProvider must not be null");
   }
 
   /**
@@ -444,8 +486,19 @@ public class DynamoDBMetadataStore implements MetadataStore,
     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);
-
+    timeProvider = new S3Guard.TtlTimeProvider(conf);
     initTable();
   }
 
@@ -481,6 +534,7 @@ public class DynamoDBMetadataStore implements MetadataStore,
   @Override
   @Retries.RetryTranslated
   public void forgetMetadata(Path path) throws IOException {
+    LOG.debug("Forget metadata for {}", path);
     innerDelete(path, false, null);
   }
 
@@ -522,14 +576,23 @@ public class DynamoDBMetadataStore implements MetadataStore,
           "Put tombstone",
           path.toString(),
           idempotent,
-          () -> table.putItem(item));
+          () -> {
+            LOG.debug("Adding tombstone to {}", path);
+            recordsWritten(1);
+            table.putItem(item);
+          });
     } else {
       PrimaryKey key = pathToKey(path);
       writeOp.retry(
           "Delete key",
           path.toString(),
           idempotent,
-          () -> table.deleteItem(key));
+          () -> {
+            // record the attempt so even on retry the counter goes up.
+            LOG.debug("Delete key {}", path);
+            recordsDeleted(1);
+            table.deleteItem(key);
+          });
     }
   }
 
@@ -547,16 +610,28 @@ public class DynamoDBMetadataStore implements MetadataStore,
       return;
     }
 
+    // Execute via the bounded threadpool.
+    final List<CompletableFuture<Void>> futures = new ArrayList<>();
     for (DescendantsIterator desc = new DescendantsIterator(this, meta);
          desc.hasNext();) {
-      innerDelete(desc.next().getPath(), true, ttlTimeProvider);
+      final Path pathToDelete = desc.next().getPath();
+      futures.add(submit(executor, () -> {
+        innerDelete(pathToDelete, true, ttlTimeProvider);
+        return null;
+      }));
+      if (futures.size() > S3GUARD_DDB_SUBMITTED_TASK_LIMIT) {
+        // first batch done; block for completion.
+        waitForCompletion(futures);
+        futures.clear();
+      }
     }
+    // now wait for the final set.
+    waitForCompletion(futures);
   }
 
   /**
    * 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
    */
@@ -569,7 +644,10 @@ public class DynamoDBMetadataStore implements MetadataStore,
     return readOp.retry("get",
         path.toString(),
         true,
-        () -> table.getItem(spec));
+        () -> {
+          recordsRead(1);
+          return table.getItem(spec);
+        });
   }
 
   @Override
@@ -583,8 +661,11 @@ public class DynamoDBMetadataStore implements MetadataStore,
   public DDBPathMetadata get(Path path, boolean wantEmptyDirectoryFlag)
       throws IOException {
     checkPath(path);
-    LOG.debug("Get from table {} in region {}: {}", tableName, region, path);
-    return innerGet(path, wantEmptyDirectoryFlag);
+    LOG.debug("Get from table {} in region {}: {}. wantEmptyDirectory={}",
+        tableName, region, path, wantEmptyDirectoryFlag);
+    DDBPathMetadata result = innerGet(path, wantEmptyDirectoryFlag);
+    LOG.debug("result of get {} is: {}", path, result);
+    return result;
   }
 
   /**
@@ -711,40 +792,194 @@ public class DynamoDBMetadataStore implements MetadataStore,
   }
 
   /**
-   * build the list of all parent entries.
+   * Build the list of all parent entries.
+   * <p>
+   * <b>Thread safety:</b> none. Callers must synchronize access.
+   * <p>
+   * Callers are required to synchronize on ancestorState.
    * @param pathsToCreate paths to create
+   * @param ancestorState ongoing ancestor state.
+   * @param ttlTimeProvider Must not be null
    * @return the full ancestry paths
    */
-  Collection<DDBPathMetadata> completeAncestry(
-      Collection<DDBPathMetadata> pathsToCreate) {
-    // Key on path to allow fast lookup
-    Map<Path, DDBPathMetadata> ancestry = new HashMap<>();
-
-    for (DDBPathMetadata meta : pathsToCreate) {
+  private Collection<DDBPathMetadata> completeAncestry(
+      final Collection<DDBPathMetadata> pathsToCreate,
+      final AncestorState ancestorState,
+      final ITtlTimeProvider ttlTimeProvider) throws PathIOException {
+    List<DDBPathMetadata> ancestorsToAdd = new ArrayList<>(0);
+    LOG.debug("Completing ancestry for {} paths", pathsToCreate.size());
+    // we sort the inputs to guarantee that the topmost entries come first.
+    // that way if the put request contains both parents and children
+    // then the existing parents will not be re-created -they will just
+    // be added to the ancestor list first.
+    List<DDBPathMetadata> sortedPaths = new ArrayList<>(pathsToCreate);
+    sortedPaths.sort(PathOrderComparators.TOPMOST_PM_FIRST);
+    // iterate through the paths.
+    for (DDBPathMetadata meta : sortedPaths) {
       Preconditions.checkArgument(meta != null);
       Path path = meta.getFileStatus().getPath();
+      LOG.debug("Adding entry {}", path);
       if (path.isRoot()) {
+        // this is a root entry: do not add it.
         break;
       }
-      ancestry.put(path, new DDBPathMetadata(meta));
+      // create the new entry
+      DDBPathMetadata entry = new DDBPathMetadata(meta);
+      // add it to the ancestor state, failing if it is already there and
+      // of a different type.
+      DDBPathMetadata oldEntry = ancestorState.put(path, entry);
+      if (oldEntry != null) {
+        if (!oldEntry.getFileStatus().isDirectory()
+            || !entry.getFileStatus().isDirectory()) {
+          // check for and warn if the existing bulk operation overwrote it.
+          // this should never occur outside tests explicitly crating it
+          LOG.warn("Overwriting a S3Guard file created in the operation: {}",
+              oldEntry);
+          LOG.warn("With new entry: {}", entry);
+          // restore the old state
+          ancestorState.put(path, oldEntry);
+          // then raise an exception
+          throw new PathIOException(path.toString(), E_INCONSISTENT_UPDATE);
+        } else {
+          // a directory is already present. Log and continue.
+          LOG.debug("Directory at {} being updated with value {}",
+              path, entry);
+        }
+      }
+      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 S3AFileStatus status = makeDirStatus(parent, username);
-        ancestry.put(parent, new DDBPathMetadata(status, Tristate.FALSE,
-            false));
+      while (!parent.isRoot()) {
+        if (!ancestorState.findEntry(parent, true)) {
+          // don't add this entry, but carry on with the parents
+          LOG.debug("auto-create ancestor path {} for child path {}",
+              parent, path);
+          final S3AFileStatus status = makeDirStatus(parent, username);
+          DDBPathMetadata md = new DDBPathMetadata(status, Tristate.FALSE,
+              false, false, ttlTimeProvider.getNow());
+          ancestorState.put(parent, md);
+          ancestorsToAdd.add(md);
+        }
         parent = parent.getParent();
       }
     }
-    return ancestry.values();
+    return ancestorsToAdd;
   }
 
+  /**
+   * {@inheritDoc}
+   * <p>
+   * The implementation scans all up the directory tree and does a get()
+   * for each entry; at each level one is found it is added to the ancestor
+   * state.
+   * <p>
+   * The original implementation would stop on finding the first non-empty
+   * parent. This (re) implementation issues a GET for every parent entry
+   * and so detects and recovers from a tombstone marker further up the tree
+   * (i.e. an inconsistent store is corrected for).
+   * <p>
+   * if {@code operationState} is not null, when this method returns the
+   * operation state will be updated with all new entries created.
+   * This ensures that subsequent operations with the same store will not
+   * trigger new updates.
+   * @param qualifiedPath path to update
+   * @param operationState (nullable) operational state for a bulk update
+   * @throws IOException on failure.
+   */
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
   @Override
   @Retries.RetryTranslated
-  public void move(Collection<Path> pathsToDelete,
-      Collection<PathMetadata> pathsToCreate, ITtlTimeProvider ttlTimeProvider)
-      throws IOException {
+  public void addAncestors(
+      final Path qualifiedPath,
+      final ITtlTimeProvider ttlTimeProvider,
+      @Nullable final BulkOperationState operationState) throws IOException {
+
+    Collection<DDBPathMetadata> newDirs = new ArrayList<>();
+    final AncestorState ancestorState = extractOrCreate(operationState,
+        BulkOperationState.OperationType.Rename);
+    Path parent = qualifiedPath.getParent();
+    boolean entryFound = false;
+
+    // Iterate up the parents.
+    // note that only ancestorState get/set operations are synchronized;
+    // the DDB read between them is not. As a result, more than one
+    // thread may probe the state, find the entry missing, do the database
+    // query and add the entry.
+    // This is done to avoid making the remote dynamo query part of the
+    // synchronized block.
+    // If a race does occur, the cost is simply one extra GET and potentially
+    // one extra PUT.
+    while (!parent.isRoot()) {
+      synchronized (ancestorState) {
+        if (ancestorState.contains(parent)) {
+          // the ancestry map contains the key, so no need to even look for it.
+          break;
+        }
+      }
+      // we don't worry about tombstone expiry here as expired or not,
+      // a directory entry will go in.
+      PathMetadata directory = get(parent);
+      if (directory == null || directory.isDeleted()) {
+        if (entryFound) {
+          LOG.warn("Inconsistent S3Guard table: adding directory {}", parent);
+        }
+        S3AFileStatus status = makeDirStatus(username, parent);
+        LOG.debug("Adding new ancestor entry {}", status);
+        DDBPathMetadata meta = new DDBPathMetadata(status, Tristate.FALSE,
+            false);
+        newDirs.add(meta);
+        // Do not update ancestor state here, as it
+        // will happen in the innerPut() call. Were we to add it
+        // here that put operation would actually (mistakenly) skip
+        // creating the entry.
+      } else {
+        // an entry was found. Check its type
+        entryFound = true;
+        if (directory.getFileStatus().isFile()) {
+          throw new PathIOException(parent.toString(),
+              "Cannot overwrite parent file: metadatstore is"
+                  + " in an inconsistent state");
+        }
+        // the directory exists. Add it to the ancestor state for next time.
+        synchronized (ancestorState) {
+          ancestorState.put(parent, new DDBPathMetadata(directory));
+        }
+      }
+      parent = parent.getParent();
+    }
+    // the listing of directories to put is all those parents which we know
+    // are not in the store or BulkOperationState.
+    if (!newDirs.isEmpty()) {
+      // patch up the time.
+      patchLastUpdated(newDirs, ttlTimeProvider);
+      innerPut(newDirs, operationState, ttlTimeProvider);
+    }
+  }
+
+  /**
+   * {@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
+   *                      ().
+   * @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<Path> pathsToDelete,
+      @Nullable Collection<PathMetadata> pathsToCreate,
+      final ITtlTimeProvider ttlTimeProvider,
+      @Nullable final BulkOperationState operationState) throws IOException {
     if (pathsToDelete == null && pathsToCreate == null) {
       return;
     }
@@ -761,18 +996,37 @@ public class DynamoDBMetadataStore implements MetadataStore,
     // 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<DDBPathMetadata> newItems = new ArrayList<>();
+    AncestorState ancestorState = extractOrCreate(operationState,
+        BulkOperationState.OperationType.Rename);
+    List<DDBPathMetadata> 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,
+                extractTimeProvider(ttlTimeProvider)));
+      }
     }
+    // sort all the new items topmost first.
+    newItems.sort(PathOrderComparators.TOPMOST_PM_FIRST);
+
+    // now process the deletions.
     if (pathsToDelete != null) {
+      List<DDBPathMetadata> tombstones = new ArrayList<>(pathsToDelete.size());
       for (Path meta : pathsToDelete) {
         Preconditions.checkArgument(ttlTimeProvider != null, "ttlTimeProvider"
             + " must not be null");
         final PathMetadata pmTombstone = PathMetadata.tombstone(meta);
         pmTombstone.setLastUpdated(ttlTimeProvider.getNow());
-        newItems.add(new DDBPathMetadata(pmTombstone));
+        tombstones.add(new DDBPathMetadata(pmTombstone));
       }
+      // sort all the tombstones lowest first.
+      tombstones.sort(PathOrderComparators.TOPMOST_PM_LAST);
+      newItems.addAll(tombstones);
     }
 
     processBatchWriteRequest(null, pathMetadataToItem(newItems));
@@ -780,9 +1034,12 @@ public class DynamoDBMetadataStore implements MetadataStore,
 
   /**
    * Helper method to issue a batch write request to DynamoDB.
-   *
+   * <ol>
+   *   <li>Keys to delete are processed ahead of writing new items.</li>
+   *   <li>No attempt is made to sort the input: the caller must do that</li>
+   * </ol>
    * As well as retrying on the operation invocation, incomplete
-   * batches are retried until all have been deleted.
+   * batches are retried until all have been processed..
    * @param keysToDelete primary keys to be deleted; can be null
    * @param itemsToPut new items to be put; can be null
    * @return the number of iterations needed to complete the call.
@@ -792,6 +1049,10 @@ public class DynamoDBMetadataStore implements MetadataStore,
       Item[] itemsToPut) throws IOException {
     final int totalToDelete = (keysToDelete == null ? 0 : keysToDelete.length);
     final int totalToPut = (itemsToPut == null ? 0 : itemsToPut.length);
+    if (totalToPut == 0 && totalToDelete == 0) {
+      LOG.debug("Ignoring empty batch write request");
+      return 0;
+    }
     int count = 0;
     int batches = 0;
     while (count < totalToDelete + totalToPut) {
@@ -843,6 +1104,12 @@ public class DynamoDBMetadataStore implements MetadataStore,
         unprocessed = res.getUnprocessedItems();
       }
     }
+    if (itemsToPut != null) {
+      recordsWritten(itemsToPut.length);
+    }
+    if (keysToDelete != null) {
+      recordsDeleted(keysToDelete.length);
+    }
     return batches;
   }
 
@@ -893,7 +1160,15 @@ public class DynamoDBMetadataStore implements MetadataStore,
 
   @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,
@@ -904,32 +1179,77 @@ public class DynamoDBMetadataStore implements MetadataStore,
 
     Collection<PathMetadata> wrapper = new ArrayList<>(1);
     wrapper.add(meta);
-    put(wrapper);
+    put(wrapper, operationState);
   }
 
   @Override
   @Retries.RetryTranslated
-  public void put(Collection<PathMetadata> metas) throws IOException {
-    innerPut(pathMetaToDDBPathMeta(metas));
+  public void put(
+      final Collection<? extends PathMetadata> metas,
+      @Nullable final BulkOperationState operationState) throws IOException {
+    innerPut(pathMetaToDDBPathMeta(metas), operationState, timeProvider);
   }
 
-  @Retries.OnceRaw
-  private void innerPut(Collection<DDBPathMetadata> metas) throws IOException {
-    Item[] items = pathMetadataToItem(completeAncestry(metas));
+  /**
+   * Internal put operation.
+   * <p>
+   * The ancestors to all entries are added to the set of entries to write,
+   * provided they are not already stored in any supplied operation state.
+   * Both the supplied metadata entries and ancestor entries are sorted
+   * so that the topmost entries are written first.
+   * This is to ensure that a failure partway through the operation will not
+   * create entries in the table without parents.
+   * @param metas metadata entries to write.
+   * @param operationState (nullable) operational state for a bulk update
+   * @param ttlTimeProvider
+   * @throws IOException failure.
+   */
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Retries.RetryTranslated
+  private void innerPut(
+      final Collection<DDBPathMetadata> metas,
+      @Nullable final BulkOperationState operationState,
+      final ITtlTimeProvider ttlTimeProvider) throws IOException {
+    if (metas.isEmpty()) {
+      // Happens when someone calls put() with an empty list.
+      LOG.debug("Ignoring empty list of entries to put");
+      return;
+    }
+    // always create or retrieve an ancestor state instance, so it can
+    // always be used for synchronization.
+    final AncestorState ancestorState = extractOrCreate(operationState,
+        BulkOperationState.OperationType.Put);
+
+    Item[] items;
+    synchronized (ancestorState) {
+      items = pathMetadataToItem(
+          completeAncestry(metas, ancestorState, ttlTimeProvider));
+    }
     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 when looking for parents which are not in
+   * any supplied ongoing operation state.
+   * Updates the operation state with found entries to reduce further checks.
+   *
+   * @param meta metadata to put
+   * @param operationState ongoing bulk state
+   * @return a possibly empty list of entries to put.
+   * @throws IOException failure
    */
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
   @VisibleForTesting
   @Retries.RetryTranslated
-  Collection<DDBPathMetadata> fullPathsToPut(DDBPathMetadata meta)
+  List<DDBPathMetadata> fullPathsToPut(DDBPathMetadata meta,
+      @Nullable BulkOperationState operationState)
       throws IOException {
     checkPathMetadata(meta);
-    final Collection<DDBPathMetadata> metasToPut = new ArrayList<>();
+    final List<DDBPathMetadata> metasToPut = new ArrayList<>();
     // root path is not persisted
     if (!meta.getFileStatus().getPath().isRoot()) {
       metasToPut.add(meta);
@@ -937,8 +1257,15 @@ public class DynamoDBMetadataStore implements MetadataStore,
 
     // put all its ancestors if not present; as an optimization we return at its
     // first existent ancestor
+    final AncestorState ancestorState = extractOrCreate(operationState,
+        BulkOperationState.OperationType.Put);
     Path path = meta.getFileStatus().getPath().getParent();
     while (path != null && !path.isRoot()) {
+      synchronized (ancestorState) {
+        if (ancestorState.findEntry(path, true)) {
+          break;
+        }
+      }
       final Item item = getConsistentItem(path);
       if (!itemExists(item)) {
         final S3AFileStatus status = makeDirStatus(path, username);
@@ -946,12 +1273,23 @@ public class DynamoDBMetadataStore implements MetadataStore,
             meta.isAuthoritativeDir(), meta.getLastUpdated()));
         path = path.getParent();
       } else {
+        // found the entry in the table, so add it to the ancestor state
+        synchronized (ancestorState) {
+          ancestorState.put(path, itemToPathMetadata(item, username));
+        }
+        // then break out of the loop.
         break;
       }
     }
     return metasToPut;
   }
 
+  /**
+   * Does an item represent an object which exists?
+   * @param item item retrieved in a query.
+   * @return true iff the item isn't null and, if there is an is_deleted
+   * column, that its value is false.
+   */
   private boolean itemExists(Item item) {
     if (item == null) {
       return false;
@@ -963,7 +1301,7 @@ public class DynamoDBMetadataStore implements MetadataStore,
     return true;
   }
 
-  /** Create a directory FileStatus using current system time as mod time. */
+  /** Create a directory FileStatus using 0 for the lastUpdated time. */
   static S3AFileStatus makeDirStatus(Path f, String owner) {
     return new S3AFileStatus(Tristate.UNKNOWN, f, owner);
   }
@@ -974,11 +1312,14 @@ public class DynamoDBMetadataStore implements MetadataStore,
    * the call to {@link #processBatchWriteRequest(PrimaryKey[], Item[])}
    * is only tried once.
    * @param meta Directory listing metadata.
+   * @param operationState operational state for a bulk update
    * @throws IOException IO problem
    */
   @Override
   @Retries.RetryTranslated
-  public void put(DirListingMetadata meta) throws IOException {
+  public void put(
+      final DirListingMetadata meta,
+      @Nullable final BulkOperationState operationState) throws IOException {
     LOG.debug("Saving to table {} in region {}: {}", tableName, region, meta);
 
     // directory path
@@ -986,14 +1327,25 @@ public class DynamoDBMetadataStore implements MetadataStore,
     DDBPathMetadata ddbPathMeta =
         new DDBPathMetadata(makeDirStatus(path, username), meta.isEmpty(),
             false, meta.isAuthoritative(), meta.getLastUpdated());
-
+    // put all its ancestors if not present; as an optimization we return at its
+    // first existent ancestor
+    final AncestorState ancestorState = extractOrCreate(operationState,
+        BulkOperationState.OperationType.Put);
     // First add any missing ancestors...
-    final Collection<DDBPathMetadata> metasToPut = fullPathsToPut(ddbPathMeta);
+    final List<DDBPathMetadata> metasToPut = fullPathsToPut(ddbPathMeta,
+        ancestorState);
 
     // next add all children of the directory
     metasToPut.addAll(pathMetaToDDBPathMeta(meta.getListing()));
 
+    // 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));
+    // and add the ancestors
+    synchronized (ancestorState) {
+      metasToPut.forEach(ancestorState::put);
+    }
   }
 
   @Override
@@ -1026,6 +1378,10 @@ public class DynamoDBMetadataStore implements MetadataStore,
       invoker.retry("delete", null, true,
           () -> table.delete());
       table.waitForDelete();
+    } catch (IllegalArgumentException ex) {
+      throw new TableDeleteTimeoutException(tableName,
+          "Timeout waiting for the table " + tableArn + " to be deleted",
+          ex);
     } catch (FileNotFoundException rnfe) {
       LOG.info("FileNotFoundException while deleting DynamoDB table {} in "
               + "region {}.  This may indicate that the table does not exist, "
@@ -1099,69 +1455,98 @@ public class DynamoDBMetadataStore implements MetadataStore,
   @Retries.RetryTranslated
   public void prune(PruneMode pruneMode, long cutoff, String keyPrefix)
       throws IOException {
+    LOG.debug("Prune files under {} with age {}", keyPrefix, cutoff);
     final ItemCollection<ScanOutcome> items =
         expiredFiles(pruneMode, cutoff, keyPrefix);
-    innerPrune(items);
+    innerPrune(keyPrefix, items);
   }
 
-  private void innerPrune(ItemCollection<ScanOutcome> items)
+  private void innerPrune(String keyPrefix, ItemCollection<ScanOutcome> items)
       throws IOException {
     int itemCount = 0;
-    try {
-      Collection<Path> deletionBatch =
+    try (AncestorState state = initiateBulkWrite(
+        BulkOperationState.OperationType.Prune, null)) {
+      ArrayList<Path> deletionBatch =
           new ArrayList<>(S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT);
       long delay = conf.getTimeDuration(
           S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY,
           S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_DEFAULT,
           TimeUnit.MILLISECONDS);
       Set<Path> parentPathSet = new HashSet<>();
+      Set<Path> clearedParentPathSet = new HashSet<>();
       for (Item item : items) {
         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);
+          // lowest path entries get deleted first.
+          deletionBatch.sort(PathOrderComparators.TOPMOST_PATH_LAST);
           processBatchWriteRequest(pathToKey(deletionBatch), null);
 
           // set authoritative false for each pruned dir listing
-          removeAuthoritativeDirFlag(parentPathSet);
+          removeAuthoritativeDirFlag(parentPathSet, state);
+          // already cleared parent paths.
+          clearedParentPathSet.addAll(parentPathSet);
           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
-        removeAuthoritativeDirFlag(parentPathSet);
+        removeAuthoritativeDirFlag(parentPathSet, state);
         parentPathSet.clear();
       }
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
       throw new InterruptedIOException("Pruning was interrupted");
+    } catch (AmazonDynamoDBException e) {
+      throw translateDynamoDBException(keyPrefix,
+          "Prune of " + keyPrefix + " failed", e);
     }
     LOG.info("Finished pruning {} items in batches of {}", itemCount,
         S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT);
   }
 
-  private void removeAuthoritativeDirFlag(Set<Path> pathSet)
-      throws IOException {
+  /**
+   * Remove the Authoritative Directory Marker from a set of paths, if
+   * those paths are in the store.
+   * If an exception is raised in the get/update process, then the exception
+   * is caught and only rethrown after all the other paths are processed.
+   * This is to ensure a best-effort attempt to update the store.
+   * @param pathSet set of paths.
+   * @param state ongoing operation state.
+   * @throws IOException only after a best effort is made to update the store.
+   */
+  private void removeAuthoritativeDirFlag(
+      final Set<Path> pathSet,
+      final AncestorState state) throws IOException {
+
     AtomicReference<IOException> rIOException = new AtomicReference<>();
 
     Set<DDBPathMetadata> metas = pathSet.stream().map(path -> {
       try {
+        if (state != null && state.get(path) != null) {
+          // there's already an entry for this path
+          LOG.debug("Ignoring update of entry already in the state map");
+          return null;
+        }
         DDBPathMetadata ddbPathMetadata = get(path);
         if(ddbPathMetadata == null) {
           return null;
@@ -1180,7 +1565,9 @@ public class DynamoDBMetadataStore implements MetadataStore,
 
     try {
       LOG.debug("innerPut on metas: {}", metas);
-      innerPut(metas);
+      if (!metas.isEmpty()) {
+        innerPut(metas, state, timeProvider);
+      }
     } catch (IOException e) {
       String msg = String.format("IOException while setting false "
           + "authoritative directory flag on: %s.", metas);
@@ -1801,6 +2188,72 @@ public class DynamoDBMetadataStore implements MetadataStore,
   }
 
   /**
+   * Record the number of records written.
+   * @param count count of records.
+   */
+  private void recordsWritten(final int count) {
+    if (instrumentation != null) {
+      instrumentation.recordsWritten(count);
+    }
+  }
+
+  /**
+   * Record the number of records read.
+   * @param count count of records.
+   */
+  private void recordsRead(final int count) {
+    if (instrumentation != null) {
+      instrumentation.recordsRead(count);
+    }
+  }
+  /**
+   * Record the number of records deleted.
+   * @param count count of records.
+   */
+  private void recordsDeleted(final int count) {
+    if (instrumentation != null) {
+      instrumentation.recordsDeleted(count);
+    }
+  }
+
+  /**
+   * Initiate the rename operation by creating the tracker for the filesystem
+   * to keep up to date with state changes in the S3A bucket.
+   * @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 S3AFileStatus sourceStatus,
+      final Path dest) {
+    return new ProgressiveRenameTracker(storeContext, this, source, dest,
+        new AncestorState(BulkOperationState.OperationType.Rename, dest));
+  }
+
+  @Override
+  public AncestorState initiateBulkWrite(
+      final BulkOperationState.OperationType operation,
+      final Path dest) {
+    return new AncestorState(operation, dest);
+  }
+
+  /**
+   * Extract a time provider from the argument or fall back to the
+   * one in the constructor.
+   * @param ttlTimeProvider nullable time source passed in as an argument.
+   * @return a non-null time source.
+   */
+  private ITtlTimeProvider extractTimeProvider(
+      @Nullable ITtlTimeProvider ttlTimeProvider) {
+    return ttlTimeProvider != null ? ttlTimeProvider : timeProvider;
+  }
+
+  /**
    * Take an {@code IllegalArgumentException} raised by a DDB operation
    * and if it contains an inner SDK exception, unwrap it.
    * @param ex exception.
@@ -1841,4 +2294,117 @@ public class DynamoDBMetadataStore implements MetadataStore,
       return new IOException(e);
     }
   }
+
+  /**
+   * Get the move state passed in; create a new one if needed.
+   * @param state state.
+   * @param operation the type of the operation to use if the state is created.
+   * @return the cast or created state.
+   */
+  @VisibleForTesting
+  static AncestorState extractOrCreate(@Nullable BulkOperationState state,
+      BulkOperationState.OperationType operation) {
+    if (state != null) {
+      return (AncestorState) state;
+    } else {
+      return new AncestorState(operation, null);
+    }
+  }
+
+  /**
+   * This tracks all the ancestors created,
+   * across multiple move/write operations.
+   * 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<Path, DDBPathMetadata> ancestry = new HashMap<>();
+
+    private final Path dest;
+
+    /**
+     * Create the state.
+     * @param operation the type of the operation.
+     * @param dest destination path.
+     */
+    AncestorState(final OperationType operation, @Nullable final Path dest) {
+      super(operation);
+      this.dest = dest;
+    }
+
+    int size() {
+      return ancestry.size();
+    }
+
+    public Path getDest() {
+      return dest;
+    }
+
+    @Override
+    public String toString() {
+      final StringBuilder sb = new StringBuilder(
+          "AncestorState{");
+      sb.append("operation=").append(getOperation());
+      sb.append("; dest=").append(dest);
+      sb.append("; size=").append(size());
+      sb.append("; paths={")
+          .append(StringUtils.join(ancestry.keySet(), " "))
+          .append('}');
+      sb.append('}');
+      return sb.toString();
+    }
+
+    /**
+     * Does the ancestor state contain a path?
+     * @param p path to check
+     * @return true if the state has an entry
+     */
+    boolean contains(Path p) {
+      return ancestry.containsKey(p);
+    }
+
+    DDBPathMetadata put(Path p, DDBPathMetadata md) {
+      return ancestry.put(p, md);
+    }
+
+    DDBPathMetadata put(DDBPathMetadata md) {
+      return ancestry.put(md.getFileStatus().getPath(), md);
+    }
+
+    DDBPathMetadata get(Path p) {
+      return ancestry.get(p);
+    }
+
+    /**
+     * Find an entry in the ancestor state, warning and optionally
+     * raising an exception if there is a file at the path.
+     * @param path path to look up
+     * @param failOnFile fail if a file was found.
+     * @return true iff a directory was found in the ancestor state.
+     * @throws PathIOException if there was a file at the path.
+     */
+    boolean findEntry(
+        final Path path,
+        final boolean failOnFile) throws PathIOException {
+      final DDBPathMetadata ancestor = get(path);
+      if (ancestor != null) {
+        // there's an entry in the ancestor state
+        if (!ancestor.getFileStatus().isDirectory()) {
+          // but: its a file, which means this update is now inconsistent.
+          final String message = E_INCONSISTENT_UPDATE + " entry is " + ancestor
+              .getFileStatus();
+          LOG.error(message);
+          if (failOnFile) {
+            // errors trigger failure
+            throw new PathIOException(path.toString(), message);
+          }
+        }
+        return true;
+      } else {
+        return false;
+      }
+    }
+  }
 }
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 6c13cd1..4327002 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;
 
@@ -30,12 +32,17 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.s3a.S3AFileStatus;
 import org.apache.hadoop.fs.s3a.Tristate;
+import org.apache.hadoop.fs.s3a.impl.StoreContext;
+import org.apache.hadoop.security.UserGroupInformation;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.net.URI;
+import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.Map;
@@ -70,6 +77,8 @@ public class LocalMetadataStore implements MetadataStore {
   /* Null iff this FS does not have an associated URI host. */
   private String uriHost;
 
+  private String username;
+
   @Override
   public void initialize(FileSystem fileSystem) throws IOException {
     Preconditions.checkNotNull(fileSystem);
@@ -100,6 +109,7 @@ public class LocalMetadataStore implements MetadataStore {
     }
 
     localCache = builder.build();
+    username = UserGroupInformation.getCurrentUser().getShortUserName();
   }
 
   @Override
@@ -192,15 +202,19 @@ public class LocalMetadataStore implements MetadataStore {
   }
 
   @Override
-  public void move(Collection<Path> pathsToDelete,
-      Collection<PathMetadata> pathsToCreate,
-      ITtlTimeProvider ttlTimeProvider) throws IOException {
+  public void move(
+      @Nullable Collection<Path> pathsToDelete,
+      @Nullable Collection<PathMetadata> pathsToCreate,
+        ITtlTimeProvider ttlTimeProvider,
+      @Nullable final BulkOperationState operationState) 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) {
@@ -214,7 +228,7 @@ public class LocalMetadataStore implements MetadataStore {
       // 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
@@ -232,7 +246,13 @@ public class LocalMetadataStore implements MetadataStore {
   }
 
   @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);
     S3AFileStatus status = meta.getFileStatus();
@@ -301,7 +321,8 @@ public class LocalMetadataStore implements MetadataStore {
   }
 
   @Override
-  public synchronized void put(DirListingMetadata meta) throws IOException {
+  public synchronized void put(DirListingMetadata meta,
+      final BulkOperationState operationState) throws IOException {
     if (LOG.isDebugEnabled()) {
       LOG.debug("put dirMeta {}", meta.prettyPrint());
     }
@@ -312,13 +333,14 @@ public class LocalMetadataStore implements MetadataStore {
     } else {
       entry.setDirListingMetadata(meta);
     }
-    put(meta.getListing());
+    put(meta.getListing(), null);
   }
 
-  public synchronized void put(Collection<PathMetadata> metas) throws
+  public synchronized void put(Collection<? extends PathMetadata> metas,
+      final BulkOperationState operationState) throws
       IOException {
     for (PathMetadata meta : metas) {
-      put(meta);
+      put(meta, operationState);
     }
   }
 
@@ -564,4 +586,35 @@ public class LocalMetadataStore implements MetadataStore {
     }
   }
 
+  @Override
+  public RenameTracker initiateRenameOperation(final StoreContext storeContext,
+      final Path source,
+      final S3AFileStatus sourceStatus, final Path dest) throws IOException {
+    return new ProgressiveRenameTracker(storeContext, this, source, dest,
+        null);
+  }
+
+  @Override
+  public void addAncestors(final Path qualifiedPath,
+      ITtlTimeProvider ttlTimeProvider,
+      @Nullable final BulkOperationState operationState) throws IOException {
+
+    Collection<PathMetadata> newDirs = new ArrayList<>();
+    Path parent = qualifiedPath.getParent();
+    while (!parent.isRoot()) {
+      PathMetadata directory = get(parent);
+      if (directory == null || directory.isDeleted()) {
+        S3AFileStatus status = new S3AFileStatus(Tristate.FALSE, parent,
+            username);
+        PathMetadata meta = new PathMetadata(status, Tristate.FALSE, false);
+        newDirs.add(meta);
+      } else {
+        break;
+      }
+      parent = parent.getParent();
+    }
+    if (!newDirs.isEmpty()) {
+      put(newDirs, 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 7875d43..397d23a 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;
@@ -30,6 +31,8 @@ import org.apache.hadoop.conf.Configuration;
 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.S3AFileStatus;
+import org.apache.hadoop.fs.s3a.impl.StoreContext;
 
 /**
  * {@code MetadataStore} defines the set of operations that any metadata store
@@ -143,6 +146,23 @@ public interface MetadataStore extends Closeable {
   DirListingMetadata listChildren(Path path) throws IOException;
 
   /**
+   * This adds all new ancestors of a path as directories.
+   * <p>
+   * Important: to propagate TTL information, any new ancestors added
+   * must have their last updated timestamps set through
+   * {@link S3Guard#patchLastUpdated(Collection, ITtlTimeProvider)}.
+   * @param qualifiedPath path to update
+   * @param timeProvider time provider for timestamps
+   * @param operationState (nullable) operational state for a bulk update
+   * @throws IOException failure
+   */
+  @RetryTranslated
+  void addAncestors(
+      Path qualifiedPath,
+      @Nullable ITtlTimeProvider timeProvider,
+      @Nullable BulkOperationState operationState) throws IOException;
+
+  /**
    * Record the effects of a {@link FileSystem#rename(Path, Path)} in the
    * MetadataStore.  Clients provide explicit enumeration of the affected
    * paths (recursively), before and after the rename.
@@ -163,15 +183,18 @@ public interface MetadataStore extends Closeable {
    * @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 ttlTimeProvider the time provider to set last_updated. Must not
    *                        be null.
+   * @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(Collection<Path> pathsToDelete,
-      Collection<PathMetadata> pathsToCreate,
-      ITtlTimeProvider ttlTimeProvider) throws IOException;
+  void move(
+      @Nullable Collection<Path> pathsToDelete,
+      @Nullable Collection<PathMetadata> pathsToCreate,
+      ITtlTimeProvider ttlTimeProvider,
+      @Nullable BulkOperationState operationState) throws IOException;
 
   /**
    * Saves metadata for exactly one path.
@@ -187,14 +210,32 @@ public interface MetadataStore extends Closeable {
   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<PathMetadata> metas) throws IOException;
+  void put(Collection<? extends PathMetadata> metas,
+      @Nullable BulkOperationState operationState) throws IOException;
 
   /**
    * Save directory listing metadata. Callers may save a partial directory
@@ -211,9 +252,11 @@ public interface MetadataStore extends Closeable {
    * another process.
    *
    * @param meta Directory listing metadata.
+   * @param operationState operational state for a bulk update
    * @throws IOException if there is an error
    */
-  void put(DirListingMetadata meta) throws IOException;
+  void put(DirListingMetadata meta,
+      @Nullable BulkOperationState operationState) throws IOException;
 
   /**
    * Destroy all resources associated with the metadata store.
@@ -303,4 +346,36 @@ public interface MetadataStore extends Closeable {
     ALL_BY_MODTIME,
     TOMBSTONES_BY_LASTUPDATED
   }
+
+  /**
+   * Start a rename operation.
+   *
+   * @param storeContext store context.
+   * @param source source path
+   * @param sourceStatus status of the source file/dir
+   * @param dest destination path.
+   * @return the rename tracker
+   * @throws IOException Failure.
+   */
+  RenameTracker initiateRenameOperation(
+      StoreContext storeContext,
+      Path source,
+      S3AFileStatus 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 operation the type of the operation.
+   * @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(
+      BulkOperationState.OperationType operation,
+      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 1472ef1..f0792ab 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
@@ -18,9 +18,14 @@
 
 package org.apache.hadoop.fs.s3a.s3guard;
 
+import javax.annotation.Nullable;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.S3AFileStatus;
+import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
+import org.apache.hadoop.fs.s3a.impl.StoreContext;
 
 import java.io.IOException;
 import java.util.Collection;
@@ -79,19 +84,27 @@ public class NullMetadataStore implements MetadataStore {
   @Override
   public void move(Collection<Path> pathsToDelete,
       Collection<PathMetadata> pathsToCreate,
-      ITtlTimeProvider ttlTimeProvider) throws IOException {
+      ITtlTimeProvider ttlTimeProvider,
+      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<PathMetadata> meta) throws IOException {
+  public void put(PathMetadata meta,
+      final BulkOperationState operationState) throws IOException {
   }
 
   @Override
-  public void put(DirListingMetadata meta) throws IOException {
+  public void put(Collection<? extends PathMetadata> meta,
+      final BulkOperationState operationState) throws IOException {
+  }
+
+  @Override
+  public void put(DirListingMetadata meta,
+      final BulkOperationState operationState) throws IOException {
   }
 
   @Override
@@ -123,4 +136,40 @@ public class NullMetadataStore implements MetadataStore {
   public void updateParameters(Map<String, String> parameters)
       throws IOException {
   }
+
+  @Override
+  public RenameTracker initiateRenameOperation(final StoreContext storeContext,
+      final Path source,
+      final S3AFileStatus sourceStatus,
+      final Path dest)
+      throws IOException {
+    return new NullRenameTracker(storeContext, source, dest, this);
+  }
+
+  @Override
+  public void addAncestors(final Path qualifiedPath,
+      final ITtlTimeProvider timeProvider,
+      @Nullable final BulkOperationState operationState) throws IOException {
+  }
+
+  private static final class NullRenameTracker extends RenameTracker {
+
+    private NullRenameTracker(
+        final StoreContext storeContext,
+        final Path source,
+        final Path dest, MetadataStore metadataStore) {
+      super("null tracker", storeContext, metadataStore, source, dest, null);
+    }
+
+    @Override
+    public void fileCopied(final Path childSource,
+        final S3ObjectAttributes sourceAttributes,
+        final S3ObjectAttributes destAttributes,
+        final Path destPath,
+        final long blockSize,
+        final boolean addAncestors) throws IOException {
+
+    }
+
+  }
 }
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 c9559ec..7d4980a 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 @@ import org.apache.hadoop.fs.s3a.Tristate;
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-final class PathMetadataDynamoDBTranslation {
+@VisibleForTesting
+public final class PathMetadataDynamoDBTranslation {
 
   /** The HASH key name of each item. */
   @VisibleForTesting
@@ -289,7 +290,8 @@ final class PathMetadataDynamoDBTranslation {
    * @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();
@@ -343,9 +345,21 @@ final class PathMetadataDynamoDBTranslation {
   private PathMetadataDynamoDBTranslation() {
   }
 
+  /**
+   * Convert a collection of metadata entries to a list
+   * of DDBPathMetadata entries.
+   * If the sources are already DDBPathMetadata instances, they
+   * are copied directly into the new list, otherwise new
+   * instances are created.
+   * @param pathMetadatas source data
+   * @return the converted list.
+   */
   static List<DDBPathMetadata> pathMetaToDDBPathMeta(
-      Collection<PathMetadata> pathMetadatas) {
-    return pathMetadatas.stream().map(p -> new DDBPathMetadata(p))
+      Collection<? extends PathMetadata> pathMetadatas) {
+    return pathMetadatas.stream().map(p ->
+        (p instanceof DDBPathMetadata)
+            ? (DDBPathMetadata) p
+            : new DDBPathMetadata(p))
         .collect(Collectors.toList());
   }
 
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 0000000..a3a7967
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathOrderComparators.java
@@ -0,0 +1,133 @@
+/*
+ * 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.Serializable;
+import java.util.Comparator;
+
+import org.apache.hadoop.fs.Path;
+
+/**
+ * Comparator of path ordering for sorting collections.
+ *
+ * The definition of "topmost" is:
+ * <ol>
+ *   <li>The depth of a path is the primary comparator.</li>
+ *   <li>Root is topmost, "0"</li>
+ *   <li>If two paths are of equal depth, {@link Path#compareTo(Path)}</li>
+ *   is used. This delegates to URI compareTo.
+ *   <li>repeated sorts do not change the order</li>
+ * </ol>
+ */
+final class PathOrderComparators {
+
+  private PathOrderComparators() {
+  }
+
+  /**
+   * The shallowest paths come first.
+   * This is to be used when adding entries.
+   */
+  static final Comparator<Path> TOPMOST_PATH_FIRST
+      = new TopmostFirst();
+
+  /**
+   * The leaves come first.
+   * This is to be used when deleting entries.
+   */
+  static final Comparator<Path> TOPMOST_PATH_LAST
+      = new TopmostLast();
+
+  /**
+   * The shallowest paths come first.
+   * This is to be used when adding entries.
+   */
+  static final Comparator<PathMetadata> TOPMOST_PM_FIRST
+      = new PathMetadataComparator(TOPMOST_PATH_FIRST);
+
+  /**
+   * The leaves come first.
+   * This is to be used when deleting entries.
+   */
+  static final Comparator<PathMetadata> TOPMOST_PM_LAST
+      = new PathMetadataComparator(TOPMOST_PATH_LAST);
+
+  private static class TopmostFirst implements Comparator<Path>, Serializable {
+
+    @Override
+    public int compare(Path pathL, Path pathR) {
+      // exit 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);
+    }
+  }
+
+  /**
+   * Compare the topmost last.
+   * For some reason the .reverse() option wasn't giving the
+   * correct outcome.
+   */
+  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;
+    }
+  }
+
+  /**
+   * Compare on path status.
+   */
+  private static final class PathMetadataComparator implements
+      Comparator<PathMetadata>, Serializable {
+
+    private final Comparator<Path> inner;
+
+    private PathMetadataComparator(final Comparator<Path> 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 0000000..87cba20
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ProgressiveRenameTracker.java
@@ -0,0 +1,252 @@
+/*
+ * 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.Path;
+import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
+import org.apache.hadoop.fs.s3a.impl.StoreContext;
+import org.apache.hadoop.util.DurationInfo;
+
+import static com.google.common.base.Preconditions.checkArgument;
+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.
+ * <p>
+ * Algorithm
+ * <ol>
+ *   <li>
+ *     As {@code RenameTracker.fileCopied()} callbacks
+ *     are raised, the metastore is updated with the new file entry.
+ *   </li>
+ *   <li>
+ *     Including parent entries, as appropriate.
+ *   </li>
+ *   <li>
+ *     All directories which have been created are tracked locally,
+ *     to avoid needing to read the store; this is a thread-safe structure.
+ *   </li>
+ *   <li>
+ *    The actual update is performed out of any synchronized block.
+ *   </li>
+ *   <li>
+ *     When deletes are executed, the store is also updated.
+ *   </li>
+ *   <li>
+ *     And at the completion of a successful rename, the source directory
+ *     is also removed.
+ *   </li>
+ * </ol>
+ * <pre>
+ *
+ * </pre>
+ */
+public class ProgressiveRenameTracker extends RenameTracker {
+
+  /**
+   * The collection of paths to delete; this is added as individual files
+   * are renamed.
+   * <p>
+   * The metastore is only updated with these entries after the DELETE
+   * call containing these paths succeeds.
+   * <p>
+   * If the DELETE fails; the filesystem will use
+   * {@code MultiObjectDeleteSupport} to remove all successfully deleted
+   * entries from the metastore.
+   */
+  private final Collection<Path> pathsToDelete = new HashSet<>();
+
+  public ProgressiveRenameTracker(
+      final StoreContext storeContext,
+      final MetadataStore metadataStore,
+      final Path sourceRoot,
+      final Path dest,
+      final BulkOperationState operationState) {
+    super("ProgressiveRenameTracker",
+        storeContext, metadataStore, sourceRoot, dest, operationState);
+  }
+
+  /**
+   * When a file is copied, any ancestors
+   * are calculated and then the store is updated with
+   * the destination entries.
+   * <p>
+   * The source entries are added to the {@link #pathsToDelete} list.
+   * @param sourcePath path of source
+   * @param sourceAttributes status of source.
+   * @param destAttributes destination attributes
+   * @param destPath destination path.
+   * @param blockSize block size.
+   * @param addAncestors should ancestors be added?
+   * @throws IOException failure
+   */
+  @Override
+  public void fileCopied(
+      final Path sourcePath,
+      final S3ObjectAttributes sourceAttributes,
+      final S3ObjectAttributes destAttributes,
+      final Path destPath,
+      final long blockSize,
+      final boolean addAncestors) throws IOException {
+
+    // build the list of entries to add in a synchronized block.
+    final List<PathMetadata> 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 lists
+      // the pathsToDelete field is incremented with the new source path,
+      // for deletion after the DELETE operation succeeds;
+      // the entriesToAdd variable is filled in with all entries
+      // to add within this method
+      S3Guard.addMoveFile(
+          store,
+          pathsToDelete,
+          entriesToAdd,
+          sourcePath,
+          destPath,
+          sourceAttributes.getLen(),
+          blockSize,
+          getOwner(),
+          destAttributes.getETag(),
+          destAttributes.getVersionId());
+      LOG.debug("New metastore entry : {}", entriesToAdd.get(0));
+      if (addAncestors) {
+        // add all new ancestors to the lists
+        addMoveAncestors(
+            store,
+            pathsToDelete,
+            entriesToAdd,
+            getSourceRoot(),
+            sourcePath,
+            destPath,
+            getOwner());
+      }
+    }
+
+    // outside the lock, the entriesToAdd variable has all the new entries to
+    // create. ...so update the store.
+    // no entries are deleted at this point.
+    try (DurationInfo ignored = new DurationInfo(LOG, false,
+        "Adding new metastore entries")) {
+      store.move(null, entriesToAdd,
+          getStoreContext().getTimeProvider(),
+          getOperationState());
+    }
+  }
+
+  /**
+   * A directory marker has been added.
+   * Add the new entry and record the source path as another entry to delete.
+   * @param sourcePath status of source.
+   * @param destPath destination path.
+   * @param addAncestors should ancestors be added?
+   * @throws IOException failure.
+   */
+  @Override
+  public void directoryMarkerCopied(
+      final Path sourcePath,
+      final Path destPath,
+      final boolean addAncestors) throws IOException {
+    // this list is created on demand.
+    final List<PathMetadata> entriesToAdd = new ArrayList<>(1);
+    MetadataStore store = getMetadataStore();
+    synchronized (this) {
+      addMoveDir(store,
+          pathsToDelete,
+          entriesToAdd,
+          sourcePath,
+          destPath,
+          getOwner());
+      // Ancestor directories may not be listed, so we explicitly add them
+      if (addAncestors) {
+        addMoveAncestors(store,
+            pathsToDelete,
+            entriesToAdd,
+            getSourceRoot(),
+            sourcePath,
+            destPath,
+            getOwner());
+      }
+    }
+    // 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())) {
+      store.move(null, entriesToAdd,
+          getStoreContext().getTimeProvider(),
+          getOperationState());
+    }
+  }
+
+  @Override
+  public synchronized void moveSourceDirectory() throws IOException {
+    // this moves the source directory in the metastore if it has not
+    // already been processed.
+    // TODO S3Guard: performance: mark destination dirs as authoritative
+    if (!pathsToDelete.contains(getSourceRoot())) {
+      final List<Path> toDelete = new ArrayList<>(1);
+      final List<PathMetadata> toAdd = new ArrayList<>(1);
+
+      addMoveDir(getMetadataStore(), pathsToDelete, toAdd,
+          getSourceRoot(),
+          getDest(),
+          getOwner());
+      getMetadataStore().move(toDelete, toAdd,
+          getStoreContext().getTimeProvider(),
+          getOperationState());
+    }
+  }
+
+  /**
+   * As source objects are deleted, so is the list of entries.
+   * @param paths path of objects deleted.
+   * @throws IOException failure.
+   */
+  @Override
+  public void sourceObjectsDeleted(
+      final Collection<Path> paths) throws IOException {
+
+    // delete the paths from the metastore
+    try (DurationInfo ignored = new DurationInfo(LOG, false,
+        "delete %s metastore entries", paths.size())) {
+      getMetadataStore().move(paths, null,
+          getStoreContext().getTimeProvider(),
+          getOperationState());
+    }
+  }
+
+  @Override
+  public synchronized void completeRename() throws IOException {
+    // and finish off; by deleting source directories.
+    sourceObjectsDeleted(pathsToDelete);
+    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
new file mode 100644
index 0000000..76e269e
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RenameTracker.java
@@ -0,0 +1,275 @@
+/*
+ * 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.Collection;
+import java.util.List;
+
+import com.amazonaws.SdkBaseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
+import org.apache.hadoop.fs.s3a.impl.StoreContext;
+import org.apache.hadoop.fs.s3a.impl.AbstractStoreOperation;
+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;
+
+/**
+ * A class which manages updating the metastore with the rename process
+ * as initiated in the S3AFilesystem rename.
+ * <p>
+ * Subclasses must provide an implementation and return it in
+ * {@code MetadataStore.initiateRenameOperation()}.
+ * <p>
+ * 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.
+ * <p>
+ * 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 AbstractStoreOperation {
+
+  public static final Logger LOG = LoggerFactory.getLogger(
+      RenameTracker.class);
+
+  /** source path. */
+  private final Path sourceRoot;
+
+  /** destination path. */
+  private final Path dest;
+
+  /**
+   * Track the duration of this operation.
+   */
+  private final DurationInfo durationInfo;
+
+  /**
+   * Generated name for strings.
+   */
+  private final String name;
+
+  /**
+   * 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 BulkOperationState operationState;
+
+  /**
+   * The metadata store for this tracker.
+   * Always non-null.
+   * <p>
+   * 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 operationState ongoing move state.
+   */
+  protected RenameTracker(
+      final String name,
+      final StoreContext storeContext,
+      final MetadataStore metadataStore,
+      final Path sourceRoot,
+      final Path dest,
+      final BulkOperationState operationState) {
+    super(checkNotNull(storeContext));
+    checkNotNull(storeContext.getUsername(), "No username");
+    this.metadataStore = checkNotNull(metadataStore);
+    this.sourceRoot = checkNotNull(sourceRoot);
+    this.dest = checkNotNull(dest);
+    this.operationState = operationState;
+    this.name = String.format("%s (%s, %s)", name, sourceRoot, dest);
+    durationInfo = new DurationInfo(LOG, false,
+        name +" (%s, %s)", sourceRoot, dest);
+  }
+
+  @Override
+  public String toString() {
+    return name;
+  }
+
+  public Path getSourceRoot() {
+    return sourceRoot;
+  }
+
+  public Path getDest() {
+    return dest;
+  }
+
+  public String getOwner() {
+    return getStoreContext().getUsername();
+  }
+
+  public BulkOperationState getOperationState() {
+    return operationState;
+  }
+
+  /**
+   * Get the metadata store.
+   * @return a non-null store.
+   */
+  protected MetadataStore getMetadataStore() {
+    return metadataStore;
+  }
+
+  /**
+   * A file has been copied.
+   *
+   * @param childSource source of the file. This may actually be different
+   * from the path of the sourceAttributes. (HOW?)
+   * @param sourceAttributes status of source.
+   * @param destAttributes destination attributes
+   * @param destPath destination path.
+   * @param blockSize block size.
+   * @param addAncestors should ancestors be added?
+   * @throws IOException failure.
+   */
+  public abstract void fileCopied(
+      Path childSource,
+      S3ObjectAttributes sourceAttributes,
+      S3ObjectAttributes destAttributes,
+      Path destPath,
+      long blockSize,
+      boolean addAncestors) throws IOException;
+
+  /**
+   * A directory marker has been copied.
+   * @param sourcePath source path.
+   * @param destPath destination path.
+   * @param addAncestors should ancestors be added?
+   * @throws IOException failure.
+   */
+  public void directoryMarkerCopied(
+      Path sourcePath,
+      Path destPath,
+      boolean addAncestors) throws IOException {
+  }
+
+  /**
+   * The delete failed.
+   * <p>
+   * 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 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<Path> pathsToDelete,
+      final List<Path> undeletedObjects) {
+
+    return convertToIOException(e);
+  }
+
+  /**
+   * Top level directory move.
+   * This is invoked after all child entries have been copied
+   * @throws IOException on failure
+   */
+  public void moveSourceDirectory() throws IOException {
+  }
+
+  /**
+   * Note that source objects have been deleted.
+   * The metastore will already have been updated.
+   * @param paths path of objects deleted.
+   */
+  public void sourceObjectsDeleted(
+      final Collection<Path> paths) throws IOException {
+  }
+
+  /**
+   * Complete the operation.
+   * @throws IOException failure.
+   */
+  public void completeRename() throws IOException {
+    IOUtils.cleanupWithLogger(LOG, operationState);
+    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.
+   * <p>
+   * The metastore now needs to be updated with its current state
+   * even though the operation is incomplete.
+   * 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.
+   * <p>
+   * 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) {
+    LOG.debug("Rename has failed", ex);
+    IOUtils.cleanupWithLogger(LOG, operationState);
+    noteRenameFinished();
+    return convertToIOException(ex);
+  }
+
+  /**
+   * 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 completeness
+      return new IOException(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 933a01c..d462636 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
@@ -47,7 +47,6 @@ import org.apache.hadoop.fs.s3a.Retries;
 import org.apache.hadoop.fs.s3a.Retries.RetryTranslated;
 import org.apache.hadoop.fs.s3a.S3AFileStatus;
 import org.apache.hadoop.fs.s3a.S3AInstrumentation;
-import org.apache.hadoop.fs.s3a.Tristate;
 import org.apache.hadoop.util.ReflectionUtils;
 
 import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_METADATASTORE_METADATA_TTL;
@@ -132,7 +131,7 @@ public final class S3Guard {
     }
     if (conf.get(S3_METADATA_STORE_IMPL) != null && LOG.isDebugEnabled()) {
       LOG.debug("Metastore option source {}",
-          conf.getPropertySources(S3_METADATA_STORE_IMPL));
+          (Object)conf.getPropertySources(S3_METADATA_STORE_IMPL));
     }
 
     Class<? extends MetadataStore> aClass = conf.getClass(
@@ -157,15 +156,65 @@ public final class S3Guard {
       S3AFileStatus status,
       S3AInstrumentation instrumentation,
       ITtlTimeProvider timeProvider) throws IOException {
+    return putAndReturn(ms, status, instrumentation, timeProvider, 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 timeProvider Time provider to use when writing entries
+   * @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,
+      final ITtlTimeProvider timeProvider,
+      @Nullable final BulkOperationState operationState) throws IOException {
     long startTimeNano = System.nanoTime();
-    S3Guard.putWithTtl(ms, new PathMetadata(status), timeProvider);
-    instrumentation.addValueToQuantiles(S3GUARD_METADATASTORE_PUT_PATH_LATENCY,
-        (System.nanoTime() - startTimeNano));
-    instrumentation.incrementCounter(S3GUARD_METADATASTORE_PUT_PATH_REQUEST, 1);
+    try {
+      putWithTtl(ms, new PathMetadata(status), timeProvider, 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 write and create an operation state for it.
+   * This may then be passed into put operations.
+   * @param metastore store
+   * @param operation the type of the operation.
+   * @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 BulkOperationState.OperationType operation,
+      final Path path) throws IOException {
+    Preconditions.checkArgument(
+        operation != BulkOperationState.OperationType.Rename,
+        "Rename operations cannot be started through initiateBulkWrite");
+    if (metastore == null || isNullMetadataStore(metastore)) {
+      return null;
+    } else {
+      return metastore.initiateBulkWrite(operation, 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
    * an empty array is returned.
@@ -250,7 +299,7 @@ public final class S3Guard {
         if (status != null
             && s.getModificationTime() > status.getModificationTime()) {
           LOG.debug("Update ms with newer metadata of: {}", status);
-          S3Guard.putWithTtl(ms, new PathMetadata(s), timeProvider);
+          S3Guard.putWithTtl(ms, new PathMetadata(s), timeProvider, null);
         }
       }
 
@@ -271,7 +320,7 @@ public final class S3Guard {
 
     if (changed && isAuthoritative) {
       dirMeta.setAuthoritative(true); // This is the full directory contents
-      S3Guard.putWithTtl(ms, dirMeta, timeProvider);
+      S3Guard.putWithTtl(ms, dirMeta, timeProvider, null);
     }
 
     return dirMetaToStatuses(dirMeta);
@@ -308,7 +357,7 @@ public final class S3Guard {
    *              dir.
    * @param owner Hadoop user name.
    * @param authoritative Whether to mark new directories as authoritative.
-   * @param timeProvider Time provider for testing.
+   * @param timeProvider Time provider.
    */
   @Deprecated
   @Retries.OnceExceptionsSwallowed
@@ -357,7 +406,7 @@ public final class S3Guard {
             children.add(new PathMetadata(prevStatus));
           }
           dirMeta = new DirListingMetadata(f, children, authoritative);
-          S3Guard.putWithTtl(ms, dirMeta, timeProvider);
+          S3Guard.putWithTtl(ms, dirMeta, timeProvider, null);
         }
 
         pathMetas.add(new PathMetadata(status));
@@ -365,7 +414,7 @@ public final class S3Guard {
       }
 
       // Batched put
-      S3Guard.putWithTtl(ms, pathMetas, timeProvider);
+      S3Guard.putWithTtl(ms, pathMetas, timeProvider, null);
     } catch (IOException ioe) {
       LOG.error("MetadataStore#put() failure:", ioe);
     }
@@ -432,7 +481,7 @@ public final class S3Guard {
    * take care of those inferred directories of this path explicitly.
    *
    * As {@link #addMoveFile} and {@link #addMoveDir}, this method adds resulting
-   * metadata to the supplied lists. It does not store in MetadataStore.
+   * metadata to the supplied lists. It does not update the MetadataStore.
    *
    * @param ms MetadataStore, no-op if it is NullMetadataStore
    * @param srcPaths stores the source path here
@@ -469,25 +518,36 @@ public final class S3Guard {
     }
   }
 
-  public static void addAncestors(MetadataStore metadataStore,
-      Path qualifiedPath, String username, ITtlTimeProvider timeProvider)
-      throws IOException {
-    Collection<PathMetadata> newDirs = new ArrayList<>();
-    Path parent = qualifiedPath.getParent();
-    while (!parent.isRoot()) {
-      PathMetadata directory = metadataStore.get(parent);
-      if (directory == null || directory.isDeleted()) {
-        S3AFileStatus s3aStatus = new S3AFileStatus(Tristate.FALSE, parent, username);
-        PathMetadata meta = new PathMetadata(s3aStatus, Tristate.FALSE, false);
-        newDirs.add(meta);
-      } else {
-        break;
-      }
-      parent = parent.getParent();
-    }
-    S3Guard.putWithTtl(metadataStore, newDirs, timeProvider);
+  /**
+   * This adds all new ancestors of a path as directories.
+   * This forwards to
+   * {@link MetadataStore#addAncestors(Path, ITtlTimeProvider, BulkOperationState)}.
+   * <p>
+   * Originally it implemented the logic to probe for an add ancestors,
+   * but with the addition of a store-specific bulk operation state
+   * it became unworkable.
+   *
+   * @param metadataStore store
+   * @param qualifiedPath path to update
+   * @param operationState (nullable) operational state for a bulk update
+   * @throws IOException failure
+   */
+  @Retries.RetryTranslated
+  public static void addAncestors(
+      final MetadataStore metadataStore,
+      final Path qualifiedPath,
+      final ITtlTimeProvider timeProvider,
+      @Nullable final BulkOperationState operationState) throws IOException {
+    metadataStore.addAncestors(qualifiedPath, timeProvider, operationState);
   }
 
+  /**
+   * Add the fact that a file was moved from a source path to a destination.
+   * @param srcPaths collection of source paths to update
+   * @param dstMetas collection of destination meta data entries to update.
+   * @param srcPath path of the source file.
+   * @param dstStatus status of the source file after it was copied.
+   */
   private static void addMoveStatus(Collection<Path> srcPaths,
       Collection<PathMetadata> dstMetas,
       Path srcPath,
@@ -570,30 +630,72 @@ public final class S3Guard {
     }
   }
 
+  /**
+   * Put a directory entry, setting the updated timestamp of the
+   * directory and its children.
+   * @param ms metastore
+   * @param dirMeta directory
+   * @param timeProvider nullable time provider
+   * @throws IOException failure.
+   */
   public static void putWithTtl(MetadataStore ms, DirListingMetadata dirMeta,
-      ITtlTimeProvider timeProvider)
+      final ITtlTimeProvider timeProvider,
+      @Nullable final BulkOperationState operationState)
       throws IOException {
-    dirMeta.setLastUpdated(timeProvider.getNow());
+    long now = timeProvider.getNow();
+    dirMeta.setLastUpdated(now);
     dirMeta.getListing()
-        .forEach(pm -> pm.setLastUpdated(timeProvider.getNow()));
-    ms.put(dirMeta);
+        .forEach(pm -> pm.setLastUpdated(now));
+    ms.put(dirMeta, operationState);
   }
 
+  /**
+   * Put an entry, using the time provider to set its timestamp.
+   * @param ms metastore
+   * @param fileMeta entry to write
+   * @param timeProvider nullable time provider
+   * @param operationState nullable state for a bulk update
+   * @throws IOException failure.
+   */
   public static void putWithTtl(MetadataStore ms, PathMetadata fileMeta,
-      @Nullable ITtlTimeProvider timeProvider) throws IOException {
+      @Nullable ITtlTimeProvider timeProvider,
+      @Nullable final BulkOperationState operationState) throws IOException {
     if (timeProvider != null) {
       fileMeta.setLastUpdated(timeProvider.getNow());
     } else {
       LOG.debug("timeProvider is null, put {} without setting last_updated",
           fileMeta);
     }
-    ms.put(fileMeta);
+    ms.put(fileMeta, operationState);
   }
 
+  /**
+   * Put entries, using the time provider to set their timestamp.
+   * @param ms metastore
+   * @param fileMetas file metadata entries.
+   * @param timeProvider nullable time provider
+   * @param operationState nullable state for a bulk update
+   * @throws IOException failure.
+   */
   public static void putWithTtl(MetadataStore ms,
-      Collection<PathMetadata> fileMetas,
-      @Nullable ITtlTimeProvider timeProvider)
+      Collection<? extends PathMetadata> fileMetas,
+      @Nullable ITtlTimeProvider timeProvider,
+      @Nullable final BulkOperationState operationState)
       throws IOException {
+    patchLastUpdated(fileMetas, timeProvider);
+    ms.put(fileMetas, operationState);
+  }
+
+  /**
+   * Patch any collection of metadata entries with the timestamp
+   * of a time provider.
+   * This <i>MUST</i> be used when creating new entries for directories.
+   * @param fileMetas file metadata entries.
+   * @param timeProvider nullable time provider
+   */
+  static void patchLastUpdated(
+      final Collection<? extends PathMetadata> fileMetas,
+      @Nullable final ITtlTimeProvider timeProvider) {
     if (timeProvider != null) {
       final long now = timeProvider.getNow();
       fileMetas.forEach(fileMeta -> fileMeta.setLastUpdated(now));
@@ -601,9 +703,16 @@ public final class S3Guard {
       LOG.debug("timeProvider is null, put {} without setting last_updated",
           fileMetas);
     }
-    ms.put(fileMetas);
   }
 
+  /**
+   * Get a path entry provided it is not considered expired.
+   * @param ms metastore
+   * @param path path to look up.
+   * @param timeProvider nullable time provider
+   * @return the metadata or null if there as no entry.
+   * @throws IOException failure.
+   */
   public static PathMetadata getWithTtl(MetadataStore ms, Path path,
       @Nullable ITtlTimeProvider timeProvider) throws IOException {
     final PathMetadata pathMetadata = ms.get(path);
@@ -616,11 +725,11 @@ public final class S3Guard {
     long ttl = timeProvider.getMetadataTtl();
 
     if (pathMetadata != null) {
-      // Special case: the pathmetadata's last updated is 0. This can happen
+      // Special case: the path metadata's last updated is 0. This can happen
       // eg. with an old db using this implementation
       if (pathMetadata.getLastUpdated() == 0) {
         LOG.debug("PathMetadata TTL for {} is 0, so it will be returned as "
-            + "not expired.");
+            + "not expired.", path);
         return pathMetadata;
       }
 
@@ -636,6 +745,14 @@ public final class S3Guard {
     return null;
   }
 
+  /**
+   * List children; mark the result as non-auth if the TTL has expired.
+   * @param ms metastore
+   * @param path path to look up.
+   * @param timeProvider nullable time provider
+   * @return the listing of entries under a path, or null if there as no entry.
+   * @throws IOException failure.
+   */
   public static DirListingMetadata listChildrenWithTtl(MetadataStore ms,
       Path path, @Nullable ITtlTimeProvider timeProvider)
       throws IOException {
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 dedb849..002f793 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;
@@ -63,6 +64,7 @@ import org.apache.hadoop.util.ToolRunner;
 
 import static org.apache.hadoop.fs.s3a.Constants.*;
 import static org.apache.hadoop.fs.s3a.Invoker.LOG_EVENT;
+import static org.apache.hadoop.fs.s3a.S3AUtils.clearBucketOption;
 import static org.apache.hadoop.service.launcher.LauncherExitCodes.*;
 
 /**
@@ -650,7 +652,13 @@ public abstract class S3GuardTool extends Configured implements Tool {
       Preconditions.checkState(getStore() != null,
           "Metadata Store is not initialized");
 
-      getStore().destroy();
+      try {
+        getStore().destroy();
+      } catch (TableDeleteTimeoutException e) {
+        LOG.warn("The table is been deleted but it is still (briefly)"
+            + " listed as present in AWS");
+        LOG.debug("Timeout waiting for table disappearing", e);
+      }
       println(out, "Metadata store is deleted.");
       return SUCCESS;
     }
@@ -696,9 +704,11 @@ public abstract class S3GuardTool extends Configured implements Tool {
      * 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) {
@@ -708,7 +718,8 @@ public abstract class S3GuardTool extends Configured implements Tool {
         S3AFileStatus dir = DynamoDBMetadataStore.makeDirStatus(parent,
             f.getOwner());
         S3Guard.putWithTtl(getStore(), new PathMetadata(dir),
-            getFilesystem().getTtlTimeProvider());
+            getFilesystem().getTtlTimeProvider(),
+            operationState);
         dirCache.add(parent);
         parent = parent.getParent();
       }
@@ -721,6 +732,9 @@ public abstract class S3GuardTool extends Configured implements Tool {
      */
     private long importDir(FileStatus status) throws IOException {
       Preconditions.checkArgument(status.isDirectory());
+      BulkOperationState operationState = getStore().initiateBulkWrite(
+          BulkOperationState.OperationType.Put,
+          status.getPath());
       RemoteIterator<S3ALocatedFileStatus> it = getFilesystem()
           .listFilesAndEmptyDirectories(status.getPath(), true);
       long items = 0;
@@ -741,9 +755,11 @@ public abstract class S3GuardTool extends Configured implements Tool {
               located.getETag(),
               located.getVersionId());
         }
-        putParentsIfNotPresent(child);
-        S3Guard.putWithTtl(getStore(), new PathMetadata(child),
-            getFilesystem().getTtlTimeProvider());
+        putParentsIfNotPresent(child, operationState);
+        S3Guard.putWithTtl(getStore(),
+            new PathMetadata(child),
+            getFilesystem().getTtlTimeProvider(),
+            operationState);
         items++;
       }
       return items;
@@ -779,7 +795,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
       long items = 1;
       if (status.isFile()) {
         PathMetadata meta = new PathMetadata(status);
-        getStore().put(meta);
+        getStore().put(meta, null);
       } else {
         items = importDir(status);
       }
@@ -1137,16 +1153,19 @@ public abstract class S3GuardTool extends Configured implements Tool {
       }
       String s3Path = paths.get(0);
       CommandFormat commands = getCommandFormat();
+      URI fsURI = toUri(s3Path);
 
       // check if UNGUARDED_FLAG is passed and use NullMetadataStore in
       // config to avoid side effects like creating the table if not exists
+      Configuration unguardedConf = getConf();
       if (commands.getOpt(UNGUARDED_FLAG)) {
         LOG.debug("Unguarded flag is passed to command :" + this.getName());
-        getConf().set(S3_METADATA_STORE_IMPL, S3GUARD_METASTORE_NULL);
+        clearBucketOption(unguardedConf, fsURI.getHost(), S3_METADATA_STORE_IMPL);
+        unguardedConf.set(S3_METADATA_STORE_IMPL, S3GUARD_METASTORE_NULL);
       }
 
       S3AFileSystem fs = (S3AFileSystem) FileSystem.newInstance(
-          toUri(s3Path), getConf());
+          fsURI, unguardedConf);
       setFilesystem(fs);
       Configuration conf = fs.getConf();
       URI fsUri = fs.getUri();
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/TableDeleteTimeoutException.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/TableDeleteTimeoutException.java
new file mode 100644
index 0000000..7969332
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/TableDeleteTimeoutException.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.s3guard;
+
+import org.apache.hadoop.fs.PathIOException;
+
+/**
+ * An exception raised when a table being deleted is still present after
+ * the wait time is exceeded.
+ */
+public class TableDeleteTimeoutException extends PathIOException {
+
+  TableDeleteTimeoutException(final String path,
+      final String error,
+      final Throwable cause) {
+    super(path, error, cause);
+  }
+}
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 337fc95..2729a9e 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
@@ -1474,6 +1474,18 @@ Caused by: java.lang.NullPointerException
   ... 1 more
 ```
 
+### 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:
+   s3guard.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/contract/s3a/ITestS3AContractRename.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRename.java
index 4339649..b19b241 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 @@ import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeEnableS3Guard;
  */
 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
@@ -51,6 +69,12 @@ public class ITestS3AContractRename extends AbstractContractRenameTest {
   }
 
   @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"
              +" from the source dir into the existing dir"
@@ -64,12 +88,55 @@ public class ITestS3AContractRename extends AbstractContractRenameTest {
     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/ITestS3AFailureHandling.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java
index 8f8d860..55d396e 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 @@ package org.apache.hadoop.fs.s3a;
 
 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 org.slf4j.LoggerFactory;
 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 class ITestS3AFailureHandling extends AbstractS3ATestBase {
 
   private void removeKeys(S3AFileSystem fileSystem, String... keys)
       throws IOException {
+    fileSystem.removeKeys(buildDeleteRequest(keys), false);
+  }
+
+  private List<DeleteObjectsRequest.KeyVersion> buildDeleteRequest(
+      final String[] keys) {
     List<DeleteObjectsRequest.KeyVersion> 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,87 @@ public class ITestS3AFailureHandling extends AbstractS3ATestBase {
     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<DeleteObjectsRequest.KeyVersion> keys
+        = buildDeleteRequest(
+            new String[]{
+                fs.pathToKey(csvPath),
+                "missing-key.csv"
+            });
+    MultiObjectDeleteException ex = intercept(
+        MultiObjectDeleteException.class,
+        () -> fs.removeKeys(keys, false));
+
+    final List<Path> 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<Path>, List<Path>> pair =
+        new MultiObjectDeleteSupport(fs.createStoreContext())
+        .splitUndeletedKeys(ex, keys);
+    assertEquals(undeleted, pair.getLeft());
+    List<Path> 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.
+   * It must not raise an exception.
+   */
+  @Test
+  public void testMultiObjectDeleteMissingEntriesSucceeds() throws Throwable {
+    describe("Delete keys which don't exist");
+    Path base = path("missing");
+    S3AFileSystem fs = getFileSystem();
+    List<DeleteObjectsRequest.KeyVersion> keys = keysToDelete(
+        Lists.newArrayList(new Path(base, "1"), new Path(base, "2")));
+    fs.removeKeys(keys, 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/ITestS3AMetadataPersistenceException.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetadataPersistenceException.java
index 26661a3..3662194 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.FSDataOutputStream;
 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 @@ public class ITestS3AMetadataPersistenceException extends AbstractS3ATestBase {
     }
 
     @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/ITestS3GuardListConsistency.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java
index c90dd7c..10ebacd 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java
@@ -47,6 +47,7 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.writeTextFile;
 import static org.apache.hadoop.fs.s3a.Constants.*;
 import static org.apache.hadoop.fs.s3a.FailureInjectionPolicy.*;
 import static org.apache.hadoop.fs.s3a.InconsistentAmazonS3Client.*;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 
 /**
  * Test S3Guard list consistency feature by injecting delayed listObjects()
@@ -253,13 +254,11 @@ public class ITestS3GuardListConsistency extends AbstractS3ATestBase {
     assertFalse(list.contains(path("a3/b/dir3-" +
         DEFAULT_DELAY_KEY_SUBSTRING)));
 
-    try {
-      RemoteIterator<S3ALocatedFileStatus> old = fs.listFilesAndEmptyDirectories(
-          path("a"), true);
-      fail("Recently renamed dir should not be visible");
-    } catch(FileNotFoundException e) {
-      // expected
-    }
+    intercept(FileNotFoundException.class, "",
+        "Recently renamed dir should not be visible",
+        () -> S3AUtils.mapLocatedFiles(
+            fs.listFilesAndEmptyDirectories(path("a"), true),
+            FileStatus::getPath));
   }
 
   @Test
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java
index 0e091a9..99f1a07 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.RemoteIterator;
 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,7 +178,8 @@ public class MockS3AFileSystem extends S3AFileSystem {
   }
 
   @Override
-  void finishedWrite(String key, long length, String eTag, String versionId) {
+  void finishedWrite(String key, long length, String eTag, String versionId,
+          BulkOperationState operationState) {
 
   }
 
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java
index 8ca2ecc..3d7cdfc 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java
@@ -32,6 +32,7 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathIOException;
 import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy;
 import org.apache.hadoop.fs.s3a.impl.ChangeTracker;
@@ -58,6 +59,8 @@ public class TestStreamChangeTracker extends HadoopTestBase {
 
   public static final String URI = "s3a://" + BUCKET + "/" + OBJECT;
 
+  public static final Path PATH = new Path(URI);
+
   @Test
   public void testVersionCheckingHandlingNoVersions() throws Throwable {
     LOG.info("If an endpoint doesn't return versions, that's OK");
@@ -434,10 +437,12 @@ public class TestStreamChangeTracker extends HadoopTestBase {
   private S3ObjectAttributes objectAttributes(
       String etag, String versionId) {
     return new S3ObjectAttributes(BUCKET,
+        PATH,
         OBJECT,
         null,
         null,
         etag,
-        versionId);
+        versionId,
+        0);
   }
 }
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 1ac52c4..7261353 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.FileOutputStream;
 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.CommitOperations;
 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 class ITestAssumeRole extends AbstractS3ATestBase {
     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 class ITestAssumeRole extends AbstractS3ATestBase {
     }
   }
 
-  /**
-   * 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.
-   * <ol>
-   *   <li>The exception raised is AccessDeniedException,
-   *   from single and multi DELETE calls.</li>
-   *   <li>It happens after the COPY. Not ideal, but, well, we can't pretend
-   *   it's a filesystem forever.</li>
-   * </ol>
-   * @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");
@@ -779,12 +608,16 @@ public class ITestAssumeRole extends AbstractS3ATestBase {
       // all those commits must fail
       List<SinglePendingCommit> 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);
@@ -840,24 +673,25 @@ public class ITestAssumeRole extends AbstractS3ATestBase {
   @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 +722,4 @@ public class ITestAssumeRole extends AbstractS3ATestBase {
         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<String> 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 dbbaee5..8942d70 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,12 @@
 
 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.stream.Collectors;
+import java.util.stream.IntStream;
 
 import com.fasterxml.jackson.core.JsonProcessingException;
 import org.junit.Assume;
@@ -160,18 +164,35 @@ public final class RoleTestUtils {
 
   /**
    * Assert that an operation is forbidden.
+   * @param <T> 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 <T> AccessDeniedException forbidden(
+      final String contained,
+      final Callable<T> eval)
+      throws Exception {
+    return forbidden("", contained, eval);
+  }
+
+  /**
+   * Assert that an operation is forbidden.
    * @param <T> 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 <T> AccessDeniedException forbidden(
-      String contained,
-      Callable<T> eval)
+      final String message,
+      final String contained,
+      final Callable<T> eval)
       throws Exception {
     return intercept(AccessDeniedException.class,
-        contained, eval);
+        contained, message, eval);
   }
 
   /**
@@ -209,4 +230,23 @@ public final class RoleTestUtils {
         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<Path> touchFiles(final FileSystem fs,
+      final Path destDir,
+      final int range) throws IOException {
+    List<Path> 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/commit/AbstractCommitITest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java
index ef594e6..e8645b8 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java
@@ -205,6 +205,7 @@ public abstract class AbstractCommitITest extends AbstractS3ATestBase {
    */
   @Override
   public void teardown() throws Exception {
+    LOG.info("AbstractCommitITest::teardown");
     waitForConsistency();
     // make sure there are no failures any more
     resetFailures();
@@ -495,6 +496,7 @@ public abstract class AbstractCommitITest extends AbstractS3ATestBase {
   public static SuccessData loadSuccessFile(final S3AFileSystem fs,
       final Path outputPath) throws IOException {
     Path success = new Path(outputPath, _SUCCESS);
+    ContractTestUtils.assertIsFile(fs, success);
     FileStatus status = fs.getFileStatus(success);
     assertTrue("0 byte success file - not a s3guard committer " + success,
         status.getLen() > 0);
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitMRJob.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitMRJob.java
index 682931d..1fb3d89 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitMRJob.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitMRJob.java
@@ -71,6 +71,9 @@ public abstract class AbstractITCommitMRJob extends AbstractYarnClusterITest {
     S3AFileSystem fs = getFileSystem();
     // final dest is in S3A
     Path outputPath = path(getMethodName());
+    // create and delete to force in a tombstone marker -see HADOOP-16207
+    fs.mkdirs(outputPath);
+    fs.delete(outputPath, true);
 
     String commitUUID = UUID.randomUUID().toString();
     String suffix = isUniqueFilenames() ? ("-" + commitUUID) : "";
@@ -116,6 +119,7 @@ public abstract class AbstractITCommitMRJob extends AbstractYarnClusterITest {
       String sysprops = String.format("-Xmx256m -Dlog4j.configuration=%s",
           log4j);
       jobConf.set(JobConf.MAPRED_MAP_TASK_JAVA_OPTS, sysprops);
+      jobConf.set(JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS, sysprops);
       jobConf.set("yarn.app.mapreduce.am.command-opts", sysprops);
     }
 
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractYarnClusterITest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractYarnClusterITest.java
index 45f0738..2501662 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractYarnClusterITest.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractYarnClusterITest.java
@@ -71,7 +71,7 @@ public abstract class AbstractYarnClusterITest extends AbstractCommitITest {
       LoggerFactory.getLogger(AbstractYarnClusterITest.class);
 
   private static final int TEST_FILE_COUNT = 2;
-  private static final int SCALE_TEST_FILE_COUNT = 20;
+  private static final int SCALE_TEST_FILE_COUNT = 50;
 
   public static final int SCALE_TEST_KEYS = 1000;
   public static final int BASE_TEST_KEYS = 10;
@@ -138,6 +138,12 @@ public abstract class AbstractYarnClusterITest extends AbstractCommitITest {
     return new ClusterBinding(miniDFSClusterService, yarnCluster);
   }
 
+  protected static void terminateCluster(ClusterBinding clusterBinding) {
+    if (clusterBinding != null) {
+      clusterBinding.terminate();
+    }
+  }
+
   /**
    * Get the cluster binding for this subclass
    * @return
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 2886a99..d453715 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 @@ package org.apache.hadoop.fs.s3a.commit;
 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.FileStatus;
 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;
@@ -268,12 +271,17 @@ public class ITestCommitOperations extends AbstractCommitITest {
   public void testBaseRelativePath() throws Throwable {
     describe("Test creating file with a __base marker and verify that it ends" +
         " up in where expected");
+    S3AFileSystem fs = getFileSystem();
     Path destDir = methodPath("testBaseRelativePath");
+    fs.delete(destDir, true);
     Path pendingBaseDir = new Path(destDir, MAGIC + "/child/" + BASE);
     String child = "subdir/child.txt";
     Path pendingChildPath = new Path(pendingBaseDir, child);
     Path expectedDestPath = new Path(destDir, child);
-    createFile(getFileSystem(), pendingChildPath, true, DATASET);
+    assertPathDoesNotExist("dest file was found before upload",
+        expectedDestPath);
+
+    createFile(fs, pendingChildPath, true, DATASET);
     commit("child.txt", pendingChildPath, expectedDestPath, 0, 0);
   }
 
@@ -281,7 +289,9 @@ public class ITestCommitOperations extends AbstractCommitITest {
       throws Exception {
     S3AFileSystem fs = getFileSystem();
     Path destFile = methodPath(filename);
+    fs.delete(destFile.getParent(), true);
     Path magicDest = makeMagic(destFile);
+    assertPathDoesNotExist("Magic file should not exist", magicDest);
     try(FSDataOutputStream stream = fs.create(magicDest, true)) {
       assertTrue(stream.hasCapability(STREAM_CAPABILITY_MAGIC_OUTPUT));
       if (data != null && data.length > 0) {
@@ -332,13 +342,21 @@ public class ITestCommitOperations extends AbstractCommitITest {
     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
@@ -348,7 +366,7 @@ public class ITestCommitOperations extends AbstractCommitITest {
   private void validateIntermediateAndFinalPaths(Path magicFilePath,
       Path destFile)
       throws IOException {
-    assertPathDoesNotExist("dest file was created", destFile);
+    assertPathDoesNotExist("dest file was found", destFile);
   }
 
   /**
@@ -439,7 +457,7 @@ public class ITestCommitOperations extends AbstractCommitITest {
     resetFailures();
     assertPathDoesNotExist("pending commit", dest);
     fullThrottle();
-    actions.commitOrFail(pendingCommit);
+    commitOrFail(dest, pendingCommit, actions);
     resetFailures();
     FileStatus status = verifyPathExists(fs,
         "uploaded file commit", dest);
@@ -454,15 +472,19 @@ public class ITestCommitOperations extends AbstractCommitITest {
     CommitOperations actions = newCommitOperations();
     Path dest = methodPath("testUploadSmallFile");
     S3AFileSystem fs = getFileSystem();
+    fs.delete(dest, true);
     fullThrottle();
+    assertPathDoesNotExist("test setup", dest);
     SinglePendingCommit pendingCommit =
         actions.uploadFileToPendingCommit(tempFile,
             dest, null,
             DEFAULT_MULTIPART_SIZE);
     resetFailures();
+    LOG.debug("Precommit validation");
     assertPathDoesNotExist("pending commit", dest);
     fullThrottle();
-    actions.commitOrFail(pendingCommit);
+    LOG.debug("Postcommit validation");
+    commitOrFail(dest, pendingCommit, actions);
     resetFailures();
     String s = readUTF8(fs, dest, -1);
     assertEquals(text, s);
@@ -544,4 +566,97 @@ public class ITestCommitOperations extends AbstractCommitITest {
     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.
+   * As changes to the store may be made during get/list calls,
+   * when the counters must be reset before each commit, this must be
+   * *after* all probes for the outcome of the previous operation.
+   */
+  @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("out");
+    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<Path> destinations = Lists.newArrayList(destFile1, destFile2,
+        destFile3);
+    List<SinglePendingCommit> 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);
+    LOG.info("Initiating commit operations");
+    try (CommitOperations.CommitContext commitContext
+             = actions.initiateCommitOperation(destDir)) {
+      // how many records have been written
+      MetricDiff writes = new MetricDiff(fs,
+          Statistic.S3GUARD_METADATASTORE_RECORD_WRITES);
+      LOG.info("Commit #1");
+      commitContext.commitOrFail(commits.get(0));
+      final String firstCommitContextString = commitContext.toString();
+      LOG.info("First Commit state {}", firstCommitContextString);
+      long writesOnFirstCommit = writes.diff();
+      assertPathExists("destFile1", destFile1);
+      assertPathExists("destination dir", destDir);
+
+      LOG.info("Commit #2");
+      writes.reset();
+      commitContext.commitOrFail(commits.get(1));
+      assertPathExists("subdirectory", subdir);
+      assertPathExists("destFile2", destFile2);
+      final String secondCommitContextString = commitContext.toString();
+      LOG.info("Second Commit state {}", secondCommitContextString);
+
+      if (writesOnFirstCommit != 0) {
+        LOG.info("DynamoDB Metastore is in use: checking write count");
+        // 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.
+        // we include the string values of the contexts because that includes
+        // the internals of the bulk operation state.
+        writes.assertDiffEquals("Number of records written after commit #2"
+                + "; first commit had " + writesOnFirstCommit
+                + "; first commit ancestors " + firstCommitContextString
+                + "; second commit ancestors: " + secondCommitContextString,
+            2);
+      }
+
+      LOG.info("Commit #3");
+      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/commit/magic/ITestMagicCommitMRJob.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitMRJob.java
index a9b9c2c..e403ab4 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,23 @@
 
 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.FileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
 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.S3ATestUtils.lsR;
+import static org.apache.hadoop.fs.s3a.S3AUtils.applyLocatedFiles;
 import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 
 /**
  * Full integration test for the Magic Committer.
@@ -55,7 +61,7 @@ public final class ITestMagicCommitMRJob extends AbstractITCommitMRJob {
 
   @AfterClass
   public static void teardownClusters() throws IOException {
-    clusterBinding.terminate();
+    terminateCluster(clusterBinding);
   }
 
   @Override
@@ -93,6 +99,22 @@ public final class ITestMagicCommitMRJob extends AbstractITCommitMRJob {
   @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
+    // tree
+    S3AFileSystem fs = getFileSystem();
+    // log the contents
+    lsR(fs, destPath, true);
+    intercept(FileNotFoundException.class, () -> {
+      final FileStatus st = fs.getFileStatus(magicDir);
+      StringBuilder result = new StringBuilder("Found magic dir which should"
+          + " have been deleted at ").append(st).append('\n');
+      result.append("[");
+      applyLocatedFiles(fs.listFiles(magicDir, true),
+          (status) -> result.append(status.getPath()).append('\n'));
+      result.append("[");
+      return result.toString();
+    });
   }
 }
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 0722959..9a2ad0e 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 class ITestS3AHugeMagicCommits extends AbstractSTestS3AHugeFiles {
     assertNotNull("jobDir", jobDir);
     Pair<PendingSet, List<Pair<LocatedFileStatus, IOException>>>
         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/commit/staging/integration/ITestDirectoryCommitMRJob.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestDirectoryCommitMRJob.java
index 8d44ddb..1e44086 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestDirectoryCommitMRJob.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestDirectoryCommitMRJob.java
@@ -46,7 +46,7 @@ public final class ITestDirectoryCommitMRJob extends AbstractITCommitMRJob {
 
   @AfterClass
   public static void teardownClusters() throws IOException {
-    clusterBinding.terminate();
+    terminateCluster(clusterBinding);
   }
 
   @Override
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestPartitionCommitMRJob.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestPartitionCommitMRJob.java
index f71479c..6106974 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestPartitionCommitMRJob.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestPartitionCommitMRJob.java
@@ -47,7 +47,7 @@ public final class ITestPartitionCommitMRJob extends AbstractITCommitMRJob {
 
   @AfterClass
   public static void teardownClusters() throws IOException {
-    clusterBinding.terminate();
+    terminateCluster(clusterBinding);
   }
 
   @Override
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitMRJob.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitMRJob.java
index d4a351f..218c72a 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitMRJob.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitMRJob.java
@@ -58,7 +58,7 @@ public final class ITestStagingCommitMRJob extends AbstractITCommitMRJob {
 
   @AfterClass
   public static void teardownClusters() throws IOException {
-    clusterBinding.terminate();
+    terminateCluster(clusterBinding);
   }
 
   @Override
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitMRJobBadDest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitMRJobBadDest.java
index 68926f9..7248813 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitMRJobBadDest.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitMRJobBadDest.java
@@ -53,7 +53,7 @@ public final class ITestStagingCommitMRJobBadDest extends AbstractITCommitMRJob
 
   @AfterClass
   public static void teardownClusters() throws IOException {
-    clusterBinding.terminate();
+    terminateCluster(clusterBinding);
   }
 
   @Override
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/terasort/AbstractCommitTerasortIT.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/terasort/AbstractCommitTerasortIT.java
index 491ecb9..7db3068 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/terasort/AbstractCommitTerasortIT.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/terasort/AbstractCommitTerasortIT.java
@@ -238,4 +238,9 @@ public abstract class AbstractCommitTerasortIT extends
   public void test_150_teracleanup() throws Throwable {
     terasortDuration = Optional.empty();
   }
+
+  @Test
+  public void test_200_directory_deletion() throws Throwable {
+    getFileSystem().delete(terasortPath, true);
+  }
 }
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 0000000..942f0b6
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestPartialRenamesDeletes.java
@@ -0,0 +1,871 @@
+/*
+ * 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.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+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;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+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;
+import org.apache.hadoop.fs.s3a.AbstractS3ATestBase;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+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.*;
+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.*;
+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.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;
+import static org.apache.hadoop.fs.s3a.test.ExtraAssertions.extractCause;
+import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
+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
+ *
+ * <ol>
+ *   <li>
+ *     The tests are parameterized to single/multi delete, which control which
+ *     of the two delete mechanisms are used.
+ *   </li>
+ *   <li>
+ *     In multi delete, in a scale test run, a significantly larger set of files
+ *     is created and then deleted.
+ *   </li>
+ *   <li>
+ *     This isn't done in the single delete as it is much slower and it is not
+ *     the situation we are trying to create.
+ *   </li>
+ * </ol>
+ *
+ * This test manages to create lots of load on the s3guard prune command
+ * when that is tested in a separate test suite;
+ * too many tombstone files for the test to complete.
+ * An attempt is made in {@link #deleteTestDirInTeardown()} to prune these test
+ * files.
+ */
+@SuppressWarnings("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);
+
+  /** 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");
+
+
+  /**
+   * The number of files in a non-scaled test.
+   * <p>
+   * Value: {@value}.
+   */
+  public static final int FILE_COUNT_NON_SCALED = 2;
+
+  /**
+   * 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.
+   * <p>
+   * Value: {@value}.
+   */
+  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.
+   * This is generated uniquely for each test.
+   */
+  private Path basePath;
+
+  /**
+   * A directory which restricted roles have full write access to.
+   */
+  private Path writableDir;
+
+  /**
+   * A directory to which restricted roles have only read access.
+   */
+  private Path readOnlyDir;
+
+  /**
+   * A file under {@link #readOnlyDir} which cannot be written or deleted.
+   */
+  private Path readOnlyChild;
+
+  /**
+   * A directory to which restricted roles have no read access.
+   */
+  private Path noReadDir;
+
+  /** delete policy: single or multi? */
+  private final boolean multiDelete;
+
+  /**
+   * Configuration for the assume role FS.
+   */
+  private Configuration assumedRoleConfig;
+
+  private int fileCount;
+  private int dirCount;
+  private int dirDepth;
+
+  /**
+   * Was the -Dscale switch passed in to the test run?
+   */
+  private boolean scaleTest;
+
+  /**
+   * Test array for parameterized test runs.
+   * <ul>
+   *   <li>Run 0: single deletes</li>
+   *   <li>Run 1: multi deletes</li>
+   * </ul>
+   *
+   * @return a list of parameter tuples.
+   */
+  @Parameterized.Parameters(name = "bulk-delete={0}")
+  public static Collection<Object[]> params() {
+    return Arrays.asList(new Object[][]{
+        {false},
+        {true},
+    });
+  }
+
+  /**
+   * Constructor.
+   * @param multiDelete single vs multi delete in the role FS?
+   */
+  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");
+    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(writableDir);
+
+    // create the baseline assumed role
+    assumedRoleConfig = createAssumedRoleConfig();
+    bindRolePolicyStatements(assumedRoleConfig,
+        STATEMENT_S3GUARD_CLIENT,
+        STATEMENT_ALL_BUCKET_READ_ACCESS,  // root:     r-x
+        new Statement(Effects.Allow)       // dest:     rwx
+            .addActions(S3_PATH_RW_OPERATIONS)
+            .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);
+
+    // 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);
+    fileCount = scaleTest ? FILE_COUNT_SCALED : FILE_COUNT_NON_SCALED;
+    dirCount = scaleTest ? DIR_COUNT_SCALED : DIR_COUNT;
+    dirDepth = scaleTest ? DEPTH_SCALED : DEPTH;
+  }
+
+  @Override
+  public void teardown() throws Exception {
+    cleanupWithLogger(LOG, roleFS);
+    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();
+    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());
+  }
+
+  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);
+    String bucketName = getTestBucketName(conf);
+
+    removeBucketOverrides(bucketName, conf, ENABLE_MULTI_DELETE);
+    conf.setBoolean(ENABLE_MULTI_DELETE, multiDelete);
+    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,
+        S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY);
+    conf.setInt(MAX_THREADS, EXECUTOR_THREAD_COUNT);
+    conf.setInt(MAXIMUM_CONNECTIONS, EXECUTOR_THREAD_COUNT * 2);
+    // 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;
+  }
+
+  /**
+   * Create a unique path, which includes method name,
+   * multidelete 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,
+        "",
+        () -> {
+          touch(roleFS, readOnlyChild);
+          return 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
+  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.
+   */
+  @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(writableDir)));
+    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(writableDir, true);
+    // as dest doesn't exist, this will map child -> dest
+    assertRenameOutcome(roleFS, readOnlyChild, writableDir, true);
+
+    assertIsFile(writableDir);
+    assertIsDirectory(readOnlyDir);
+    Path renamedDestPath = new Path(readOnlyDir, writableDir.getName());
+    assertRenameOutcome(roleFS, writableDir, readOnlyDir, true);
+    assertIsFile(renamedDestPath);
+    roleFS.delete(readOnlyDir, true);
+    roleFS.delete(writableDir, true);
+  }
+
+  @Test
+  public void testRenameSingleFileFailsInDelete() 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(writableDir, true);
+    roleFS.mkdirs(writableDir);
+    // rename will fail in the delete phase
+    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(writableDir, readOnlyFile.getName());
+
+    assertIsFile(renamedFile);
+
+    ContractTestUtils.assertDeleted(roleFS, renamedFile, true);
+    assertFileCount("Empty Dest Dir", roleFS,
+        writableDir, 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 the test
+   * explores the outcome on bulk delete calls, including verifying that a
+   * MultiObjectDeleteException is translated to an AccessDeniedException.
+   * <ol>
+   *   <li>The exception raised is AccessDeniedException,
+   *   from single and multi DELETE calls.</li>
+   *   <li>It happens after the COPY. Not ideal, but, well, we can't pretend
+   *   it's a filesystem forever.</li>
+   * </ol>
+   */
+  @Test
+  public void testRenameDirFailsInDelete() throws Throwable {
+    describe("rename with source read only; multi=%s", multiDelete);
+
+    // the full FS
+    S3AFileSystem fs = getFileSystem();
+
+    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<Path> createdFiles = createFiles(fs, readOnlyDir, dirDepth, fileCount,
+        dirCount);
+    // are they all there?
+    int expectedFileCount = createdFiles.size();
+    assertFileCount("files ready to rename", roleFS,
+        readOnlyDir, expectedFileCount);
+
+    // try to rename the directory
+    LOG.info("Renaming readonly files {} to {}", readOnlyDir, writableDir);
+
+    AccessDeniedException deniedException = expectRenameForbidden(readOnlyDir,
+        writableDir);
+    if (multiDelete) {
+      // look in that exception for a multidelete
+      MultiObjectDeleteException mde = extractCause(
+          MultiObjectDeleteException.class, deniedException);
+      final List<Path> undeleted
+          = extractUndeletedPaths(mde, fs::keyToQualifiedPath);
+      Assertions.assertThat(undeleted)
+          .as("files which could not be deleted")
+          .hasSize(expectedFileCount)
+          .containsAll(createdFiles)
+          .containsExactlyInAnyOrderElementsOf(createdFiles);
+    }
+    LOG.info("Result of renaming read-only files is as expected",
+        deniedException);
+    assertFileCount("files in the source directory", roleFS,
+        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
+    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);
+  }
+
+  @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();
+    Path source = new Path(writableDir, "source");
+    touch(fs, source);
+    fs.mkdirs(readOnlyDir);
+    Path dest = new Path(readOnlyDir, "dest");
+    describe("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 testCopyDirFailsToReadOnlyDir() throws Throwable {
+    describe("Try to copy to a read-only destination");
+    roleFS.mkdirs(writableDir);
+    S3AFileSystem fs = getFileSystem();
+    List<Path> 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,
+        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");
+    S3AFileSystem fs = getFileSystem();
+    List<Path> 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());
+  }
+
+  /**
+   * Have a directory with full R/W permissions, but then remove
+   * write access underneath, and try to delete it.
+   * This verifies that failures in the delete fake dir stage.
+   * are not visible.
+   */
+  @Test
+  public void testPartialEmptyDirDelete() throws Throwable {
+    describe("delete an empty directory with parent dir r/o"
+        + " multidelete=%s", multiDelete);
+
+    // the full FS
+    final Path deletableChild = new Path(writableDir, "deletableChild");
+    // deletable child is created.
+    roleFS.mkdirs(deletableChild);
+    assertPathExists("parent dir after create", writableDir);
+    assertPathExists("grandparent dir after create", writableDir.getParent());
+    // now delete it.
+    roleFS.delete(deletableChild, true);
+    assertPathExists("parent dir after deletion", writableDir);
+    assertPathExists("grandparent dir after deletion", writableDir.getParent());
+    assertPathDoesNotExist("deletable dir after deletion", deletableChild);
+  }
+
+  /**
+   * Have a directory with full R/W permissions, but then remove
+   * write access underneath, and try to delete it.
+   */
+  @Test
+  public void testPartialDirDelete() throws Throwable {
+    describe("delete with part of the child tree read only;"
+            + " multidelete=%s", multiDelete);
+
+    // the full FS
+    S3AFileSystem fs = getFileSystem();
+
+    List<Path> readOnlyFiles = createFiles(fs, readOnlyDir,
+        dirDepth, fileCount, dirCount);
+    List<Path> 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);
+    assertTrue("delete " + head + " failed",
+        roleFS.delete(head, false));
+    List<Path> 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");
+    AccessDeniedException ex = expectDeleteForbidden(readOnlyDir);
+    if (multiDelete) {
+      // multi-delete status checks
+      extractCause(MultiObjectDeleteException.class, ex);
+      rejectionCount.assertDiffEquals("Wrong rejection count",
+          readOnlyFiles.size());
+      deleteVerbCount.assertDiffEquals("Wrong delete count", 1);
+      reset(rejectionCount, deleteVerbCount);
+    }
+    // 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);
+    if (multiDelete) {
+      // multi-delete status checks
+      extractCause(MultiObjectDeleteException.class, ex);
+      deleteVerbCount.assertDiffEquals("Wrong delete count", 1);
+      MultiObjectDeleteException mde = extractCause(
+          MultiObjectDeleteException.class, ex);
+      final List<Path> 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",
+          readOnlyFiles.size());
+    }
+    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<Path> readOnlyListing = listFilesUnderPath(readOnlyDir, true);
+
+    String directoryList = readOnlyListing.stream()
+        .map(Path::toString)
+        .collect(Collectors.joining(", ", "[", "]"));
+
+    Assertions.assertThat(readOnlyListing)
+        .as("ReadOnly directory " + directoryList)
+        .containsAll(readOnlyFiles);
+
+    // do this prune in the test as well as teardown, so that the test
+    // reporting includes it in the runtime of a successful run.
+    prune(basePath);
+  }
+
+  /**
+   * 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)) {
+      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 using the roleFS.
+   * @param src source
+   * @param dest dest
+   * @return the exception raised.
+   * @throws Exception any other failure
+   */
+  private AccessDeniedException expectRenameForbidden(Path src, Path dest)
+      throws Exception {
+    try (DurationInfo ignored =
+            new DurationInfo(LOG, true,
+                "rename(%s, %s)", src, dest)) {
+      return forbidden(
+          "Renaming " + src + " to " + dest,
+          "",
+          () -> {
+            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;
+          });
+    }
+  }
+
+  /**
+   * 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));
+  }
+
+  /**
+   * 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(
+            MetadataStore.PruneMode.ALL_BY_MODTIME,
+            System.currentTimeMillis(),
+            PathMetadataDynamoDBTranslation.pathToParentKey(fs.qualify(path)));
+      }
+    }
+  }
+
+  /**
+   * 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<Path> listFilesUnderPath(Path path, boolean recursive)
+      throws IOException {
+    Set<Path> files = new TreeSet<>();
+    try (DurationInfo ignore =
+             new DurationInfo(LOG, "ls -R %s", path)) {
+      applyLocatedFiles(getFileSystem().listFiles(path, recursive),
+          (status) -> files.add(status.getPath()));
+    }
+    return files;
+  }
+
+  /**
+   * Write the text to a file asynchronously. Logs the operation duration.
+   * @param fs filesystem
+   * @param path path
+   * @return future to the patch created.
+   */
+  private static CompletableFuture<Path> put(FileSystem fs,
+      Path path, String text) {
+    return submit(EXECUTOR, () -> {
+      try (DurationInfo ignore =
+               new DurationInfo(LOG, false, "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 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<Path> createFiles(final FileSystem fs,
+      final Path destDir,
+      final int depth,
+      final int fileCount,
+      final int dirCount) throws IOException {
+    List<CompletableFuture<Path>> futures = new ArrayList<>(fileCount);
+    List<Path> paths = new ArrayList<>(fileCount);
+    List<Path> dirs = new ArrayList<>(fileCount);
+    buildPaths(paths, dirs, destDir, depth, fileCount, dirCount);
+    try (DurationInfo ignore =
+            new DurationInfo(LOG, "Creating %d files", fileCount)) {
+      for (Path path : paths) {
+        futures.add(put(fs, path, path.getName()));
+      }
+      waitForCompletion(futures);
+      return paths;
+    }
+  }
+
+  /**
+   * 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<Path> filePaths,
+      final List<Path> 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
new file mode 100644
index 0000000..c44fa00
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java
@@ -0,0 +1,393 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.impl;
+
+import javax.annotation.Nullable;
+import java.io.File;
+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.Constants;
+import org.apache.hadoop.fs.s3a.Invoker;
+import org.apache.hadoop.fs.s3a.S3AFileStatus;
+import org.apache.hadoop.fs.s3a.S3AInputPolicy;
+import org.apache.hadoop.fs.s3a.S3AInstrumentation;
+import org.apache.hadoop.fs.s3a.S3AStorageStatistics;
+import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
+import org.apache.hadoop.fs.s3a.s3guard.DirListingMetadata;
+import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider;
+import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
+import org.apache.hadoop.fs.s3a.s3guard.PathMetadata;
+import org.apache.hadoop.fs.s3a.s3guard.RenameTracker;
+import org.apache.hadoop.fs.s3a.s3guard.S3Guard;
+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.removeUndeletedPaths;
+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 static final ContextAccessors CONTEXT_ACCESSORS
+      = new MinimalContextAccessor();
+
+  private StoreContext context;
+
+  private static Path qualifyKey(String k) {
+    return new Path("s3a://bucket/" + k);
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    context = createMockStoreContext(true,
+        new OperationTrackingStore());
+  }
+
+  @Test
+  public void testDeleteExtraction() {
+    List<Path> src = pathList("a", "a/b", "a/c");
+    List<Path> rejected = pathList("a/b");
+    MultiObjectDeleteException ex = createDeleteException(ACCESS_DENIED,
+        rejected);
+    List<Path> undeleted = removeUndeletedPaths(ex, src,
+        TestPartialDeleteFailures::qualifyKey);
+    assertEquals("mismatch of rejected and undeleted entries",
+        rejected, undeleted);
+  }
+
+  @Test
+  public void testSplitKeysFromResults() throws Throwable {
+    List<Path> src = pathList("a", "a/b", "a/c");
+    List<Path> rejected = pathList("a/b");
+    List<DeleteObjectsRequest.KeyVersion> keys = keysToDelete(src);
+    MultiObjectDeleteException ex = createDeleteException(ACCESS_DENIED,
+        rejected);
+    Pair<List<Path>, List<Path>> pair =
+        new MultiObjectDeleteSupport(context)
+          .splitUndeletedKeys(ex, keys);
+    List<Path> undeleted = pair.getLeft();
+    List<Path> 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 qualified paths from vararg parameters.
+   * @param paths paths to qualify and then convert to a lst.
+   * @return same paths as a list.
+   */
+  private List<Path> pathList(String... paths) {
+    return Arrays.stream(paths)
+        .map(TestPartialDeleteFailures::qualifyKey)
+        .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<Path> rejected) {
+    List<MultiObjectDeleteException.DeleteError> 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<DeleteObjectsRequest.KeyVersion> keysToDelete(
+      List<Path> 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 = qualifyKey("/a");
+    Path pathAB = qualifyKey("/a/b");
+    Path pathAC = qualifyKey("/a/c");
+    List<Path> src = Lists.newArrayList(pathA, pathAB, pathAC);
+    List<DeleteObjectsRequest.KeyVersion> keyList = keysToDelete(src);
+    List<Path> deleteForbidden = Lists.newArrayList(pathAB);
+    final List<Path> deleteAllowed = Lists.newArrayList(pathA, pathAC);
+    MultiObjectDeleteException ex = createDeleteException(ACCESS_DENIED,
+        deleteForbidden);
+    OperationTrackingStore store
+        = new OperationTrackingStore();
+    StoreContext storeContext = createMockStoreContext(true, store);
+    MultiObjectDeleteSupport deleteSupport
+        = new MultiObjectDeleteSupport(storeContext);
+    Triple<List<Path>, List<Path>, List<Pair<Path, IOException>>>
+        triple = deleteSupport.processDeleteFailure(ex, keyList);
+    Assertions.assertThat(triple.getRight())
+        .as("failure list")
+        .isEmpty();
+    List<Path> undeleted = triple.getLeft();
+    List<Path> deleted = triple.getMiddle();
+    Assertions.assertThat(deleted).
+        as("deleted files")
+        .containsAll(deleteAllowed)
+        .doesNotContainAnyElementsOf(deleteForbidden);
+    Assertions.assertThat(undeleted).
+        as("undeleted store entries")
+        .containsAll(deleteForbidden)
+        .doesNotContainAnyElementsOf(deleteAllowed);
+  }
+
+
+  private StoreContext createMockStoreContext(boolean multiDelete,
+      OperationTrackingStore store) throws URISyntaxException, IOException {
+    URI name = new URI("s3a://bucket");
+    Configuration conf = new Configuration();
+    return new StoreContext(
+        name,
+        "bucket",
+        conf,
+        "alice",
+        UserGroupInformation.getCurrentUser(),
+        BlockingThreadPoolExecutorService.newInstance(
+            4,
+            4,
+            10, TimeUnit.SECONDS,
+            "s3a-transfer-shared"),
+        Constants.DEFAULT_EXECUTOR_CAPACITY,
+        new Invoker(RetryPolicies.TRY_ONCE_THEN_FAIL, Invoker.LOG_EVENT),
+        new S3AInstrumentation(name),
+        new S3AStorageStatistics(),
+        S3AInputPolicy.Normal,
+        ChangeDetectionPolicy.createPolicy(ChangeDetectionPolicy.Mode.None,
+            ChangeDetectionPolicy.Source.ETag, false),
+        multiDelete,
+        store,
+        false,
+        CONTEXT_ACCESSORS,
+        new S3Guard.TtlTimeProvider(conf));
+  }
+
+  private static class MinimalContextAccessor implements ContextAccessors {
+
+    @Override
+    public Path keyToPath(final String key) {
+      return qualifyKey(key);
+    }
+
+    @Override
+    public String pathToKey(final Path path) {
+      return null;
+    }
+
+    @Override
+    public File createTempFile(final String prefix, final long size)
+        throws IOException {
+      throw new UnsupportedOperationException("unsppported");
+    }
+
+    @Override
+    public String getBucketLocation() throws IOException {
+      return null;
+    }
+  }
+  /**
+   * MetadataStore which tracks what is deleted and added.
+   */
+  private static class OperationTrackingStore implements MetadataStore {
+
+    private final List<Path> deleted = new ArrayList<>();
+
+    private final List<Path> created = new ArrayList<>();
+
+    @Override
+    public void initialize(final FileSystem fs) {
+    }
+
+    @Override
+    public void initialize(final Configuration conf) {
+    }
+
+    @Override
+    public void forgetMetadata(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 put(final PathMetadata meta) {
+      put(meta, null);
+    }
+
+    @Override
+    public void put(final PathMetadata meta,
+        final BulkOperationState operationState) {
+      created.add(meta.getFileStatus().getPath());
+    }
+
+    @Override
+    public void put(final Collection<? extends PathMetadata> metas,
+        final BulkOperationState operationState) {
+      metas.stream().forEach(meta -> put(meta, null));
+    }
+
+    @Override
+    public void put(final DirListingMetadata meta,
+        final BulkOperationState operationState) {
+      created.add(meta.getPath());
+    }
+
+    @Override
+    public void destroy() {
+    }
+
+    @Override
+    public void delete(final Path path,
+        final ITtlTimeProvider ttlTimeProvider) {
+      deleted.add(path);
+    }
+
+    @Override
+    public void deleteSubtree(final Path path,
+        final ITtlTimeProvider ttlTimeProvider) {
+
+    }
+
+    @Override
+    public void move(@Nullable final Collection<Path> pathsToDelete,
+        @Nullable final Collection<PathMetadata> pathsToCreate,
+        final ITtlTimeProvider ttlTimeProvider,
+        @Nullable final BulkOperationState operationState) {
+    }
+
+    @Override
+    public void prune(final PruneMode pruneMode, final long cutoff) {
+    }
+
+    @Override
+    public void prune(final PruneMode pruneMode,
+        final long cutoff,
+        final String keyPrefix) {
+
+    }
+
+    @Override
+    public BulkOperationState initiateBulkWrite(
+        final BulkOperationState.OperationType operation,
+        final Path dest) {
+      return null;
+    }
+
+    @Override
+    public Map<String, String> getDiagnostics() {
+      return null;
+    }
+
+    @Override
+    public void updateParameters(final Map<String, String> parameters) {
+    }
+
+    @Override
+    public void close() {
+    }
+
+    public List<Path> getDeleted() {
+      return deleted;
+    }
+
+    public List<Path> getCreated() {
+      return created;
+    }
+
+    @Override
+    public RenameTracker initiateRenameOperation(
+        final StoreContext storeContext,
+        final Path source,
+        final S3AFileStatus sourceStatus,
+        final Path dest) {
+      throw new UnsupportedOperationException("unsupported");
+    }
+
+    @Override
+    public void addAncestors(final Path qualifiedPath,
+        final ITtlTimeProvider timeProvider,
+        @Nullable final BulkOperationState operationState) {
+
+    }
+  }
+
+}
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 f616190..eaaa50c 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
@@ -80,6 +80,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,
@@ -187,19 +197,24 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
       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));
-    Path parent = f.getPath().getParent();
-    while (parent != null) {
-      S3AFileStatus dir = new S3AFileStatus(false, parent, f.getOwner());
-      ms.put(new PathMetadata(dir));
-      parent = parent.getParent();
+    try (BulkOperationState bulkWrite =
+             ms.initiateBulkWrite(
+                 BulkOperationState.OperationType.Put,
+                 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();
+      }
     }
   }
 
@@ -252,12 +267,13 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
       String...args) throws Exception {
     Path keepParent = path("prune-cli-keep");
     StopWatch timer = new StopWatch();
+    final S3AFileSystem fs = getFileSystem();
     try {
       S3GuardTool.Prune cmd = new S3GuardTool.Prune(cmdConf);
       cmd.setMetadataStore(ms);
 
-      getFileSystem().mkdirs(parent);
-      getFileSystem().mkdirs(keepParent);
+      fs.mkdirs(parent);
+      fs.mkdirs(keepParent);
       createFile(new Path(parent, "stale"), true, true);
       createFile(new Path(keepParent, "stale-to-keep"), true, true);
 
@@ -279,8 +295,14 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
       assertMetastoreListingCount(keepParent,
           "This child should have been kept (prefix restriction).", 1);
     } finally {
-      getFileSystem().delete(parent, true);
-      ms.prune(MetadataStore.PruneMode.ALL_BY_MODTIME, Long.MAX_VALUE);
+      fs.delete(parent, true);
+      fs.delete(keepParent, true);
+      ms.prune(MetadataStore.PruneMode.ALL_BY_MODTIME,
+          Long.MAX_VALUE,
+          fs.pathToKey(parent));
+      ms.prune(MetadataStore.PruneMode.ALL_BY_MODTIME,
+          Long.MAX_VALUE,
+          fs.pathToKey(keepParent));
     }
   }
 
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 5241dd4..03ebe1e 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
@@ -35,19 +35,21 @@ import com.amazonaws.services.dynamodbv2.document.Table;
 import com.amazonaws.services.dynamodbv2.model.ListTagsOfResourceRequest;
 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.PathIOException;
 import org.apache.hadoop.fs.contract.s3a.S3AContract;
 import org.apache.hadoop.fs.s3a.Constants;
 import org.apache.hadoop.fs.s3a.S3ATestConstants;
 import org.apache.hadoop.fs.s3a.Tristate;
-
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.DurationInfo;
+
 import org.junit.AfterClass;
-import org.junit.Assert;
 import org.junit.Assume;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -62,6 +64,7 @@ import org.apache.hadoop.fs.s3a.S3AFileStatus;
 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.S3AUtils.clearBucketOption;
@@ -159,7 +162,8 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
     testDynamoDBTableName = conf.get(
         S3ATestConstants.S3GUARD_DDB_TEST_TABLE_NAME_KEY);
     String dynamoDbTableName = conf.getTrimmed(S3GUARD_DDB_TABLE_NAME_KEY);
-    Assume.assumeTrue("No DynamoDB table name configured",
+    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
@@ -181,6 +185,19 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
     // We can use that table in the test if these assertions are valid
     conf.set(S3GUARD_DDB_TABLE_NAME_KEY, testDynamoDBTableName);
 
+    // remove some prune delays
+    conf.setInt(S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY, 0);
+
+    // clear all table tagging config before this test
+    conf.getPropsWithPrefix(S3GUARD_DDB_TABLE_TAG).keySet().forEach(
+        propKey -> conf.unset(S3GUARD_DDB_TABLE_TAG + propKey)
+    );
+
+    // set the tags on the table so that it can be tested later.
+    Map<String, String> tagMap = createTagMap();
+    for (Map.Entry<String, String> tagEntry : tagMap.entrySet()) {
+      conf.set(S3GUARD_DDB_TABLE_TAG + tagEntry.getKey(), tagEntry.getValue());
+    }
     LOG.debug("Creating static ddbms which will be shared between tests.");
     enableOnDemand(conf);
 
@@ -191,14 +208,23 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
   @AfterClass
   public static void afterClassTeardown() {
     LOG.debug("Destroying static DynamoDBMetadataStore.");
-    if (ddbmsStatic != null) {
+    destroy(ddbmsStatic);
+    ddbmsStatic = null;
+  }
+
+  /**
+   * Destroy and then close() a metastore instance.
+   * Exceptions are caught and logged at debug.
+   * @param ddbms store -may be null.
+   */
+  private static void destroy(final DynamoDBMetadataStore ddbms) {
+    if (ddbms != null) {
       try {
-        ddbmsStatic.destroy();
-      } catch (Exception e) {
-        LOG.warn("Failed to destroy tables in teardown", e);
+        ddbms.destroy();
+        IOUtils.closeStream(ddbms);
+      } catch (IOException e) {
+        LOG.debug("On ddbms shutdown", e);
       }
-      IOUtils.closeStream(ddbmsStatic);
-      ddbmsStatic = null;
     }
   }
 
@@ -208,26 +234,91 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
             Constants.S3GUARD_METASTORE_DYNAMO));
   }
 
-
+  /**
+   * This teardown does not call super.teardown() so as to avoid the DDMBS table
+   * from being destroyed.
+   * <p>
+   * This is potentially quite slow, depending on DDB IO Capacity and number
+   * of entries to forget.
+   */
   @Override
   public void tearDown() throws Exception {
+    LOG.info("Removing data from ddbms table in teardown.");
+    Thread.currentThread().setName("Teardown");
+    // The following is a way to be sure the table will be cleared and there
+    // will be no leftovers after the test.
     try {
-      if (ddbmsStatic != null) {
-        LOG.info("Removing data from ddbms table in teardown.");
-        // The following is a way to be sure the table will be cleared and there
-        // will be no leftovers after the test.
-        PathMetadata meta = ddbmsStatic.get(strToPath("/"));
-        if (meta != null){
-          for (DescendantsIterator desc =
-              new DescendantsIterator(ddbmsStatic, meta);
-              desc.hasNext();) {
-            ddbmsStatic.forgetMetadata(desc.next().getPath());
-          }
+      deleteAllMetadata();
+    } finally {
+      IOUtils.cleanupWithLogger(LOG, fileSystem);
+    }
+  }
+
+  /**
+   * Forget all metadata in the store.
+   * This originally did an iterate and forget, but using prune() hands off the
+   * bulk IO into the metastore itself; the forgetting is used
+   * to purge anything which wasn't pruned.
+   */
+  private void deleteAllMetadata() throws IOException {
+    // The following is a way to be sure the table will be cleared and there
+    // will be no leftovers after the test.
+    // only executed if there is a filesystem, as failure during test setup
+    // means that strToPath will NPE.
+    if (getContract() != null && getContract().getFileSystem() != null) {
+      deleteMetadataUnderPath(ddbmsStatic, strToPath("/"), true);
+    }
+  }
+
+  /**
+   * Delete all metadata under a path.
+   * Attempt to use prune first as it scales slightly better.
+   * @param ms store
+   * @param path path to prune under
+   * @param suppressErrors should errors be suppressed?
+   * @throws IOException if there is a failure and suppressErrors == false
+   */
+  public static void deleteMetadataUnderPath(final DynamoDBMetadataStore ms,
+      final Path path, final boolean suppressErrors) throws IOException {
+    ThrottleTracker throttleTracker = new ThrottleTracker(ms);
+    try (DurationInfo ignored = new DurationInfo(LOG, true, "prune")) {
+      ms.prune(PruneMode.ALL_BY_MODTIME,
+          System.currentTimeMillis(),
+          PathMetadataDynamoDBTranslation.pathToParentKey(path));
+      LOG.info("Throttle statistics: {}", throttleTracker);
+    } catch (FileNotFoundException fnfe) {
+      // there is no table.
+      return;
+    } catch (IOException ioe) {
+      // prune failed. warn and then fall back to forget.
+      LOG.warn("Failed to prune {}", path, ioe);
+      if (!suppressErrors) {
+        throw ioe;
+      }
+    }
+    // and after the pruning, make sure all other metadata is gone
+    int forgotten = 0;
+    try (DurationInfo ignored = new DurationInfo(LOG, true, "forget")) {
+      PathMetadata meta = ms.get(path);
+      if (meta != null) {
+        for (DescendantsIterator desc = new DescendantsIterator(ms,
+            meta);
+            desc.hasNext();) {
+          forgotten++;
+          ms.forgetMetadata(desc.next().getPath());
         }
+        LOG.info("Forgot {} entries", forgotten);
+      }
+    } catch (FileNotFoundException fnfe) {
+      // there is no table.
+      return;
+    } catch (IOException ioe) {
+      LOG.warn("Failed to forget entries under {}", path, ioe);
+      if (!suppressErrors) {
+        throw ioe;
       }
-    } catch (IOException ignored) {
     }
-    IOUtils.cleanupWithLogger(LOG, fileSystem);
+    LOG.info("Throttle statistics: {}", throttleTracker);
   }
 
   @Override protected String getPathStringForPrune(String path)
@@ -270,7 +361,7 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
   }
 
   @Override
-  S3AFileStatus basicFileStatus(Path path, int size, boolean isDir)
+  protected S3AFileStatus basicFileStatus(Path path, int size, boolean isDir)
       throws IOException {
     String owner = UserGroupInformation.getCurrentUser().getShortUserName();
     return isDir
@@ -279,6 +370,15 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
             null, null);
   }
 
+  /**
+   * Create a directory status entry.
+   * @param dir directory.
+   * @return the status
+   */
+  private S3AFileStatus dirStatus(Path dir) throws IOException {
+    return basicFileStatus(dir, 0, true);
+  }
+
   private DynamoDBMetadataStore getDynamoMetadataStore() throws IOException {
     return (DynamoDBMetadataStore) getContract().getMetadataStore();
   }
@@ -335,8 +435,7 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
           expectedRegion,
           ddbms.getRegion());
     } finally {
-      ddbms.destroy();
-      ddbms.close();
+      destroy(ddbms);
     }
   }
 
@@ -377,31 +476,69 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
           keySchema(),
           ddbms.getTable().describe().getKeySchema());
     } finally {
-      ddbms.destroy();
-      ddbms.close();
+      destroy(ddbms);
     }
   }
 
   /**
-   * Test that for a large batch write request, the limit is handled correctly.
+   * This should really drive a parameterized test run of 5^2 entries, but it
+   * would require a major refactoring to set things up.
+   * For now, each source test has its own entry, with the destination written
+   * to.
+   * This seems to be enough to stop DDB throttling from triggering test
+   * timeouts.
    */
+  private static final int[] NUM_METAS_TO_DELETE_OR_PUT = {
+      -1, // null
+      0, // empty collection
+      1, // one path
+      S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT, // exact limit of a batch request
+      S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT + 1 // limit + 1
+  };
+
   @Test
-  public void testBatchWrite() throws IOException {
-    final int[] numMetasToDeleteOrPut = {
-        -1, // null
-        0, // empty collection
-        1, // one path
-        S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT, // exact limit of a batch request
-        S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT + 1 // limit + 1
-    };
-    DynamoDBMetadataStore ms = getDynamoMetadataStore();
-    for (int numOldMetas : numMetasToDeleteOrPut) {
-      for (int numNewMetas : numMetasToDeleteOrPut) {
-        doTestBatchWrite(numOldMetas, numNewMetas, ms);
-      }
+  public void testBatchWrite00() throws IOException {
+    doBatchWriteForOneSet(0);
+  }
+
+  @Test
+  public void testBatchWrite01() throws IOException {
+    doBatchWriteForOneSet(1);
+  }
+
+  @Test
+  public void testBatchWrite02() throws IOException {
+    doBatchWriteForOneSet(2);
+  }
+
+  @Test
+  public void testBatchWrite03() throws IOException {
+    doBatchWriteForOneSet(3);
+  }
+
+  @Test
+  public void testBatchWrite04() throws IOException {
+    doBatchWriteForOneSet(4);
+  }
+
+  /**
+   * Test that for a large batch write request, the limit is handled correctly.
+   * With cleanup afterwards.
+   */
+  private void doBatchWriteForOneSet(int index) throws IOException {
+    for (int numNewMetas : NUM_METAS_TO_DELETE_OR_PUT) {
+      doTestBatchWrite(NUM_METAS_TO_DELETE_OR_PUT[index],
+          numNewMetas,
+          getDynamoMetadataStore());
     }
+    // The following is a way to be sure the table will be cleared and there
+    // will be no leftovers after the test.
+    deleteMetadataUnderPath(ddbmsStatic, strToPath("/"), false);
   }
 
+  /**
+   * Test that for a large batch write request, the limit is handled correctly.
+   */
   private void doTestBatchWrite(int numDelete, int numPut,
       DynamoDBMetadataStore ms) throws IOException {
     Path path = new Path(
@@ -411,15 +548,20 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
     final Path oldDir = new Path(root, "oldDir");
     final Path newDir = new Path(root, "newDir");
     LOG.info("doTestBatchWrite: oldDir={}, newDir={}", oldDir, newDir);
+    Thread.currentThread()
+        .setName(String.format("Bulk put=%d; delete=%d", numPut, numDelete));
 
-    ms.put(new PathMetadata(basicFileStatus(oldDir, 0, true)));
-    ms.put(new PathMetadata(basicFileStatus(newDir, 0, true)));
+    AncestorState putState = checkNotNull(ms.initiateBulkWrite(
+        BulkOperationState.OperationType.Put, newDir),
+        "No state from initiateBulkWrite()");
+    ms.put(new PathMetadata(dirStatus(oldDir)), putState);
+    ms.put(new PathMetadata(dirStatus(newDir)), putState);
 
     final List<PathMetadata> oldMetas = numDelete < 0 ? null :
         new ArrayList<>(numDelete);
     for (int i = 0; i < numDelete; i++) {
       oldMetas.add(new PathMetadata(
-          basicFileStatus(new Path(oldDir, "child" + i), i, true)));
+          basicFileStatus(new Path(oldDir, "child" + i), i, false)));
     }
     final List<PathMetadata> newMetas = numPut < 0 ? null :
         new ArrayList<>(numPut);
@@ -431,8 +573,13 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
     Collection<Path> pathsToDelete = null;
     if (oldMetas != null) {
       // put all metadata of old paths and verify
-      ms.put(new DirListingMetadata(oldDir, oldMetas, false));
-      assertEquals(0, ms.listChildren(newDir).withoutTombstones().numEntries());
+      ms.put(new DirListingMetadata(oldDir, oldMetas, false), putState);
+      assertEquals("Child count",
+          0, ms.listChildren(newDir).withoutTombstones().numEntries());
+      Assertions.assertThat(ms.listChildren(oldDir).getListing())
+          .describedAs("Old Directory listing")
+          .containsExactlyInAnyOrderElementsOf(oldMetas);
+
       assertTrue(CollectionUtils
           .isEqualCollection(oldMetas, ms.listChildren(oldDir).getListing()));
 
@@ -443,11 +590,28 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
     }
 
     // move the old paths to new paths and verify
-    ms.move(pathsToDelete, newMetas, getTtlTimeProvider());
-    assertEquals(0, ms.listChildren(oldDir).withoutTombstones().numEntries());
+    AncestorState state = checkNotNull(ms.initiateBulkWrite(
+        BulkOperationState.OperationType.Put, newDir),
+        "No state from initiateBulkWrite()");
+    assertEquals("bulk write destination", newDir, state.getDest());
+
+    ThrottleTracker throttleTracker = new ThrottleTracker(ms);
+    try(DurationInfo ignored = new DurationInfo(LOG, true,
+        "Move")) {
+      ms.move(pathsToDelete, newMetas, getTtlTimeProvider(), state);
+    }
+    LOG.info("Throttle status {}", throttleTracker);
+    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());
+      }
     }
   }
 
@@ -483,63 +647,66 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
   }
 
   /**
+   * Test versioning handling.
+   * <ol>
+   *   <li>Create the table.</li>
+   *   <li>Verify tag propagation.</li>
+   *   <li>Delete the version marker -verify failure.</li>
+   *   <li>Reinstate a different version marker -verify failure</li>
+   * </ol>
    * Delete the version marker and verify that table init fails.
+   * This also includes the checks for tagging, which goes against all
+   * principles of unit tests.
+   * However, merging the routines saves
    */
   @Test
-  public void testTableVersionRequired() throws Exception {
+  public void testTableVersioning() throws Exception {
     String tableName = getTestTableName("testTableVersionRequired");
     Configuration conf = getTableCreationConfig();
     int maxRetries = conf.getInt(S3GUARD_DDB_MAX_RETRIES,
         S3GUARD_DDB_MAX_RETRIES_DEFAULT);
     conf.setInt(S3GUARD_DDB_MAX_RETRIES, 3);
     conf.set(S3GUARD_DDB_TABLE_NAME_KEY, tableName);
-
+    tagConfiguration(conf);
     DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore();
     try {
       ddbms.initialize(conf);
       Table table = verifyTableInitialized(tableName, ddbms.getDynamoDB());
+      // check the tagging too
+      verifyStoreTags(createTagMap(), ddbms);
+
+      Item originalVersionMarker = table.getItem(VERSION_MARKER_PRIMARY_KEY);
       table.deleteItem(VERSION_MARKER_PRIMARY_KEY);
 
       // create existing table
       intercept(IOException.class, E_NO_VERSION_MARKER,
           () -> ddbms.initTable());
 
-      conf.setInt(S3GUARD_DDB_MAX_RETRIES, maxRetries);
-    } finally {
-      ddbms.destroy();
-      ddbms.close();
-    }
-  }
-
-  /**
-   * Set the version value to a different number and verify that
-   * table init fails.
-   */
-  @Test
-  public void testTableVersionMismatch() throws Exception {
-    String tableName = getTestTableName("testTableVersionMismatch");
-    Configuration conf = getTableCreationConfig();
-    conf.set(S3GUARD_DDB_TABLE_NAME_KEY, tableName);
-
-    DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore();
-    try {
-      ddbms.initialize(conf);
-      Table table = verifyTableInitialized(tableName, ddbms.getDynamoDB());
-      table.deleteItem(VERSION_MARKER_PRIMARY_KEY);
-      Item v200 = createVersionMarker(VERSION_MARKER, 200, 0);
+      // now add a different version marker
+      Item v200 = createVersionMarker(VERSION_MARKER, VERSION * 2, 0);
       table.putItem(v200);
 
       // create existing table
       intercept(IOException.class, E_INCOMPATIBLE_VERSION,
           () -> ddbms.initTable());
-    } finally {
-      ddbms.destroy();
-      ddbms.close();
-    }
-  }
 
+      // create a marker with no version and expect failure
+      final Item invalidMarker = new Item().withPrimaryKey(
+          createVersionMarkerPrimaryKey(VERSION_MARKER))
+          .withLong(TABLE_CREATED, 0);
+      table.putItem(invalidMarker);
 
+      intercept(IOException.class, E_NOT_VERSION_MARKER,
+          () -> ddbms.initTable());
 
+      // reinstate the version marker
+      table.putItem(originalVersionMarker);
+      ddbms.initTable();
+      conf.setInt(S3GUARD_DDB_MAX_RETRIES, maxRetries);
+    } finally {
+      destroy(ddbms);
+    }
+  }
 
   /**
    * Test that initTable fails with IOException when table does not exist and
@@ -580,7 +747,8 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
 
     ddbms.put(new PathMetadata(new S3AFileStatus(true,
         new Path(rootPath, "foo"),
-        UserGroupInformation.getCurrentUser().getShortUserName())));
+        UserGroupInformation.getCurrentUser().getShortUserName())),
+        null);
     verifyRootDirectory(ddbms.get(rootPath), false);
   }
 
@@ -631,9 +799,13 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
     final String destRoot = testRoot + "/c/d/e/dest";
 
     final Path nestedPath1 = strToPath(srcRoot + "/file1.txt");
-    ddbms.put(new PathMetadata(basicFileStatus(nestedPath1, 1024, false)));
+    AncestorState bulkWrite = ddbms.initiateBulkWrite(
+        BulkOperationState.OperationType.Put, nestedPath1);
+    ddbms.put(new PathMetadata(basicFileStatus(nestedPath1, 1024, false)),
+        bulkWrite);
     final Path nestedPath2 = strToPath(srcRoot + "/dir1/dir2");
-    ddbms.put(new PathMetadata(basicFileStatus(nestedPath2, 0, true)));
+    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.
@@ -643,8 +815,8 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
         strToPath(srcRoot),
         strToPath(srcRoot + "/dir1"),
         strToPath(srcRoot + "/dir1/dir2"),
-        strToPath(srcRoot + "/file1.txt")
-    );
+        strToPath(srcRoot + "/file1.txt"));
+    final String finalFile = destRoot + "/file1.txt";
     final Collection<PathMetadata> pathsToCreate = Lists.newArrayList(
         new PathMetadata(basicFileStatus(strToPath(destRoot),
             0, true)),
@@ -652,22 +824,101 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
             0, true)),
         new PathMetadata(basicFileStatus(strToPath(destRoot + "/dir1/dir2"),
             0, true)),
-        new PathMetadata(basicFileStatus(strToPath(destRoot + "/file1.txt"),
+        new PathMetadata(basicFileStatus(strToPath(finalFile),
             1024, false))
     );
 
-    ddbms.move(fullSourcePaths, pathsToCreate, getTtlTimeProvider());
-
+    ddbms.move(fullSourcePaths, pathsToCreate, getTtlTimeProvider(),
+        bulkWrite);
+    bulkWrite.close();
     // assert that all the ancestors should have been populated automatically
-    assertCached(testRoot + "/c");
-    assertCached(testRoot + "/c/d");
-    assertCached(testRoot + "/c/d/e");
-    assertCached(destRoot /* /c/d/e/dest */);
-
+    List<String> paths = Lists.newArrayList(
+        testRoot + "/c", testRoot + "/c/d", testRoot + "/c/d/e", destRoot,
+        destRoot + "/dir1", destRoot + "/dir1/dir2");
+    for (String p : paths) {
+      assertCached(p);
+      verifyInAncestor(bulkWrite, p, true);
+    }
     // Also check moved files while we're at it
-    assertCached(destRoot + "/dir1");
-    assertCached(destRoot + "/dir1/dir2");
-    assertCached(destRoot + "/file1.txt");
+    assertCached(finalFile);
+    verifyInAncestor(bulkWrite, finalFile, false);
+  }
+
+  @Test
+  public void testAncestorOverwriteConflict() throws Throwable {
+    final DynamoDBMetadataStore ddbms = getDynamoMetadataStore();
+    String testRoot = "/" + getMethodName();
+    String parent = testRoot + "/parent";
+    Path parentPath = strToPath(parent);
+    String child = parent + "/child";
+    Path childPath = strToPath(child);
+    String grandchild = child + "/grandchild";
+    Path grandchildPath = strToPath(grandchild);
+    String child2 = parent + "/child2";
+    String grandchild2 = child2 + "/grandchild2";
+    Path grandchild2Path = strToPath(grandchild2);
+    AncestorState bulkWrite = ddbms.initiateBulkWrite(
+        BulkOperationState.OperationType.Put, parentPath);
+
+    // writing a child creates ancestors
+    ddbms.put(
+        new PathMetadata(basicFileStatus(childPath, 1024, false)),
+        bulkWrite);
+    verifyInAncestor(bulkWrite, child, false);
+    verifyInAncestor(bulkWrite, parent, true);
+
+    // overwrite an ancestor with a file entry in the same operation
+    // is an error.
+    intercept(PathIOException.class, E_INCONSISTENT_UPDATE,
+        () -> ddbms.put(
+            new PathMetadata(basicFileStatus(parentPath, 1024, false)),
+            bulkWrite));
+
+    // now put a file under the child and expect the put operation
+    // to fail fast, because the ancestor state includes a file at a parent.
+
+    intercept(PathIOException.class, E_INCONSISTENT_UPDATE,
+        () -> ddbms.put(
+            new PathMetadata(basicFileStatus(grandchildPath, 1024, false)),
+            bulkWrite));
+
+    // and expect a failure for directory update under the child
+    DirListingMetadata grandchildListing = new DirListingMetadata(
+        grandchildPath,
+        new ArrayList<>(), false);
+    intercept(PathIOException.class, E_INCONSISTENT_UPDATE,
+        () -> ddbms.put(grandchildListing, bulkWrite));
+
+    // but a directory update under another path is fine
+    DirListingMetadata grandchild2Listing = new DirListingMetadata(
+        grandchild2Path,
+        new ArrayList<>(), false);
+    ddbms.put(grandchild2Listing, bulkWrite);
+    // and it creates a new entry for its parent
+    verifyInAncestor(bulkWrite, child2, true);
+  }
+
+  /**
+   * Assert that a path has an entry in the ancestor state.
+   * @param state ancestor state
+   * @param path path to look for
+   * @param isDirectory is it a directory
+   * @return the value
+   * @throws IOException IO failure
+   * @throws AssertionError assertion failure.
+   */
+  private DDBPathMetadata verifyInAncestor(AncestorState state,
+      String path,
+      final boolean isDirectory)
+      throws IOException {
+    final Path p = strToPath(path);
+    assertTrue("Path " + p + " not found in ancestor state", state.contains(p));
+    final DDBPathMetadata md = state.get(p);
+    assertTrue("Ancestor value for "+  path,
+        isDirectory
+            ? md.getFileStatus().isDirectory()
+            : md.getFileStatus().isFile());
+    return md;
   }
 
   @Test
@@ -731,50 +982,48 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
       intercept(IOException.class, "",
           "Should have failed after the table is destroyed!",
           () -> ddbms.listChildren(testPath));
-    } finally {
       ddbms.destroy();
-      ddbms.close();
+      intercept(FileNotFoundException.class, "",
+          "Destroyed table should raise FileNotFoundException when pruned",
+          () -> ddbms.prune(PruneMode.ALL_BY_MODTIME, 0));
+    } finally {
+      destroy(ddbms);
     }
   }
 
-  @Test
-  public void testTableTagging() throws IOException {
-    final Configuration conf = getTableCreationConfig();
-    // clear all table tagging config before this test
-    conf.getPropsWithPrefix(S3GUARD_DDB_TABLE_TAG).keySet().forEach(
-        propKey -> conf.unset(S3GUARD_DDB_TABLE_TAG + propKey)
-    );
+  protected void verifyStoreTags(final Map<String, String> tagMap,
+      final DynamoDBMetadataStore store) {
+    List<Tag> tags = listTagsOfStore(store);
+    Map<String, String> actual = new HashMap<>();
+    tags.forEach(t -> actual.put(t.getKey(), t.getValue()));
+    Assertions.assertThat(actual)
+        .describedAs("Tags from DDB table")
+        .containsExactlyEntriesOf(tagMap);
+    assertEquals(tagMap.size(), tags.size());
... 1859 lines suppressed ...


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org