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 2017/09/29 10:29:38 UTC

[1/8] hadoop git commit: HADOOP-13345 S3Guard: Improved Consistency for S3A. Contributed by: Chris Nauroth, Aaron Fabbri, Mingliang Liu, Lei (Eddy) Xu, Sean Mackrory, Steve Loughran and others.

Repository: hadoop
Updated Branches:
  refs/heads/branch-2 a641bcec0 -> a1afc6aa5


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractITestS3AMetadataStoreScale.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractITestS3AMetadataStoreScale.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractITestS3AMetadataStoreScale.java
new file mode 100644
index 0000000..876cc80
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractITestS3AMetadataStoreScale.java
@@ -0,0 +1,250 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.scale;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.S3AFileStatus;
+import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
+import org.apache.hadoop.fs.s3a.s3guard.PathMetadata;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.NanoTimer;
+
+/**
+ * Test the performance of a MetadataStore.  Useful for load testing.
+ * Could be separated from S3A code, but we're using the S3A scale test
+ * framework for convenience.
+ */
+public abstract class AbstractITestS3AMetadataStoreScale extends
+    S3AScaleTestBase {
+  private static final Logger LOG = LoggerFactory.getLogger(
+      AbstractITestS3AMetadataStoreScale.class);
+
+  /** Some dummy values for FileStatus contents. */
+  static final long BLOCK_SIZE = 32 * 1024 * 1024;
+  static final long SIZE = BLOCK_SIZE * 2;
+  static final String OWNER = "bob";
+  static final long ACCESS_TIME = System.currentTimeMillis();
+
+  static final Path BUCKET_ROOT = new Path("s3a://fake-bucket/");
+
+  /**
+   * Subclasses should override this to provide the MetadataStore they which
+   * to test.
+   * @return MetadataStore to test against
+   * @throws IOException
+   */
+  public abstract MetadataStore createMetadataStore() throws IOException;
+
+  @Test
+  public void testPut() throws Throwable {
+    describe("Test workload of put() operations");
+
+    // As described in hadoop-aws site docs, count parameter is used for
+    // width and depth of directory tree
+    int width = getConf().getInt(KEY_DIRECTORY_COUNT, DEFAULT_DIRECTORY_COUNT);
+    int depth = width;
+
+    List<PathMetadata> paths = new ArrayList<>();
+    createDirTree(BUCKET_ROOT, depth, width, paths);
+
+    long count = 1; // Some value in case we throw an exception below
+    try (MetadataStore ms = createMetadataStore()) {
+
+      try {
+        count = populateMetadataStore(paths, ms);
+      } finally {
+        clearMetadataStore(ms, count);
+      }
+    }
+  }
+
+  @Test
+  public void testMoves() throws Throwable {
+    describe("Test workload of batched move() operations");
+
+    // As described in hadoop-aws site docs, count parameter is used for
+    // width and depth of directory tree
+    int width = getConf().getInt(KEY_DIRECTORY_COUNT, DEFAULT_DIRECTORY_COUNT);
+    int depth = width;
+
+    long operations = getConf().getLong(KEY_OPERATION_COUNT,
+        DEFAULT_OPERATION_COUNT);
+
+    List<PathMetadata> origMetas = new ArrayList<>();
+    createDirTree(BUCKET_ROOT, depth, width, origMetas);
+
+    // Pre-compute source and destination paths for move() loop below
+    List<Path> origPaths = metasToPaths(origMetas);
+    List<PathMetadata> movedMetas = moveMetas(origMetas, BUCKET_ROOT,
+        new Path(BUCKET_ROOT, "moved-here"));
+    List<Path> movedPaths = metasToPaths(movedMetas);
+
+    long count = 1; // Some value in case we throw an exception below
+    try (MetadataStore ms = createMetadataStore()) {
+
+      try {
+        // Setup
+        count = populateMetadataStore(origMetas, ms);
+
+        // Main loop: move things back and forth
+        describe("Running move workload");
+        NanoTimer moveTimer = new NanoTimer();
+        LOG.info("Running {} moves of {} paths each", operations,
+            origMetas.size());
+        for (int i = 0; i < operations; i++) {
+          Collection<Path> toDelete;
+          Collection<PathMetadata> toCreate;
+          if (i % 2 == 0) {
+            toDelete = origPaths;
+            toCreate = movedMetas;
+          } else {
+            toDelete = movedPaths;
+            toCreate = origMetas;
+          }
+          ms.move(toDelete, toCreate);
+        }
+        moveTimer.end();
+        printTiming(LOG, "move", moveTimer, operations);
+      } finally {
+        // Cleanup
+        clearMetadataStore(ms, count);
+      }
+    }
+  }
+
+  /**
+   * Create a copy of given list of PathMetadatas with the paths moved from
+   * src to dest.
+   */
+  private List<PathMetadata> moveMetas(List<PathMetadata> metas, Path src,
+      Path dest) throws IOException {
+    List<PathMetadata> moved = new ArrayList<>(metas.size());
+    for (PathMetadata srcMeta : metas) {
+      S3AFileStatus status = copyStatus((S3AFileStatus)srcMeta.getFileStatus());
+      status.setPath(movePath(status.getPath(), src, dest));
+      moved.add(new PathMetadata(status));
+    }
+    return moved;
+  }
+
+  private Path movePath(Path p, Path src, Path dest) {
+    String srcStr = src.toUri().getPath();
+    String pathStr = p.toUri().getPath();
+    // Strip off src dir
+    pathStr = pathStr.substring(srcStr.length());
+    // Prepend new dest
+    return new Path(dest, pathStr);
+  }
+
+  private S3AFileStatus copyStatus(S3AFileStatus status) {
+    if (status.isDirectory()) {
+      return new S3AFileStatus(status.isEmptyDirectory(), status.getPath(),
+          status.getOwner());
+    } else {
+      return new S3AFileStatus(status.getLen(), status.getModificationTime(),
+          status.getPath(), status.getBlockSize(), status.getOwner());
+    }
+  }
+
+  /** @return number of PathMetadatas put() into MetadataStore */
+  private long populateMetadataStore(Collection<PathMetadata> paths,
+      MetadataStore ms) throws IOException {
+    long count = 0;
+    NanoTimer putTimer = new NanoTimer();
+    describe("Inserting into MetadataStore");
+    for (PathMetadata p : paths) {
+      ms.put(p);
+      count++;
+    }
+    putTimer.end();
+    printTiming(LOG, "put", putTimer, count);
+    return count;
+  }
+
+  private void clearMetadataStore(MetadataStore ms, long count)
+      throws IOException {
+    describe("Recursive deletion");
+    NanoTimer deleteTimer = new NanoTimer();
+    ms.deleteSubtree(BUCKET_ROOT);
+    deleteTimer.end();
+    printTiming(LOG, "delete", deleteTimer, count);
+  }
+
+  private static void printTiming(Logger log, String op, NanoTimer timer,
+      long count) {
+    double msec = (double)timer.duration() / 1000;
+    double msecPerOp = msec / count;
+    log.info(String.format("Elapsed %.2f msec. %.3f msec / %s (%d ops)", msec,
+        msecPerOp, op, count));
+  }
+
+  private static S3AFileStatus makeFileStatus(Path path) throws IOException {
+    return new S3AFileStatus(SIZE, ACCESS_TIME, path, BLOCK_SIZE, OWNER);
+  }
+
+  private static S3AFileStatus makeDirStatus(Path p) throws IOException {
+    return new S3AFileStatus(false, p, OWNER);
+  }
+
+  private List<Path> metasToPaths(List<PathMetadata> metas) {
+    List<Path> paths = new ArrayList<>(metas.size());
+    for (PathMetadata meta : metas) {
+      paths.add(meta.getFileStatus().getPath());
+    }
+    return paths;
+  }
+
+  /**
+   * Recursively create a directory tree.
+   * @param parent Parent dir of the paths to create.
+   * @param depth How many more levels deep past parent to create.
+   * @param width Number of files (and directories, if depth > 0) per directory.
+   * @param paths List to add generated paths to.
+   */
+  private static void createDirTree(Path parent, int depth, int width,
+      Collection<PathMetadata> paths) throws IOException {
+
+    // Create files
+    for (int i = 0; i < width; i++) {
+      Path p = new Path(parent, String.format("file-%d", i));
+      PathMetadata meta = new PathMetadata(makeFileStatus(p));
+      paths.add(meta);
+    }
+
+    if (depth == 0) {
+      return;
+    }
+
+    // Create directories if there is depth remaining
+    for (int i = 0; i < width; i++) {
+      Path dir = new Path(parent, String.format("dir-%d", i));
+      PathMetadata meta = new PathMetadata(makeDirStatus(dir));
+      paths.add(meta);
+      createDirTree(dir, depth-1, width, paths);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java
index 89fae82..8b163cb 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java
@@ -25,6 +25,7 @@ import java.util.concurrent.atomic.AtomicLong;
 import com.amazonaws.event.ProgressEvent;
 import com.amazonaws.event.ProgressEventType;
 import com.amazonaws.event.ProgressListener;
+import org.apache.hadoop.fs.FileStatus;
 import org.junit.FixMethodOrder;
 import org.junit.Test;
 import org.junit.runners.MethodSorters;
@@ -34,11 +35,9 @@ import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageStatistics;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.apache.hadoop.fs.s3a.S3AFileStatus;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
 import org.apache.hadoop.fs.s3a.S3AInstrumentation;
 import org.apache.hadoop.fs.s3a.Statistic;
@@ -222,7 +221,7 @@ public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
     assertEquals("active put requests in \n" + fs,
         0, gaugeValue(putRequestsActive));
     ContractTestUtils.assertPathExists(fs, "Huge file", hugefile);
-    S3AFileStatus status = fs.getFileStatus(hugefile);
+    FileStatus status = fs.getFileStatus(hugefile);
     ContractTestUtils.assertIsFile(hugefile, status);
     assertEquals("File size in " + status, filesize, status.getLen());
     if (progress != null) {
@@ -324,7 +323,7 @@ public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
     String filetype = encrypted ? "encrypted file" : "file";
     describe("Positioned reads of %s %s", filetype, hugefile);
     S3AFileSystem fs = getFileSystem();
-    S3AFileStatus status = fs.getFileStatus(hugefile);
+    FileStatus status = fs.getFileStatus(hugefile);
     long filesize = status.getLen();
     int ops = 0;
     final int bufferSize = 8192;
@@ -364,7 +363,7 @@ public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
     assumeHugeFileExists();
     describe("Reading %s", hugefile);
     S3AFileSystem fs = getFileSystem();
-    S3AFileStatus status = fs.getFileStatus(hugefile);
+    FileStatus status = fs.getFileStatus(hugefile);
     long filesize = status.getLen();
     long blocks = filesize / uploadBlockSize;
     byte[] data = new byte[uploadBlockSize];
@@ -390,7 +389,7 @@ public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
     assumeHugeFileExists();
     describe("renaming %s to %s", hugefile, hugefileRenamed);
     S3AFileSystem fs = getFileSystem();
-    S3AFileStatus status = fs.getFileStatus(hugefile);
+    FileStatus status = fs.getFileStatus(hugefile);
     long filesize = status.getLen();
     fs.delete(hugefileRenamed, false);
     ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
@@ -401,7 +400,7 @@ public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
         toHuman(timer.nanosPerOperation(mb)));
     bandwidth(timer, filesize);
     logFSState();
-    S3AFileStatus destFileStatus = fs.getFileStatus(hugefileRenamed);
+    FileStatus destFileStatus = fs.getFileStatus(hugefileRenamed);
     assertEquals(filesize, destFileStatus.getLen());
 
     // rename back

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestDynamoDBMetadataStoreScale.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestDynamoDBMetadataStoreScale.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestDynamoDBMetadataStoreScale.java
new file mode 100644
index 0000000..3de1935
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestDynamoDBMetadataStoreScale.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.scale;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStore;
+import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
+
+import java.io.IOException;
+
+import static org.junit.Assume.*;
+import static org.apache.hadoop.fs.s3a.Constants.*;
+
+/**
+ * Scale test for DynamoDBMetadataStore.
+ */
+public class ITestDynamoDBMetadataStoreScale
+    extends AbstractITestS3AMetadataStoreScale {
+
+  @Override
+  public MetadataStore createMetadataStore() throws IOException {
+    Configuration conf = getFileSystem().getConf();
+    String ddbTable = conf.get(S3GUARD_DDB_TABLE_NAME_KEY);
+    assumeNotNull("DynamoDB table is configured", ddbTable);
+    String ddbEndpoint = conf.get(S3GUARD_DDB_REGION_KEY);
+    assumeNotNull("DynamoDB endpoint is configured", ddbEndpoint);
+
+    DynamoDBMetadataStore ms = new DynamoDBMetadataStore();
+    ms.initialize(getFileSystem().getConf());
+    return ms;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestLocalMetadataStoreScale.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestLocalMetadataStoreScale.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestLocalMetadataStoreScale.java
new file mode 100644
index 0000000..591fb0e
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestLocalMetadataStoreScale.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.scale;
+
+import org.apache.hadoop.fs.s3a.s3guard.LocalMetadataStore;
+import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
+
+import java.io.IOException;
+
+/**
+ * Scale test for LocalMetadataStore.
+ */
+public class ITestLocalMetadataStoreScale
+    extends AbstractITestS3AMetadataStoreScale {
+  @Override
+  public MetadataStore createMetadataStore() throws IOException {
+    MetadataStore ms = new LocalMetadataStore();
+    ms.initialize(getFileSystem());
+    return ms;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AConcurrentOps.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AConcurrentOps.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AConcurrentOps.java
index b4d3862..e320bb2 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AConcurrentOps.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AConcurrentOps.java
@@ -107,7 +107,7 @@ public class ITestS3AConcurrentOps extends S3AScaleTestBase {
 
   private S3AFileSystem getNormalFileSystem() throws Exception {
     S3AFileSystem s3a = new S3AFileSystem();
-    Configuration conf = new Configuration();
+    Configuration conf = createScaleConfiguration();
     URI rootURI = new URI(conf.get(TEST_FS_S3A_NAME));
     s3a.initialize(rootURI, conf);
     return s3a;
@@ -115,6 +115,7 @@ public class ITestS3AConcurrentOps extends S3AScaleTestBase {
 
   @After
   public void teardown() throws Exception {
+    super.teardown();
     if (auxFs != null) {
       auxFs.delete(testRoot, true);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ACreatePerformance.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ACreatePerformance.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ACreatePerformance.java
new file mode 100644
index 0000000..fd32ba5
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ACreatePerformance.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.scale;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.OutputStream;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
+
+/**
+ * Tests for create(): performance and/or load testing.
+ */
+public class ITestS3ACreatePerformance extends S3AScaleTestBase {
+  private static final Logger LOG = LoggerFactory.getLogger(
+      ITestS3ADirectoryPerformance.class);
+
+  private Path basePath;
+  private int basePathDepth;
+  private static final int PATH_DEPTH = 10;
+
+  @Override
+  public void setup() throws Exception {
+    super.setup();
+    basePath = getTestPath();
+    basePathDepth = basePath.depth();
+  }
+
+  /**
+   * Test rate at which we can create deeply-nested files from a single thread.
+   * @throws Exception
+   */
+  @Test
+  public void testDeepSequentialCreate() throws Exception {
+    long numOperations = getOperationCount();
+    S3AFileSystem fs = getFileSystem();
+
+    NanoTimer timer = new NanoTimer();
+    for (int i = 0; i < numOperations; i++) {
+      Path p = getPathIteration(i, PATH_DEPTH);
+      OutputStream out = fs.create(p);
+      out.write(40);  // one byte file with some value 40
+      out.close();
+    }
+    timer.end("Time to create %d files of depth %d", getOperationCount(),
+        PATH_DEPTH);
+    LOG.info("Time per create: {} msec",
+        timer.nanosPerOperation(numOperations) / 1000);
+  }
+
+  /* Get a unique path of depth totalDepth for given test iteration. */
+  private Path getPathIteration(long iter, int totalDepth) throws Exception {
+    assertTrue("Test path too long, increase PATH_DEPTH in test.",
+        totalDepth > basePathDepth);
+
+    int neededDirs = totalDepth - basePathDepth - 1;
+    StringBuilder sb = new StringBuilder();
+    for (int i = 0; i < neededDirs; i++) {
+      sb.append("iter-").append(iter);
+      sb.append("-dir-").append(i);
+      sb.append("/");
+    }
+    sb.append("file").append(iter);
+    return new Path(basePath, sb.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java
index d71364f..03f1e22 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java
@@ -113,14 +113,15 @@ public class ITestS3ADirectoryPerformance extends S3AScaleTestBase {
           listContinueRequests,
           listStatusCalls,
           getFileStatusCalls);
-      assertEquals(listRequests.toString(), 2, listRequests.diff());
+      if (!fs.hasMetadataStore()) {
+        assertEquals(listRequests.toString(), 2, listRequests.diff());
+      }
       reset(metadataRequests,
           listRequests,
           listContinueRequests,
           listStatusCalls,
           getFileStatusCalls);
 
-
     } finally {
       describe("deletion");
       // deletion at the end of the run

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java
index 236ffcd..83ab210 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java
@@ -20,10 +20,10 @@ package org.apache.hadoop.fs.s3a.scale;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
+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.S3AFileStatus;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
 import org.apache.hadoop.fs.s3a.S3AInputPolicy;
 import org.apache.hadoop.fs.s3a.S3AInputStream;
@@ -56,7 +56,7 @@ public class ITestS3AInputStreamPerformance extends S3AScaleTestBase {
 
   private S3AFileSystem s3aFS;
   private Path testData;
-  private S3AFileStatus testDataStatus;
+  private FileStatus testDataStatus;
   private FSDataInputStream in;
   private S3AInstrumentation.InputStreamStatistics streamStatistics;
   public static final int BLOCK_SIZE = 32 * 1024;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java
index 0f844b1..b2a1aa0 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java
@@ -126,7 +126,7 @@ public class S3AScaleTestBase extends AbstractS3ATestBase {
    * @return a configuration with which to create FS instances
    */
   protected Configuration createScaleConfiguration() {
-    return new Configuration();
+    return super.createConfiguration();
   }
 
   protected Path getTestPath() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/resources/core-site.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/resources/core-site.xml b/hadoop-tools/hadoop-aws/src/test/resources/core-site.xml
index d424aa4..e8200da 100644
--- a/hadoop-tools/hadoop-aws/src/test/resources/core-site.xml
+++ b/hadoop-tools/hadoop-aws/src/test/resources/core-site.xml
@@ -36,6 +36,25 @@
     <description>The endpoint for s3a://landsat-pds URLs</description>
   </property>
 
+  <!-- Make sure S3Guard is disabled for read-only bucket tests. -->
+  <property>
+    <name>fs.s3a.bucket.landsat-pds.metadatastore.impl</name>
+    <value>${s3guard.null}</value>
+    <description>The read-only landsat-pds repository isn't
+        managed by s3guard</description>
+  </property>
+
+  <!-- Convenience definitions. -->
+  <property>
+    <name>s3guard.null</name>
+    <value>org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore</value>
+  </property>
+
+  <property>
+    <name>s3guard.dynamo</name>
+    <value>org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStore</value>
+  </property>
+
   <!--
    This is the default endpoint, which can be used to interact
    with any v2 region.
@@ -110,6 +129,13 @@
     <value>${central.endpoint}</value>
   </property>
 
+  <!-- Scale integration tests may time out on slower connections
+       you can reduce the operation count like so to mitigate this.
+  <property>
+      <name>scale.test.operation.count</name>
+      <value>500</value>
+  </property>
+  -->
 
   <!-- Turn security off for tests by default -->
   <property>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties
index 1330ed1..9376ebd 100644
--- a/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties
+++ b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties
@@ -19,5 +19,16 @@ log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} (%F:%
 
 log4j.logger.org.apache.hadoop.util.NativeCodeLoader=ERROR
 
-# for debugging low level S3a operations, uncomment this line
-# log4j.logger.org.apache.hadoop.fs.s3a=DEBUG
+# for debugging low level S3a operations, uncomment these lines
+# Log all S3A classes
+#log4j.logger.org.apache.hadoop.fs.s3a=DEBUG
+
+# Log S3Guard classes
+#log4j.logger.org.apache.hadoop.fs.s3a.s3guard=DEBUG
+
+# Enable debug logging of AWS DynamoDB client
+#log4j.logger.com.amazonaws.services.dynamodbv2.AmazonDynamoDB=DEBUG
+
+# Log all HTTP requests made; includes S3 interaction. This may
+# include sensitive information such as account IDs in HTTP headers.
+#log4j.logger.com.amazonaws.request=DEBUG


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


[6/8] hadoop git commit: HADOOP-13345 S3Guard: Improved Consistency for S3A. Contributed by: Chris Nauroth, Aaron Fabbri, Mingliang Liu, Lei (Eddy) Xu, Sean Mackrory, Steve Loughran and others.

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBClientFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBClientFactory.java
new file mode 100644
index 0000000..66ada49
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBClientFactory.java
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.s3guard;
+
+import java.io.IOException;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.regions.Regions;
+import com.amazonaws.services.dynamodbv2.AmazonDynamoDB;
+import com.amazonaws.services.dynamodbv2.AmazonDynamoDBClientBuilder;
+import com.google.common.base.Preconditions;
+import org.apache.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.s3a.DefaultS3ClientFactory;
+
+import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_DDB_REGION_KEY;
+import static org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProviderSet;
+
+/**
+ * Interface to create a DynamoDB client.
+ *
+ * Implementation should be configured for setting and getting configuration.
+ */
+@InterfaceAudience.Private
+public interface DynamoDBClientFactory extends Configurable {
+  Logger LOG = LoggerFactory.getLogger(DynamoDBClientFactory.class);
+
+  /**
+   * Create a DynamoDB client object from configuration.
+   *
+   * The DynamoDB client to create does not have to relate to any S3 buckets.
+   * All information needed to create a DynamoDB client is from the hadoop
+   * configuration. Specially, if the region is not configured, it will use the
+   * provided region parameter. If region is neither configured nor provided,
+   * it will indicate an error.
+   *
+   * @param defaultRegion the default region of the AmazonDynamoDB client
+   * @return a new DynamoDB client
+   * @throws IOException if any IO error happens
+   */
+  AmazonDynamoDB createDynamoDBClient(String defaultRegion) throws IOException;
+
+  /**
+   * The default implementation for creating an AmazonDynamoDB.
+   */
+  class DefaultDynamoDBClientFactory extends Configured
+      implements DynamoDBClientFactory {
+    @Override
+    public AmazonDynamoDB createDynamoDBClient(String defaultRegion)
+        throws IOException {
+      Preconditions.checkNotNull(getConf(),
+          "Should have been configured before usage");
+
+      final Configuration conf = getConf();
+      final AWSCredentialsProvider credentials =
+          createAWSCredentialProviderSet(null, conf);
+      final ClientConfiguration awsConf =
+          DefaultS3ClientFactory.createAwsConf(conf);
+
+      final String region = getRegion(conf, defaultRegion);
+      LOG.debug("Creating DynamoDB client in region {}", region);
+
+      return AmazonDynamoDBClientBuilder.standard()
+          .withCredentials(credentials)
+          .withClientConfiguration(awsConf)
+          .withRegion(region)
+          .build();
+    }
+
+    /**
+     * Helper method to get and validate the AWS region for DynamoDBClient.
+     *
+     * @param conf configuration
+     * @param defaultRegion the default region
+     * @return configured region or else the provided default region
+     * @throws IOException if the region is not valid
+     */
+    static String getRegion(Configuration conf, String defaultRegion)
+        throws IOException {
+      String region = conf.getTrimmed(S3GUARD_DDB_REGION_KEY);
+      if (StringUtils.isEmpty(region)) {
+        region = defaultRegion;
+      }
+      try {
+        Regions.fromName(region);
+      } catch (IllegalArgumentException | NullPointerException e) {
+        throw new IOException("Invalid region specified: " + region + "; " +
+            "Region can be configured with " + S3GUARD_DDB_REGION_KEY + ": " +
+            validRegionsString());
+      }
+      return region;
+    }
+
+    private static String validRegionsString() {
+      final String delimiter = ", ";
+      Regions[] regions = Regions.values();
+      StringBuilder sb = new StringBuilder();
+      for (int i = 0; i < regions.length; i++) {
+        if (i > 0) {
+          sb.append(delimiter);
+        }
+        sb.append(regions[i].getName());
+      }
+      return sb.toString();
+
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..1bed03d
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java
@@ -0,0 +1,1010 @@
+/*
+ * 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.FileNotFoundException;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import com.amazonaws.AmazonClientException;
+import com.amazonaws.services.dynamodbv2.AmazonDynamoDB;
+import com.amazonaws.services.dynamodbv2.document.BatchWriteItemOutcome;
+import com.amazonaws.services.dynamodbv2.document.DynamoDB;
+import com.amazonaws.services.dynamodbv2.document.Item;
+import com.amazonaws.services.dynamodbv2.document.ItemCollection;
+import com.amazonaws.services.dynamodbv2.document.PrimaryKey;
+import com.amazonaws.services.dynamodbv2.document.PutItemOutcome;
+import com.amazonaws.services.dynamodbv2.document.QueryOutcome;
+import com.amazonaws.services.dynamodbv2.document.ScanOutcome;
+import com.amazonaws.services.dynamodbv2.document.Table;
+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.CreateTableRequest;
+import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughput;
+import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughputDescription;
+import com.amazonaws.services.dynamodbv2.model.ResourceInUseException;
+import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException;
+import com.amazonaws.services.dynamodbv2.model.WriteRequest;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.Constants;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.S3AInstrumentation;
+import org.apache.hadoop.fs.s3a.Tristate;
+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.ReflectionUtils;
+
+import static org.apache.hadoop.fs.s3a.Constants.*;
+import static org.apache.hadoop.fs.s3a.S3AUtils.translateException;
+import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.*;
+import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.*;
+
+/**
+ * DynamoDBMetadataStore is a {@link MetadataStore} that persists
+ * file system metadata to DynamoDB.
+ *
+ * The current implementation uses a schema consisting of a single table.  The
+ * name of the table can be configured by config key
+ * {@link org.apache.hadoop.fs.s3a.Constants#S3GUARD_DDB_TABLE_NAME_KEY}.
+ * By default, it matches the name of the S3 bucket.  Each item in the table
+ * represents a single directory or file.  Its path is split into separate table
+ * attributes:
+ * <ul>
+ * <li> parent (absolute path of the parent, with bucket name inserted as
+ * first path component). </li>
+ * <li> child (path of that specific child, relative to parent). </li>
+ * <li> optional boolean attribute tracking whether the path is a directory.
+ *      Absence or a false value indicates the path is a file. </li>
+ * <li> optional long attribute revealing modification time of file.
+ *      This attribute is meaningful only to file items.</li>
+ * <li> optional long attribute revealing file length.
+ *      This attribute is meaningful only to file items.</li>
+ * <li> optional long attribute revealing block size of the file.
+ *      This attribute is meaningful only to file items.</li>
+ * </ul>
+ *
+ * The DynamoDB partition key is the parent, and the range key is the child.
+ *
+ * To allow multiple buckets to share the same DynamoDB table, the bucket
+ * name is treated as the root directory.
+ *
+ * For example, assume the consistent store contains metadata representing this
+ * file system structure:
+ *
+ * <pre>
+ * s3a://bucket/dir1
+ * |-- dir2
+ * |   |-- file1
+ * |   `-- file2
+ * `-- dir3
+ *     |-- dir4
+ *     |   `-- file3
+ *     |-- dir5
+ *     |   `-- file4
+ *     `-- dir6
+ * </pre>
+ *
+ * This is persisted to a single DynamoDB table as:
+ *
+ * <pre>
+ * =========================================================================
+ * | parent                 | child | is_dir | mod_time | len |     ...    |
+ * =========================================================================
+ * | /bucket                | dir1  | true   |          |     |            |
+ * | /bucket/dir1           | dir2  | true   |          |     |            |
+ * | /bucket/dir1           | dir3  | true   |          |     |            |
+ * | /bucket/dir1/dir2      | file1 |        |   100    | 111 |            |
+ * | /bucket/dir1/dir2      | file2 |        |   200    | 222 |            |
+ * | /bucket/dir1/dir3      | dir4  | true   |          |     |            |
+ * | /bucket/dir1/dir3      | dir5  | true   |          |     |            |
+ * | /bucket/dir1/dir3/dir4 | file3 |        |   300    | 333 |            |
+ * | /bucket/dir1/dir3/dir5 | file4 |        |   400    | 444 |            |
+ * | /bucket/dir1/dir3      | dir6  | true   |          |     |            |
+ * =========================================================================
+ * </pre>
+ *
+ * This choice of schema is efficient for read access patterns.
+ * {@link #get(Path)} can be served from a single item lookup.
+ * {@link #listChildren(Path)} can be served from a query against all rows
+ * matching the parent (the partition key) and the returned list is guaranteed
+ * to be sorted by child (the range key).  Tracking whether or not a path is a
+ * directory helps prevent unnecessary queries during traversal of an entire
+ * sub-tree.
+ *
+ * Some mutating operations, notably {@link #deleteSubtree(Path)} and
+ * {@link #move(Collection, Collection)}, are less efficient with this schema.
+ * They require mutating multiple items in the DynamoDB table.
+ *
+ * By default, DynamoDB access is performed within the same AWS region as
+ * the S3 bucket that hosts the S3A instance.  During initialization, it checks
+ * the location of the S3 bucket and creates a DynamoDB client connected to the
+ * same region. The region may also be set explicitly by setting the config
+ * parameter {@code fs.s3a.s3guard.ddb.region} to the corresponding region.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class DynamoDBMetadataStore implements MetadataStore {
+  public static final Logger LOG = LoggerFactory.getLogger(
+      DynamoDBMetadataStore.class);
+
+  /** parent/child name to use in the version marker. */
+  public static final String VERSION_MARKER = "../VERSION";
+
+  /** Current version number. */
+  public static final int VERSION = 100;
+
+  /** Error: version marker not found in table. */
+  public static final String E_NO_VERSION_MARKER
+      = "S3Guard table lacks version marker.";
+
+  /** Error: version mismatch. */
+  public static final String E_INCOMPATIBLE_VERSION
+      = "Database table is from an incompatible S3Guard version.";
+
+  /** Initial delay for retries when batched operations get throttled by
+   * DynamoDB. Value is {@value} msec. */
+  public static final long MIN_RETRY_SLEEP_MSEC = 100;
+
+  private static ValueMap deleteTrackingValueMap =
+      new ValueMap().withBoolean(":false", false);
+
+  private DynamoDB dynamoDB;
+  private String region;
+  private Table table;
+  private String tableName;
+  private Configuration conf;
+  private String username;
+
+  private RetryPolicy dataAccessRetryPolicy;
+  private S3AInstrumentation.S3GuardInstrumentation instrumentation;
+
+  /**
+   * A utility function to create DynamoDB instance.
+   * @param conf the file system configuration
+   * @param s3Region region of the associated S3 bucket (if any).
+   * @return DynamoDB instance.
+   * @throws IOException I/O error.
+   */
+  private static DynamoDB createDynamoDB(Configuration conf, String s3Region)
+      throws IOException {
+    Preconditions.checkNotNull(conf);
+    final Class<? extends DynamoDBClientFactory> cls = conf.getClass(
+        S3GUARD_DDB_CLIENT_FACTORY_IMPL,
+        S3GUARD_DDB_CLIENT_FACTORY_IMPL_DEFAULT,
+        DynamoDBClientFactory.class);
+    LOG.debug("Creating DynamoDB client {} with S3 region {}", cls, s3Region);
+    final AmazonDynamoDB dynamoDBClient = ReflectionUtils.newInstance(cls, conf)
+        .createDynamoDBClient(s3Region);
+    return new DynamoDB(dynamoDBClient);
+  }
+
+  @Override
+  public void initialize(FileSystem fs) throws IOException {
+    Preconditions.checkArgument(fs instanceof S3AFileSystem,
+        "DynamoDBMetadataStore only supports S3A filesystem.");
+    final S3AFileSystem s3afs = (S3AFileSystem) fs;
+    instrumentation = s3afs.getInstrumentation().getS3GuardInstrumentation();
+    final String bucket = s3afs.getBucket();
+    String confRegion = s3afs.getConf().getTrimmed(S3GUARD_DDB_REGION_KEY);
+    if (!StringUtils.isEmpty(confRegion)) {
+      region = confRegion;
+      LOG.debug("Overriding S3 region with configured DynamoDB region: {}",
+          region);
+    } else {
+      region = s3afs.getBucketLocation();
+      LOG.debug("Inferring DynamoDB region from S3 bucket: {}", region);
+    }
+    username = s3afs.getUsername();
+    conf = s3afs.getConf();
+    dynamoDB = createDynamoDB(conf, region);
+
+    // use the bucket as the DynamoDB table name if not specified in config
+    tableName = conf.getTrimmed(S3GUARD_DDB_TABLE_NAME_KEY, bucket);
+    setMaxRetries(conf);
+
+    initTable();
+
+    instrumentation.initialized();
+  }
+
+  /**
+   * Performs one-time initialization of the metadata store via configuration.
+   *
+   * This initialization depends on the configuration object to get AWS
+   * credentials, DynamoDBFactory implementation class, DynamoDB endpoints,
+   * DynamoDB table names etc. After initialization, this metadata store does
+   * not explicitly relate to any S3 bucket, which be nonexistent.
+   *
+   * This is used to operate the metadata store directly beyond the scope of the
+   * S3AFileSystem integration, e.g. command line tools.
+   * Generally, callers should use {@link #initialize(FileSystem)}
+   * with an initialized {@code S3AFileSystem} instance.
+   *
+   * Without a filesystem to act as a reference point, the configuration itself
+   * must declare the table name and region in the
+   * {@link Constants#S3GUARD_DDB_TABLE_NAME_KEY} and
+   * {@link Constants#S3GUARD_DDB_REGION_KEY} respectively.
+   *
+   * @see #initialize(FileSystem)
+   * @throws IOException if there is an error
+   * @throws IllegalArgumentException if the configuration is incomplete
+   */
+  @Override
+  public void initialize(Configuration config) throws IOException {
+    conf = config;
+    // use the bucket as the DynamoDB table name if not specified in config
+    tableName = conf.getTrimmed(S3GUARD_DDB_TABLE_NAME_KEY);
+    Preconditions.checkArgument(!StringUtils.isEmpty(tableName),
+        "No DynamoDB table name configured");
+    region = conf.getTrimmed(S3GUARD_DDB_REGION_KEY);
+    Preconditions.checkArgument(!StringUtils.isEmpty(region),
+        "No DynamoDB region configured");
+    dynamoDB = createDynamoDB(conf, region);
+
+    username = UserGroupInformation.getCurrentUser().getShortUserName();
+    setMaxRetries(conf);
+
+    initTable();
+  }
+
+  /**
+   * Set retry policy. This is driven by the value of
+   * {@link Constants#S3GUARD_DDB_MAX_RETRIES} with an exponential backoff
+   * between each attempt of {@link #MIN_RETRY_SLEEP_MSEC} milliseconds.
+   * @param config
+   */
+  private void setMaxRetries(Configuration config) {
+    int maxRetries = config.getInt(S3GUARD_DDB_MAX_RETRIES,
+        S3GUARD_DDB_MAX_RETRIES_DEFAULT);
+    dataAccessRetryPolicy = RetryPolicies
+        .exponentialBackoffRetry(maxRetries, MIN_RETRY_SLEEP_MSEC,
+            TimeUnit.MILLISECONDS);
+  }
+
+  @Override
+  public void delete(Path path) throws IOException {
+    innerDelete(path, true);
+  }
+
+  @Override
+  public void forgetMetadata(Path path) throws IOException {
+    innerDelete(path, false);
+  }
+
+  /**
+   * Inner delete option, action based on the {@code tombstone} flag.
+   * No tombstone: delete the entry. Tombstone: create a tombstone entry.
+   * There is no check as to whether the entry exists in the table first.
+   * @param path path to delete
+   * @param tombstone flag to create a tombstone marker
+   * @throws IOException I/O error.
+   */
+  private void innerDelete(Path path, boolean tombstone)
+      throws IOException {
+    path = checkPath(path);
+    LOG.debug("Deleting from table {} in region {}: {}",
+        tableName, region, path);
+
+    // deleting nonexistent item consumes 1 write capacity; skip it
+    if (path.isRoot()) {
+      LOG.debug("Skip deleting root directory as it does not exist in table");
+      return;
+    }
+
+    try {
+      if (tombstone) {
+        Item item = PathMetadataDynamoDBTranslation.pathMetadataToItem(
+            PathMetadata.tombstone(path));
+        table.putItem(item);
+      } else {
+        table.deleteItem(pathToKey(path));
+      }
+    } catch (AmazonClientException e) {
+      throw translateException("delete", path, e);
+    }
+  }
+
+  @Override
+  public void deleteSubtree(Path path) throws IOException {
+    path = checkPath(path);
+    LOG.debug("Deleting subtree from table {} in region {}: {}",
+        tableName, region, path);
+
+    final PathMetadata meta = get(path);
+    if (meta == null || meta.isDeleted()) {
+      LOG.debug("Subtree path {} does not exist; this will be a no-op", path);
+      return;
+    }
+
+    for (DescendantsIterator desc = new DescendantsIterator(this, meta);
+         desc.hasNext();) {
+      innerDelete(desc.next().getPath(), true);
+    }
+  }
+
+  private Item getConsistentItem(PrimaryKey key) {
+    final GetItemSpec spec = new GetItemSpec()
+        .withPrimaryKey(key)
+        .withConsistentRead(true); // strictly consistent read
+    return table.getItem(spec);
+  }
+
+  @Override
+  public PathMetadata get(Path path) throws IOException {
+    return get(path, false);
+  }
+
+  @Override
+  public PathMetadata get(Path path, boolean wantEmptyDirectoryFlag)
+      throws IOException {
+    path = checkPath(path);
+    LOG.debug("Get from table {} in region {}: {}", tableName, region, path);
+
+    try {
+      final PathMetadata meta;
+      if (path.isRoot()) {
+        // Root does not persist in the table
+        meta = new PathMetadata(makeDirStatus(username, path));
+      } else {
+        final Item item = getConsistentItem(pathToKey(path));
+        meta = itemToPathMetadata(item, username);
+        LOG.debug("Get from table {} in region {} returning for {}: {}",
+            tableName, region, path, meta);
+      }
+
+      if (wantEmptyDirectoryFlag && meta != null) {
+        final FileStatus status = meta.getFileStatus();
+        // for directory, we query its direct children to determine isEmpty bit
+        if (status.isDirectory()) {
+          final QuerySpec spec = new QuerySpec()
+              .withHashKey(pathToParentKeyAttribute(path))
+              .withConsistentRead(true)
+              .withFilterExpression(IS_DELETED + " = :false")
+              .withValueMap(deleteTrackingValueMap);
+          final ItemCollection<QueryOutcome> items = table.query(spec);
+          boolean hasChildren = items.iterator().hasNext();
+          // When this class has support for authoritative
+          // (fully-cached) directory listings, we may also be able to answer
+          // TRUE here.  Until then, we don't know if we have full listing or
+          // not, thus the UNKNOWN here:
+          meta.setIsEmptyDirectory(
+              hasChildren ? Tristate.FALSE : Tristate.UNKNOWN);
+        }
+      }
+
+      return meta;
+    } catch (AmazonClientException e) {
+      throw translateException("get", path, e);
+    }
+  }
+
+  /**
+   * Make a FileStatus object for a directory at given path.  The FileStatus
+   * only contains what S3A needs, and omits mod time since S3A uses its own
+   * implementation which returns current system time.
+   * @param owner  username of owner
+   * @param path   path to dir
+   * @return new FileStatus
+   */
+  private FileStatus makeDirStatus(String owner, Path path) {
+    return new FileStatus(0, true, 1, 0, 0, 0, null,
+            owner, null, path);
+  }
+
+  @Override
+  public DirListingMetadata listChildren(Path path) throws IOException {
+    path = checkPath(path);
+    LOG.debug("Listing table {} in region {}: {}", tableName, region, path);
+
+    // find the children in the table
+    try {
+      final QuerySpec spec = new QuerySpec()
+          .withHashKey(pathToParentKeyAttribute(path))
+          .withConsistentRead(true); // strictly consistent read
+      final ItemCollection<QueryOutcome> items = table.query(spec);
+
+      final List<PathMetadata> metas = new ArrayList<>();
+      for (Item item : items) {
+        PathMetadata meta = itemToPathMetadata(item, username);
+        metas.add(meta);
+      }
+      LOG.trace("Listing table {} in region {} for {} returning {}",
+          tableName, region, path, metas);
+
+      return (metas.isEmpty() && get(path) == null)
+          ? null
+          : new DirListingMetadata(path, metas, false);
+    } catch (AmazonClientException e) {
+      // failure, including the path not being present
+      throw translateException("listChildren", path, e);
+    }
+  }
+
+  // build the list of all parent entries.
+  Collection<PathMetadata> completeAncestry(
+      Collection<PathMetadata> pathsToCreate) {
+    // Key on path to allow fast lookup
+    Map<Path, PathMetadata> ancestry = new HashMap<>();
+
+    for (PathMetadata meta : pathsToCreate) {
+      Preconditions.checkArgument(meta != null);
+      Path path = meta.getFileStatus().getPath();
+      if (path.isRoot()) {
+        break;
+      }
+      ancestry.put(path, meta);
+      Path parent = path.getParent();
+      while (!parent.isRoot() && !ancestry.containsKey(parent)) {
+        LOG.debug("auto-create ancestor path {} for child path {}",
+            parent, path);
+        final FileStatus status = makeDirStatus(parent, username);
+        ancestry.put(parent, new PathMetadata(status, Tristate.FALSE, false));
+        parent = parent.getParent();
+      }
+    }
+    return ancestry.values();
+  }
+
+  @Override
+  public void move(Collection<Path> pathsToDelete,
+      Collection<PathMetadata> pathsToCreate) throws IOException {
+    if (pathsToDelete == null && pathsToCreate == null) {
+      return;
+    }
+
+    LOG.debug("Moving paths of table {} in region {}: {} paths to delete and {}"
+        + " paths to create", tableName, region,
+        pathsToDelete == null ? 0 : pathsToDelete.size(),
+        pathsToCreate == null ? 0 : pathsToCreate.size());
+    LOG.trace("move: pathsToDelete = {}, pathsToCreate = {}", pathsToDelete,
+        pathsToCreate);
+
+    // In DynamoDBMetadataStore implementation, we assume that if a path
+    // exists, all its ancestors will also exist in the table.
+    // 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<PathMetadata> newItems = new ArrayList<>();
+    if (pathsToCreate != null) {
+      newItems.addAll(completeAncestry(pathsToCreate));
+    }
+    if (pathsToDelete != null) {
+      for (Path meta : pathsToDelete) {
+        newItems.add(PathMetadata.tombstone(meta));
+      }
+    }
+
+    try {
+      processBatchWriteRequest(null, pathMetadataToItem(newItems));
+    } catch (AmazonClientException e) {
+      throw translateException("move", (String) null, e);
+    }
+  }
+
+  /**
+   * Helper method to issue a batch write request to DynamoDB.
+   *
+   * Callers of this method should catch the {@link AmazonClientException} and
+   * translate it for better error report and easier debugging.
+   * @param keysToDelete primary keys to be deleted; can be null
+   * @param itemsToPut new items to be put; can be null
+   */
+  private void processBatchWriteRequest(PrimaryKey[] keysToDelete,
+      Item[] itemsToPut) throws IOException {
+    final int totalToDelete = (keysToDelete == null ? 0 : keysToDelete.length);
+    final int totalToPut = (itemsToPut == null ? 0 : itemsToPut.length);
+    int count = 0;
+    while (count < totalToDelete + totalToPut) {
+      final TableWriteItems writeItems = new TableWriteItems(tableName);
+      int numToDelete = 0;
+      if (keysToDelete != null
+          && count < totalToDelete) {
+        numToDelete = Math.min(S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT,
+            totalToDelete - count);
+        writeItems.withPrimaryKeysToDelete(
+            Arrays.copyOfRange(keysToDelete, count, count + numToDelete));
+        count += numToDelete;
+      }
+
+      if (numToDelete < S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT
+          && itemsToPut != null
+          && count < totalToDelete + totalToPut) {
+        final int numToPut = Math.min(
+            S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT - numToDelete,
+            totalToDelete + totalToPut - count);
+        final int index = count - totalToDelete;
+        writeItems.withItemsToPut(
+            Arrays.copyOfRange(itemsToPut, index, index + numToPut));
+        count += numToPut;
+      }
+
+      BatchWriteItemOutcome res = dynamoDB.batchWriteItem(writeItems);
+      // Check for unprocessed keys in case of exceeding provisioned throughput
+      Map<String, List<WriteRequest>> unprocessed = res.getUnprocessedItems();
+      int retryCount = 0;
+      while (unprocessed.size() > 0) {
+        retryBackoff(retryCount++);
+        res = dynamoDB.batchWriteItemUnprocessed(unprocessed);
+        unprocessed = res.getUnprocessedItems();
+      }
+    }
+  }
+
+  /**
+   * Put the current thread to sleep to implement exponential backoff
+   * depending on retryCount.  If max retries are exceeded, throws an
+   * exception instead.
+   * @param retryCount number of retries so far
+   * @throws IOException when max retryCount is exceeded.
+   */
+  private void retryBackoff(int retryCount) throws IOException {
+    try {
+      // Our RetryPolicy ignores everything but retryCount here.
+      RetryPolicy.RetryAction action = dataAccessRetryPolicy.shouldRetry(null,
+          retryCount, 0, true);
+      if (action.action == RetryPolicy.RetryAction.RetryDecision.FAIL) {
+        throw new IOException(
+            String.format("Max retries exceeded (%d) for DynamoDB",
+                retryCount));
+      } else {
+        LOG.debug("Sleeping {} msec before next retry", action.delayMillis);
+        Thread.sleep(action.delayMillis);
+      }
+    } catch (Exception e) {
+      throw new IOException("Unexpected exception", e);
+    }
+  }
+
+  @Override
+  public void put(PathMetadata meta) 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,
+    // all its ancestors will also exist in the table.
+    // For performance purpose, we generate the full paths to put and use batch
+    // write item request to save the items.
+    LOG.debug("Saving to table {} in region {}: {}", tableName, region, meta);
+
+    Collection<PathMetadata> wrapper = new ArrayList<>(1);
+    wrapper.add(meta);
+    put(wrapper);
+  }
+
+  @Override
+  public void put(Collection<PathMetadata> metas) throws IOException {
+    LOG.debug("Saving batch to table {} in region {}", tableName, region);
+
+    processBatchWriteRequest(null, pathMetadataToItem(completeAncestry(metas)));
+  }
+
+  /**
+   * Helper method to get full path of ancestors that are nonexistent in table.
+   */
+  private Collection<PathMetadata> fullPathsToPut(PathMetadata meta)
+      throws IOException {
+    checkPathMetadata(meta);
+    final Collection<PathMetadata> metasToPut = new ArrayList<>();
+    // root path is not persisted
+    if (!meta.getFileStatus().getPath().isRoot()) {
+      metasToPut.add(meta);
+    }
+
+    // put all its ancestors if not present; as an optimization we return at its
+    // first existent ancestor
+    Path path = meta.getFileStatus().getPath().getParent();
+    while (path != null && !path.isRoot()) {
+      final Item item = getConsistentItem(pathToKey(path));
+      if (!itemExists(item)) {
+        final FileStatus status = makeDirStatus(path, username);
+        metasToPut.add(new PathMetadata(status, Tristate.FALSE, false));
+        path = path.getParent();
+      } else {
+        break;
+      }
+    }
+    return metasToPut;
+  }
+
+  private boolean itemExists(Item item) {
+    if (item == null) {
+      return false;
+    }
+    if (item.hasAttribute(IS_DELETED) &&
+        item.getBoolean(IS_DELETED)) {
+      return false;
+    }
+    return true;
+  }
+
+  /** Create a directory FileStatus using current system time as mod time. */
+  static FileStatus makeDirStatus(Path f, String owner) {
+    return  new FileStatus(0, true, 1, 0, System.currentTimeMillis(), 0,
+        null, owner, owner, f);
+  }
+
+  @Override
+  public void put(DirListingMetadata meta) throws IOException {
+    LOG.debug("Saving to table {} in region {}: {}", tableName, region, meta);
+
+    // directory path
+    PathMetadata p = new PathMetadata(makeDirStatus(meta.getPath(), username),
+        meta.isEmpty(), false);
+
+    // First add any missing ancestors...
+    final Collection<PathMetadata> metasToPut = fullPathsToPut(p);
+
+    // next add all children of the directory
+    metasToPut.addAll(meta.getListing());
+
+    try {
+      processBatchWriteRequest(null, pathMetadataToItem(metasToPut));
+    } catch (AmazonClientException e) {
+      throw translateException("put", (String) null, e);
+    }
+  }
+
+  @Override
+  public synchronized void close() {
+    if (instrumentation != null) {
+      instrumentation.storeClosed();
+    }
+    if (dynamoDB != null) {
+      LOG.debug("Shutting down {}", this);
+      dynamoDB.shutdown();
+      dynamoDB = null;
+    }
+  }
+
+  @Override
+  public void destroy() throws IOException {
+    if (table == null) {
+      LOG.info("In destroy(): no table to delete");
+      return;
+    }
+    LOG.info("Deleting DynamoDB table {} in region {}", tableName, region);
+    Preconditions.checkNotNull(dynamoDB, "Not connected to DynamoDB");
+    try {
+      table.delete();
+      table.waitForDelete();
+    } catch (ResourceNotFoundException rnfe) {
+      LOG.info("ResourceNotFoundException while deleting DynamoDB table {} in "
+              + "region {}.  This may indicate that the table does not exist, "
+              + "or has been deleted by another concurrent thread or process.",
+          tableName, region);
+    } catch (InterruptedException ie) {
+      Thread.currentThread().interrupt();
+      LOG.warn("Interrupted while waiting for DynamoDB table {} being deleted",
+          tableName, ie);
+      throw new InterruptedIOException("Table " + tableName
+          + " in region " + region + " has not been deleted");
+    } catch (AmazonClientException e) {
+      throw translateException("destroy", (String) null, e);
+    }
+  }
+
+  private ItemCollection<ScanOutcome> expiredFiles(long modTime) {
+    String filterExpression = "mod_time < :mod_time";
+    String projectionExpression = "parent,child";
+    ValueMap map = new ValueMap().withLong(":mod_time", modTime);
+    return table.scan(filterExpression, projectionExpression, null, map);
+  }
+
+  @Override
+  public void prune(long modTime) throws IOException {
+    int itemCount = 0;
+    try {
+      Collection<Path> deletionBatch =
+          new ArrayList<>(S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT);
+      int delay = conf.getInt(S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY,
+          S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_DEFAULT);
+      for (Item item : expiredFiles(modTime)) {
+        PathMetadata md = PathMetadataDynamoDBTranslation
+            .itemToPathMetadata(item, username);
+        Path path = md.getFileStatus().getPath();
+        deletionBatch.add(path);
+        itemCount++;
+        if (deletionBatch.size() == S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT) {
+          Thread.sleep(delay);
+          processBatchWriteRequest(pathToKey(deletionBatch), null);
+          deletionBatch.clear();
+        }
+      }
+      if (deletionBatch.size() > 0) {
+        Thread.sleep(delay);
+        processBatchWriteRequest(pathToKey(deletionBatch), null);
+      }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new InterruptedIOException("Pruning was interrupted");
+    }
+    LOG.info("Finished pruning {} items in batches of {}", itemCount,
+        S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT);
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + '{'
+        + "region=" + region
+        + ", tableName=" + tableName
+        + '}';
+  }
+
+  /**
+   * Create a table if it does not exist and wait for it to become active.
+   *
+   * If a table with the intended name already exists, then it uses that table.
+   * Otherwise, it will automatically create the table if the config
+   * {@link org.apache.hadoop.fs.s3a.Constants#S3GUARD_DDB_TABLE_CREATE_KEY} is
+   * enabled. The DynamoDB table creation API is asynchronous.  This method wait
+   * for the table to become active after sending the creation request, so
+   * overall, this method is synchronous, and the table is guaranteed to exist
+   * after this method returns successfully.
+   *
+   * @throws IOException if table does not exist and auto-creation is disabled;
+   * or table is being deleted, or any other I/O exception occurred.
+   */
+  @VisibleForTesting
+  void initTable() throws IOException {
+    table = dynamoDB.getTable(tableName);
+    try {
+      try {
+        LOG.debug("Binding to table {}", tableName);
+        final String status = table.describe().getTableStatus();
+        switch (status) {
+        case "CREATING":
+        case "UPDATING":
+          LOG.debug("Table {} in region {} is being created/updated. This may"
+                  + " indicate that the table is being operated by another "
+                  + "concurrent thread or process. Waiting for active...",
+              tableName, region);
+          waitForTableActive(table);
+          break;
+        case "DELETING":
+          throw new FileNotFoundException("DynamoDB table "
+              + "'" + tableName + "' is being "
+              + "deleted in region " + region);
+        case "ACTIVE":
+          break;
+        default:
+          throw new IOException("Unknown DynamoDB table status " + status
+              + ": tableName='" + tableName + "', region=" + region);
+        }
+
+        final Item versionMarker = getVersionMarkerItem();
+        verifyVersionCompatibility(tableName, versionMarker);
+        Long created = extractCreationTimeFromMarker(versionMarker);
+        LOG.debug("Using existing DynamoDB table {} in region {} created {}",
+            tableName, region, (created != null) ? new Date(created) : null);
+      } catch (ResourceNotFoundException rnfe) {
+        if (conf.getBoolean(S3GUARD_DDB_TABLE_CREATE_KEY, false)) {
+          final ProvisionedThroughput capacity = new ProvisionedThroughput(
+              conf.getLong(S3GUARD_DDB_TABLE_CAPACITY_READ_KEY,
+                  S3GUARD_DDB_TABLE_CAPACITY_READ_DEFAULT),
+              conf.getLong(S3GUARD_DDB_TABLE_CAPACITY_WRITE_KEY,
+                  S3GUARD_DDB_TABLE_CAPACITY_WRITE_DEFAULT));
+
+          createTable(capacity);
+        } else {
+          throw new FileNotFoundException("DynamoDB table "
+              + "'" + tableName + "' does not "
+              + "exist in region " + region + "; auto-creation is turned off");
+        }
+      }
+
+    } catch (AmazonClientException e) {
+      throw translateException("initTable", (String) null, e);
+    }
+  }
+
+  /**
+   * Get the version mark item in the existing DynamoDB table.
+   *
+   * As the version marker item may be created by another concurrent thread or
+   * process, we retry a limited times before we fail to get it.
+   */
+  private Item getVersionMarkerItem() throws IOException {
+    final PrimaryKey versionMarkerKey =
+        createVersionMarkerPrimaryKey(VERSION_MARKER);
+    int retryCount = 0;
+    Item versionMarker = table.getItem(versionMarkerKey);
+    while (versionMarker == null) {
+      try {
+        RetryPolicy.RetryAction action = dataAccessRetryPolicy.shouldRetry(null,
+            retryCount, 0, true);
+        if (action.action == RetryPolicy.RetryAction.RetryDecision.FAIL) {
+          break;
+        } else {
+          LOG.debug("Sleeping {} ms before next retry", action.delayMillis);
+          Thread.sleep(action.delayMillis);
+        }
+      } catch (Exception e) {
+        throw new IOException("initTable: Unexpected exception", e);
+      }
+      retryCount++;
+      versionMarker = table.getItem(versionMarkerKey);
+    }
+    return versionMarker;
+  }
+
+  /**
+   * Verify that a table version is compatible with this S3Guard client.
+   * @param tableName name of the table (for error messages)
+   * @param versionMarker the version marker retrieved from the table
+   * @throws IOException on any incompatibility
+   */
+  @VisibleForTesting
+  static void verifyVersionCompatibility(String tableName,
+      Item versionMarker) throws IOException {
+    if (versionMarker == null) {
+      LOG.warn("Table {} contains no version marker", tableName);
+      throw new IOException(E_NO_VERSION_MARKER
+      + " Table: " + tableName);
+    } else {
+      final int version = extractVersionFromMarker(versionMarker);
+      if (VERSION != version) {
+        // version mismatch. Unless/until there is support for
+        // upgrading versions, treat this as an incompatible change
+        // and fail.
+        throw new IOException(E_INCOMPATIBLE_VERSION
+            + " Table "+  tableName
+            + " Expected version " + VERSION + " actual " + version);
+      }
+    }
+  }
+
+  /**
+   * Wait for table being active.
+   * @param t table to block on.
+   * @throws IOException IO problems
+   * @throws InterruptedIOException if the wait was interrupted
+   */
+  private void waitForTableActive(Table t) throws IOException {
+    try {
+      t.waitForActive();
+    } catch (InterruptedException e) {
+      LOG.warn("Interrupted while waiting for table {} in region {} active",
+          tableName, region, e);
+      Thread.currentThread().interrupt();
+      throw (IOException) new InterruptedIOException("DynamoDB table '"
+          + tableName + "' is not active yet in region " + region).initCause(e);
+    }
+  }
+
+  /**
+   * Create a table, wait for it to become active, then add the version
+   * marker.
+   * @param capacity capacity to provision
+   * @throws IOException on any failure.
+   * @throws InterruptedIOException if the wait was interrupted
+   */
+  private void createTable(ProvisionedThroughput capacity) throws IOException {
+    try {
+      LOG.info("Creating non-existent DynamoDB table {} in region {}",
+          tableName, region);
+      table = dynamoDB.createTable(new CreateTableRequest()
+          .withTableName(tableName)
+          .withKeySchema(keySchema())
+          .withAttributeDefinitions(attributeDefinitions())
+          .withProvisionedThroughput(capacity));
+      LOG.debug("Awaiting table becoming active");
+    } catch (ResourceInUseException e) {
+      LOG.warn("ResourceInUseException while creating DynamoDB table {} "
+              + "in region {}.  This may indicate that the table was "
+              + "created by another concurrent thread or process.",
+          tableName, region);
+    }
+    waitForTableActive(table);
+    final Item marker = createVersionMarker(VERSION_MARKER, VERSION,
+        System.currentTimeMillis());
+    putItem(marker);
+  }
+
+  /**
+   * PUT a single item to the table.
+   * @param item item to put
+   * @return the outcome.
+   */
+  PutItemOutcome putItem(Item item) {
+    LOG.debug("Putting item {}", item);
+    return table.putItem(item);
+  }
+
+  /**
+   * Provision the table with given read and write capacity units.
+   */
+  void provisionTable(Long readCapacity, Long writeCapacity)
+      throws IOException {
+    final ProvisionedThroughput toProvision = new ProvisionedThroughput()
+        .withReadCapacityUnits(readCapacity)
+        .withWriteCapacityUnits(writeCapacity);
+    try {
+      final ProvisionedThroughputDescription p =
+          table.updateTable(toProvision).getProvisionedThroughput();
+      LOG.info("Provision table {} in region {}: readCapacityUnits={}, "
+              + "writeCapacityUnits={}",
+          tableName, region, p.getReadCapacityUnits(),
+          p.getWriteCapacityUnits());
+    } catch (AmazonClientException e) {
+      throw translateException("provisionTable", (String) null, e);
+    }
+  }
+
+  Table getTable() {
+    return table;
+  }
+
+  String getRegion() {
+    return region;
+  }
+
+  @VisibleForTesting
+  DynamoDB getDynamoDB() {
+    return dynamoDB;
+  }
+
+  /**
+   * Validates a path object; it must be absolute, and contain a host
+   * (bucket) component.
+   */
+  private Path checkPath(Path path) {
+    Preconditions.checkNotNull(path);
+    Preconditions.checkArgument(path.isAbsolute(), "Path %s is not absolute",
+        path);
+    URI uri = path.toUri();
+    Preconditions.checkNotNull(uri.getScheme(), "Path %s missing scheme", path);
+    Preconditions.checkArgument(uri.getScheme().equals(Constants.FS_S3A),
+        "Path %s scheme must be %s", path, Constants.FS_S3A);
+    Preconditions.checkArgument(!StringUtils.isEmpty(uri.getHost()), "Path %s" +
+        " is missing bucket.", path);
+    return path;
+  }
+
+  /**
+   * Validates a path meta-data object.
+   */
+  private static void checkPathMetadata(PathMetadata meta) {
+    Preconditions.checkNotNull(meta);
+    Preconditions.checkNotNull(meta.getFileStatus());
+    Preconditions.checkNotNull(meta.getFileStatus().getPath());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..1ef8b0d
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java
@@ -0,0 +1,435 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.Tristate;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.Map;
+
+/**
+ * This is a local, in-memory, implementation of MetadataStore.
+ * This is <i>not</i> a coherent cache across processes.  It is only
+ * locally-coherent.
+ *
+ * The purpose of this is for unit and integration testing.
+ * It could also be used to accelerate local-only operations where only one
+ * process is operating on a given object store, or multiple processes are
+ * accessing a read-only storage bucket.
+ *
+ * This MetadataStore does not enforce filesystem rules such as disallowing
+ * non-recursive removal of non-empty directories.  It is assumed the caller
+ * already has to perform these sorts of checks.
+ */
+public class LocalMetadataStore implements MetadataStore {
+
+  public static final Logger LOG = LoggerFactory.getLogger(MetadataStore.class);
+  // TODO HADOOP-13649: use time instead of capacity for eviction.
+  public static final int DEFAULT_MAX_RECORDS = 128;
+
+  /**
+   * Maximum number of records.
+   */
+  public static final String CONF_MAX_RECORDS =
+      "fs.metadatastore.local.max_records";
+
+  /** Contains directories and files. */
+  private LruHashMap<Path, PathMetadata> fileHash;
+
+  /** Contains directory listings. */
+  private LruHashMap<Path, DirListingMetadata> dirHash;
+
+  private FileSystem fs;
+  /* Null iff this FS does not have an associated URI host. */
+  private String uriHost;
+
+  @Override
+  public void initialize(FileSystem fileSystem) throws IOException {
+    Preconditions.checkNotNull(fileSystem);
+    fs = fileSystem;
+    URI fsURI = fs.getUri();
+    uriHost = fsURI.getHost();
+    if (uriHost != null && uriHost.equals("")) {
+      uriHost = null;
+    }
+
+    initialize(fs.getConf());
+  }
+
+  @Override
+  public void initialize(Configuration conf) throws IOException {
+    Preconditions.checkNotNull(conf);
+    int maxRecords = conf.getInt(CONF_MAX_RECORDS, DEFAULT_MAX_RECORDS);
+    if (maxRecords < 4) {
+      maxRecords = 4;
+    }
+    // Start w/ less than max capacity.  Space / time trade off.
+    fileHash = new LruHashMap<>(maxRecords/2, maxRecords);
+    dirHash = new LruHashMap<>(maxRecords/4, maxRecords);
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "LocalMetadataStore{");
+    sb.append(", uriHost='").append(uriHost).append('\'');
+    sb.append('}');
+    return sb.toString();
+  }
+
+  @Override
+  public void delete(Path p) throws IOException {
+    doDelete(p, false, true);
+  }
+
+  @Override
+  public void forgetMetadata(Path p) throws IOException {
+    doDelete(p, false, false);
+  }
+
+  @Override
+  public void deleteSubtree(Path path) throws IOException {
+    doDelete(path, true, true);
+  }
+
+  private synchronized void doDelete(Path p, boolean recursive, boolean
+      tombstone) {
+
+    Path path = standardize(p);
+
+    // Delete entry from file cache, then from cached parent directory, if any
+
+    deleteHashEntries(path, tombstone);
+
+    if (recursive) {
+      // Remove all entries that have this dir as path prefix.
+      deleteHashByAncestor(path, dirHash, tombstone);
+      deleteHashByAncestor(path, fileHash, tombstone);
+    }
+  }
+
+  @Override
+  public synchronized PathMetadata get(Path p) throws IOException {
+    return get(p, false);
+  }
+
+  @Override
+  public PathMetadata get(Path p, boolean wantEmptyDirectoryFlag)
+      throws IOException {
+    Path path = standardize(p);
+    synchronized (this) {
+      PathMetadata m = fileHash.mruGet(path);
+
+      if (wantEmptyDirectoryFlag && m != null &&
+          m.getFileStatus().isDirectory()) {
+        m.setIsEmptyDirectory(isEmptyDirectory(p));
+      }
+
+      LOG.debug("get({}) -> {}", path, m == null ? "null" : m.prettyPrint());
+      return m;
+    }
+  }
+
+  /**
+   * Determine if directory is empty.
+   * Call with lock held.
+   * @param p a Path, already filtered through standardize()
+   * @return TRUE / FALSE if known empty / not-empty, UNKNOWN otherwise.
+   */
+  private Tristate isEmptyDirectory(Path p) {
+    DirListingMetadata dirMeta = dirHash.get(p);
+    return dirMeta.withoutTombstones().isEmpty();
+  }
+
+  @Override
+  public synchronized DirListingMetadata listChildren(Path p) throws
+      IOException {
+    Path path = standardize(p);
+    DirListingMetadata listing = dirHash.mruGet(path);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("listChildren({}) -> {}", path,
+          listing == null ? "null" : listing.prettyPrint());
+    }
+    // Make a copy so callers can mutate without affecting our state
+    return listing == null ? null : new DirListingMetadata(listing);
+  }
+
+  @Override
+  public void move(Collection<Path> pathsToDelete,
+      Collection<PathMetadata> pathsToCreate) throws IOException {
+
+    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.");
+
+    // I feel dirty for using reentrant lock. :-|
+    synchronized (this) {
+
+      // 1. Delete pathsToDelete
+      for (Path meta : pathsToDelete) {
+        LOG.debug("move: deleting metadata {}", meta);
+        delete(meta);
+      }
+
+      // 2. Create new destination path metadata
+      for (PathMetadata meta : pathsToCreate) {
+        LOG.debug("move: adding metadata {}", meta);
+        put(meta);
+      }
+
+      // 3. We now know full contents of all dirs in destination subtree
+      for (PathMetadata meta : pathsToCreate) {
+        FileStatus status = meta.getFileStatus();
+        if (status == null || status.isDirectory()) {
+          continue;
+        }
+        DirListingMetadata dir = listChildren(status.getPath());
+        if (dir != null) {  // could be evicted already
+          dir.setAuthoritative(true);
+        }
+      }
+    }
+  }
+
+  @Override
+  public void put(PathMetadata meta) throws IOException {
+
+    Preconditions.checkNotNull(meta);
+    FileStatus status = meta.getFileStatus();
+    Path path = standardize(status.getPath());
+    synchronized (this) {
+
+      /* Add entry for this file. */
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("put {} -> {}", path, meta.prettyPrint());
+      }
+      fileHash.put(path, meta);
+
+      /* Directory case:
+       * We also make sure we have an entry in the dirHash, so subsequent
+       * listStatus(path) at least see the directory.
+       *
+       * If we had a boolean flag argument "isNew", we would know whether this
+       * is an existing directory the client discovered via getFileStatus(),
+       * or if it is a newly-created directory.  In the latter case, we would
+       * be able to mark the directory as authoritative (fully-cached),
+       * saving round trips to underlying store for subsequent listStatus()
+       */
+
+      if (status.isDirectory()) {
+        DirListingMetadata dir = dirHash.mruGet(path);
+        if (dir == null) {
+          dirHash.put(path, new DirListingMetadata(path, DirListingMetadata
+              .EMPTY_DIR, false));
+        }
+      }
+
+      /* Update cached parent dir. */
+      Path parentPath = path.getParent();
+      if (parentPath != null) {
+        DirListingMetadata parent = dirHash.mruGet(parentPath);
+        if (parent == null) {
+        /* Track this new file's listing in parent.  Parent is not
+         * authoritative, since there may be other items in it we don't know
+         * about. */
+          parent = new DirListingMetadata(parentPath,
+              DirListingMetadata.EMPTY_DIR, false);
+          dirHash.put(parentPath, parent);
+        }
+        parent.put(status);
+      }
+    }
+  }
+
+  @Override
+  public synchronized void put(DirListingMetadata meta) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("put dirMeta {}", meta.prettyPrint());
+    }
+    dirHash.put(standardize(meta.getPath()), meta);
+  }
+
+  public synchronized void put(Collection<PathMetadata> metas) throws
+      IOException {
+    for (PathMetadata meta : metas) {
+      put(meta);
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+
+  }
+
+  @Override
+  public void destroy() throws IOException {
+    if (dirHash != null) {
+      dirHash.clear();
+    }
+  }
+
+  @Override
+  public synchronized void prune(long modTime) throws IOException {
+    Iterator<Map.Entry<Path, PathMetadata>> files =
+        fileHash.entrySet().iterator();
+    while (files.hasNext()) {
+      Map.Entry<Path, PathMetadata> entry = files.next();
+      if (expired(entry.getValue().getFileStatus(), modTime)) {
+        files.remove();
+      }
+    }
+    Iterator<Map.Entry<Path, DirListingMetadata>> dirs =
+        dirHash.entrySet().iterator();
+    while (dirs.hasNext()) {
+      Map.Entry<Path, DirListingMetadata> entry = dirs.next();
+      Path path = entry.getKey();
+      DirListingMetadata metadata = entry.getValue();
+      Collection<PathMetadata> oldChildren = metadata.getListing();
+      Collection<PathMetadata> newChildren = new LinkedList<>();
+
+      for (PathMetadata child : oldChildren) {
+        FileStatus status = child.getFileStatus();
+        if (!expired(status, modTime)) {
+          newChildren.add(child);
+        }
+      }
+      if (newChildren.size() != oldChildren.size()) {
+        dirHash.put(path, new DirListingMetadata(path, newChildren, false));
+        if (!path.isRoot()) {
+          DirListingMetadata parent = dirHash.get(path.getParent());
+          if (parent != null) {
+            parent.setAuthoritative(false);
+          }
+        }
+      }
+    }
+  }
+
+  private boolean expired(FileStatus status, long expiry) {
+    // Note: S3 doesn't track modification time on directories, so for
+    // consistency with the DynamoDB implementation we ignore that here
+    return status.getModificationTime() < expiry && !status.isDirectory();
+  }
+
+  @VisibleForTesting
+  static <T> void deleteHashByAncestor(Path ancestor, Map<Path, T> hash,
+                                       boolean tombstone) {
+    for (Iterator<Map.Entry<Path, T>> it = hash.entrySet().iterator();
+         it.hasNext();) {
+      Map.Entry<Path, T> entry = it.next();
+      Path f = entry.getKey();
+      T meta = entry.getValue();
+      if (isAncestorOf(ancestor, f)) {
+        if (tombstone) {
+          if (meta instanceof PathMetadata) {
+            entry.setValue((T) PathMetadata.tombstone(f));
+          } else if (meta instanceof DirListingMetadata) {
+            it.remove();
+          } else {
+            throw new IllegalStateException("Unknown type in hash");
+          }
+        } else {
+          it.remove();
+        }
+      }
+    }
+  }
+
+  /**
+   * @return true iff 'ancestor' is ancestor dir in path 'f'.
+   * All paths here are absolute.  Dir does not count as its own ancestor.
+   */
+  private static boolean isAncestorOf(Path ancestor, Path f) {
+    String aStr = ancestor.toString();
+    if (!ancestor.isRoot()) {
+      aStr += "/";
+    }
+    String fStr = f.toString();
+    return (fStr.startsWith(aStr));
+  }
+
+  /**
+   * Update fileHash and dirHash to reflect deletion of file 'f'.  Call with
+   * lock held.
+   */
+  private void deleteHashEntries(Path path, boolean tombstone) {
+
+    // Remove target file/dir
+    LOG.debug("delete file entry for {}", path);
+    if (tombstone) {
+      fileHash.put(path, PathMetadata.tombstone(path));
+    } else {
+      fileHash.remove(path);
+    }
+
+    // Update this and parent dir listing, if any
+
+    /* If this path is a dir, remove its listing */
+    LOG.debug("removing listing of {}", path);
+
+    dirHash.remove(path);
+
+    /* Remove this path from parent's dir listing */
+    Path parent = path.getParent();
+    if (parent != null) {
+      DirListingMetadata dir = dirHash.get(parent);
+      if (dir != null) {
+        LOG.debug("removing parent's entry for {} ", path);
+        if (tombstone) {
+          dir.markDeleted(path);
+        } else {
+          dir.remove(path);
+        }
+      }
+    }
+  }
+
+  /**
+   * Return a "standardized" version of a path so we always have a consistent
+   * hash value.  Also asserts the path is absolute, and contains host
+   * component.
+   * @param p input Path
+   * @return standardized version of Path, suitable for hash key
+   */
+  private Path standardize(Path p) {
+    Preconditions.checkArgument(p.isAbsolute(), "Path must be absolute");
+    URI uri = p.toUri();
+    if (uriHost != null) {
+      Preconditions.checkArgument(!isEmpty(uri.getHost()));
+    }
+    return p;
+  }
+
+  private static boolean isEmpty(String s) {
+    return (s == null || s.isEmpty());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LruHashMap.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LruHashMap.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LruHashMap.java
new file mode 100644
index 0000000..e355095
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LruHashMap.java
@@ -0,0 +1,50 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one or more
+ *  contributor license agreements.  See the NOTICE file distributed with
+ *  this work for additional information regarding copyright ownership.
+ *  The ASF licenses this file to You under the Apache License, Version 2.0
+ *  (the "License"); you may not use this file except in compliance with
+ *  the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+package org.apache.hadoop.fs.s3a.s3guard;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/**
+ * LinkedHashMap that implements a maximum size and LRU eviction policy.
+ */
+public class LruHashMap<K, V> extends LinkedHashMap<K, V> {
+  private final int maxSize;
+  public LruHashMap(int initialCapacity, int maxSize) {
+    super(initialCapacity);
+    this.maxSize = maxSize;
+  }
+
+  @Override
+  protected boolean removeEldestEntry(Map.Entry<K, V> eldest) {
+    return size() > maxSize;
+  }
+
+  /**
+   * get() plus side-effect of making the element Most Recently Used.
+   * @param key lookup key
+   * @return value
+   */
+
+  public V mruGet(K key) {
+    V val = remove(key);
+    if (val != null) {
+      put(key, val);
+    }
+    return val;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..dd8077b
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java
@@ -0,0 +1,221 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.s3guard;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collection;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * {@code MetadataStore} defines the set of operations that any metadata store
+ * implementation must provide.  Note that all {@link Path} objects provided
+ * to methods must be absolute, not relative paths.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface MetadataStore extends Closeable {
+
+  /**
+   * Performs one-time initialization of the metadata store.
+   *
+   * @param fs {@code FileSystem} associated with the MetadataStore
+   * @throws IOException if there is an error
+   */
+  void initialize(FileSystem fs) throws IOException;
+
+  /**
+   * Performs one-time initialization of the metadata store via configuration.
+   * @see #initialize(FileSystem)
+   * @param conf Configuration.
+   * @throws IOException if there is an error
+   */
+  void initialize(Configuration conf) throws IOException;
+
+  /**
+   * Deletes exactly one path, leaving a tombstone to prevent lingering,
+   * inconsistent copies of it from being listed.
+   *
+   * @param path the path to delete
+   * @throws IOException if there is an error
+   */
+  void delete(Path path) throws IOException;
+
+  /**
+   * Removes the record of exactly one path.  Does not leave a tombstone (see
+   * {@link MetadataStore#delete(Path)}. It is currently intended for testing
+   * only, and a need to use it as part of normal FileSystem usage is not
+   * anticipated.
+   *
+   * @param path the path to delete
+   * @throws IOException if there is an error
+   */
+  @VisibleForTesting
+  void forgetMetadata(Path path) throws IOException;
+
+  /**
+   * Deletes the entire sub-tree rooted at the given path, leaving tombstones
+   * to prevent lingering, inconsistent copies of it from being listed.
+   *
+   * In addition to affecting future calls to {@link #get(Path)},
+   * implementations must also update any stored {@code DirListingMetadata}
+   * objects which track the parent of this file.
+   *
+   * @param path the root of the sub-tree to delete
+   * @throws IOException if there is an error
+   */
+  void deleteSubtree(Path path) throws IOException;
+
+  /**
+   * Gets metadata for a path.
+   *
+   * @param path the path to get
+   * @return metadata for {@code path}, {@code null} if not found
+   * @throws IOException if there is an error
+   */
+  PathMetadata get(Path path) throws IOException;
+
+  /**
+   * Gets metadata for a path.  Alternate method that includes a hint
+   * whether or not the MetadataStore should do work to compute the value for
+   * {@link PathMetadata#isEmptyDirectory()}.  Since determining emptiness
+   * may be an expensive operation, this can save wasted work.
+   *
+   * @param path the path to get
+   * @param wantEmptyDirectoryFlag Set to true to give a hint to the
+   *   MetadataStore that it should try to compute the empty directory flag.
+   * @return metadata for {@code path}, {@code null} if not found
+   * @throws IOException if there is an error
+   */
+  PathMetadata get(Path path, boolean wantEmptyDirectoryFlag)
+      throws IOException;
+
+  /**
+   * Lists metadata for all direct children of a path.
+   *
+   * @param path the path to list
+   * @return metadata for all direct children of {@code path} which are being
+   *     tracked by the MetadataStore, or {@code null} if the path was not found
+   *     in the MetadataStore.
+   * @throws IOException if there is an error
+   */
+  DirListingMetadata listChildren(Path path) 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.
+   *
+   * This operation is not atomic, unless specific implementations claim
+   * otherwise.
+   *
+   * On the need to provide an enumeration of directory trees instead of just
+   * source and destination paths:
+   * Since a MetadataStore does not have to track all metadata for the
+   * underlying storage system, and a new MetadataStore may be created on an
+   * existing underlying filesystem, this move() may be the first time the
+   * MetadataStore sees the affected paths.  Therefore, simply providing src
+   * and destination paths may not be enough to record the deletions (under
+   * src path) and creations (at destination) that are happening during the
+   * rename().
+   *
+   * @param pathsToDelete Collection of all paths that were removed from the
+   *                      source directory tree of the move.
+   * @param pathsToCreate Collection of all PathMetadata for the new paths
+   *                      that were created at the destination of the rename
+   *                      ().
+   * @throws IOException if there is an error
+   */
+  void move(Collection<Path> pathsToDelete,
+      Collection<PathMetadata> pathsToCreate) throws IOException;
+
+  /**
+   * Saves metadata for exactly one path.
+   *
+   * 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
+   * @throws IOException if there is an error
+   */
+  void put(PathMetadata meta) throws IOException;
+
+  /**
+   * Saves metadata for any number of paths.
+   *
+   * Semantics are otherwise the same as single-path puts.
+   *
+   * @param metas the metadata to save
+   * @throws IOException if there is an error
+   */
+  void put(Collection<PathMetadata> metas) throws IOException;
+
+  /**
+   * Save directory listing metadata. Callers may save a partial directory
+   * listing for a given path, or may store a complete and authoritative copy
+   * of the directory listing.  {@code MetadataStore} implementations may
+   * subsequently keep track of all modifications to the directory contents at
+   * this path, and return authoritative results from subsequent calls to
+   * {@link #listChildren(Path)}. See {@link DirListingMetadata}.
+   *
+   * Any authoritative results returned are only authoritative for the scope
+   * of the {@code MetadataStore}:  A per-process {@code MetadataStore}, for
+   * example, would only show results visible to that process, potentially
+   * missing metadata updates (create, delete) made to the same path by
+   * another process.
+   *
+   * @param meta Directory listing metadata.
+   * @throws IOException if there is an error
+   */
+  void put(DirListingMetadata meta) throws IOException;
+
+  /**
+   * Destroy all resources associated with the metadata store.
+   *
+   * The destroyed resources can be DynamoDB tables, MySQL databases/tables, or
+   * HDFS directories. Any operations after calling this method may possibly
+   * fail.
+   *
+   * This operation is idempotent.
+   *
+   * @throws IOException if there is an error
+   */
+  void destroy() throws IOException;
+
+  /**
+   * Clear any metadata older than a specified time from the repository.
+   * Implementations MUST clear file metadata, and MAY clear directory metadata
+   * (s3a itself does not track modification time for directories).
+   * Implementations may also choose to throw UnsupportedOperationException
+   * istead. Note that modification times should be in UTC, as returned by
+   * System.currentTimeMillis at the time of modification.
+   *
+   * @param modTime Oldest modification time to allow
+   * @throws IOException if there is an error
+   * @throws UnsupportedOperationException if not implemented
+   */
+  void prune(long modTime) throws IOException, UnsupportedOperationException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreListFilesIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreListFilesIterator.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreListFilesIterator.java
new file mode 100644
index 0000000..378d109
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreListFilesIterator.java
@@ -0,0 +1,169 @@
+/*
+ * 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.Iterator;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+
+/**
+ * {@code MetadataStoreListFilesIterator} is a {@link RemoteIterator} that
+ * is similar to {@code DescendantsIterator} but does not return directories
+ * that have (or may have) children, and will also provide access to the set of
+ * tombstones to allow recently deleted S3 objects to be filtered out from a
+ * corresponding request.  In other words, it returns tombstones and the same
+ * set of objects that should exist in S3: empty directories, and files, and not
+ * other directories whose existence is inferred therefrom.
+ *
+ * For example, assume the consistent store contains metadata representing this
+ * file system structure:
+ *
+ * <pre>
+ * /dir1
+ * |-- dir2
+ * |   |-- file1
+ * |   `-- file2
+ * `-- dir3
+ *     |-- dir4
+ *     |   `-- file3
+ *     |-- dir5
+ *     |   `-- file4
+ *     `-- dir6
+ * </pre>
+ *
+ * Consider this code sample:
+ * <pre>
+ * final PathMetadata dir1 = get(new Path("/dir1"));
+ * for (MetadataStoreListFilesIterator files =
+ *     new MetadataStoreListFilesIterator(dir1); files.hasNext(); ) {
+ *   final FileStatus status = files.next().getFileStatus();
+ *   System.out.printf("%s %s%n", status.isDirectory() ? 'D' : 'F',
+ *       status.getPath());
+ * }
+ * </pre>
+ *
+ * The output is:
+ * <pre>
+ * F /dir1/dir2/file1
+ * F /dir1/dir2/file2
+ * F /dir1/dir3/dir4/file3
+ * F /dir1/dir3/dir5/file4
+ * D /dir1/dir3/dir6
+ * </pre>
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class MetadataStoreListFilesIterator implements
+    RemoteIterator<FileStatus> {
+  public static final Logger LOG = LoggerFactory.getLogger(
+      MetadataStoreListFilesIterator.class);
+
+  private final boolean allowAuthoritative;
+  private final MetadataStore metadataStore;
+  private final Set<Path> tombstones = new HashSet<>();
+  private Iterator<FileStatus> leafNodesIterator = null;
+
+  public MetadataStoreListFilesIterator(MetadataStore ms, PathMetadata meta,
+      boolean allowAuthoritative) throws IOException {
+    Preconditions.checkNotNull(ms);
+    this.metadataStore = ms;
+    this.allowAuthoritative = allowAuthoritative;
+    prefetch(meta);
+  }
+
+  private void prefetch(PathMetadata meta) throws IOException {
+    final Queue<PathMetadata> queue = new LinkedList<>();
+    final Collection<FileStatus> leafNodes = new ArrayList<>();
+
+    if (meta != null) {
+      final Path path = meta.getFileStatus().getPath();
+      if (path.isRoot()) {
+        DirListingMetadata rootListing = metadataStore.listChildren(path);
+        if (rootListing != null) {
+          tombstones.addAll(rootListing.listTombstones());
+          queue.addAll(rootListing.withoutTombstones().getListing());
+        }
+      } else {
+        queue.add(meta);
+      }
+    }
+
+    while(!queue.isEmpty()) {
+      PathMetadata nextMetadata = queue.poll();
+      FileStatus nextStatus = nextMetadata.getFileStatus();
+      if (nextStatus.isFile()) {
+        // All files are leaf nodes by definition
+        leafNodes.add(nextStatus);
+        continue;
+      }
+      if (nextStatus.isDirectory()) {
+        final Path path = nextStatus.getPath();
+        DirListingMetadata children = metadataStore.listChildren(path);
+        if (children != null) {
+          tombstones.addAll(children.listTombstones());
+          Collection<PathMetadata> liveChildren =
+              children.withoutTombstones().getListing();
+          if (!liveChildren.isEmpty()) {
+            // If it's a directory, has children, not all deleted, then we
+            // add the children to the queue and move on to the next node
+            queue.addAll(liveChildren);
+            continue;
+          } else if (allowAuthoritative && children.isAuthoritative()) {
+            leafNodes.add(nextStatus);
+          }
+        }
+      }
+      // Directories that *might* be empty are ignored for now, since we
+      // cannot confirm that they are empty without incurring other costs.
+      // Users of this class can still discover empty directories via S3's
+      // fake directories, subject to the same consistency semantics as before.
+      // The only other possibility is a symlink, which is unsupported on S3A.
+    }
+    leafNodesIterator = leafNodes.iterator();
+  }
+
+  @Override
+  public boolean hasNext() {
+    return leafNodesIterator.hasNext();
+  }
+
+  @Override
+  public FileStatus next() {
+    return leafNodesIterator.next();
+  }
+
+  public Set<Path> listTombstones() {
+    return tombstones;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..08ae89e
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java
@@ -0,0 +1,104 @@
+/*
+ * 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.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import java.io.IOException;
+import java.util.Collection;
+
+/**
+ * A no-op implementation of MetadataStore.  Clients that use this
+ * implementation should behave the same as they would without any
+ * MetadataStore.
+ */
+public class NullMetadataStore implements MetadataStore {
+
+  @Override
+  public void initialize(FileSystem fs) throws IOException {
+  }
+
+  @Override
+  public void initialize(Configuration conf) throws IOException {
+  }
+
+  @Override
+  public void close() throws IOException {
+  }
+
+  @Override
+  public void delete(Path path) throws IOException {
+  }
+
+  @Override
+  public void forgetMetadata(Path path) throws IOException {
+  }
+
+  @Override
+  public void deleteSubtree(Path path) throws IOException {
+  }
+
+  @Override
+  public PathMetadata get(Path path) throws IOException {
+    return null;
+  }
+
+  @Override
+  public PathMetadata get(Path path, boolean wantEmptyDirectoryFlag)
+      throws IOException {
+    return null;
+  }
+
+  @Override
+  public DirListingMetadata listChildren(Path path) throws IOException {
+    return null;
+  }
+
+  @Override
+  public void move(Collection<Path> pathsToDelete,
+      Collection<PathMetadata> pathsToCreate) throws IOException {
+  }
+
+  @Override
+  public void put(PathMetadata meta) throws IOException {
+  }
+
+  @Override
+  public void put(Collection<PathMetadata> meta) throws IOException {
+  }
+
+  @Override
+  public void put(DirListingMetadata meta) throws IOException {
+  }
+
+  @Override
+  public void destroy() throws IOException {
+  }
+
+  @Override
+  public void prune(long modTime) {
+  }
+
+  @Override
+  public String toString() {
+    return "NullMetadataStore";
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadata.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadata.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadata.java
new file mode 100644
index 0000000..2a0219e
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadata.java
@@ -0,0 +1,143 @@
+/*
+ * 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 com.google.common.base.Preconditions;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.Tristate;
+
+/**
+ * {@code PathMetadata} models path metadata stored in the
+ * {@link MetadataStore}.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class PathMetadata {
+
+  private final FileStatus fileStatus;
+  private Tristate isEmptyDirectory;
+  private boolean isDeleted;
+
+  /**
+   * Create a tombstone from the current time.
+   * @param path path to tombstone
+   * @return the entry.
+   */
+  public static PathMetadata tombstone(Path path) {
+    long now = System.currentTimeMillis();
+    FileStatus status = new FileStatus(0, false, 0, 0, now, path);
+    return new PathMetadata(status, Tristate.UNKNOWN, true);
+  }
+
+  /**
+   * Creates a new {@code PathMetadata} containing given {@code FileStatus}.
+   * @param fileStatus file status containing an absolute path.
+   */
+  public PathMetadata(FileStatus fileStatus) {
+    this(fileStatus, Tristate.UNKNOWN);
+  }
+
+  public PathMetadata(FileStatus fileStatus, Tristate isEmptyDir) {
+    this(fileStatus, isEmptyDir, false);
+  }
+
+  public PathMetadata(FileStatus fileStatus, Tristate isEmptyDir, boolean
+      isDeleted) {
+    Preconditions.checkNotNull(fileStatus, "fileStatus must be non-null");
+    Preconditions.checkNotNull(fileStatus.getPath(), "fileStatus path must be" +
+        " non-null");
+    Preconditions.checkArgument(fileStatus.getPath().isAbsolute(), "path must" +
+        " be absolute");
+    this.fileStatus = fileStatus;
+    this.isEmptyDirectory = isEmptyDir;
+    this.isDeleted = isDeleted;
+  }
+
+  /**
+   * @return {@code FileStatus} contained in this {@code PathMetadata}.
+   */
+  public final FileStatus getFileStatus() {
+    return fileStatus;
+  }
+
+  /**
+   * Query if a directory is empty.
+   * @return Tristate.TRUE if this is known to be an empty directory,
+   * Tristate.FALSE if known to not be empty, and Tristate.UNKNOWN if the
+   * MetadataStore does have enough information to determine either way.
+   */
+  public Tristate isEmptyDirectory() {
+    return isEmptyDirectory;
+  }
+
+  void setIsEmptyDirectory(Tristate isEmptyDirectory) {
+    this.isEmptyDirectory = isEmptyDirectory;
+  }
+
+  public boolean isDeleted() {
+    return isDeleted;
+  }
+
+  void setIsDeleted(boolean isDeleted) {
+    this.isDeleted = isDeleted;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (!(o instanceof PathMetadata)) {
+      return false;
+    }
+    return this.fileStatus.equals(((PathMetadata)o).fileStatus);
+  }
+
+  @Override
+  public int hashCode() {
+    return fileStatus.hashCode();
+  }
+
+  @Override
+  public String toString() {
+    return "PathMetadata{" +
+        "fileStatus=" + fileStatus +
+        "; isEmptyDirectory=" + isEmptyDirectory +
+        "; isDeleted=" + isDeleted +
+        '}';
+  }
+
+  /**
+   * Log contents to supplied StringBuilder in a pretty fashion.
+   * @param sb target StringBuilder
+   */
+  public void prettyPrint(StringBuilder sb) {
+    sb.append(String.format("%-5s %-20s %-7d %-8s %-6s",
+        fileStatus.isDirectory() ? "dir" : "file",
+        fileStatus.getPath().toString(), fileStatus.getLen(),
+        isEmptyDirectory.name(), isDeleted));
+    sb.append(fileStatus);
+  }
+
+  public String prettyPrint() {
+    StringBuilder sb = new StringBuilder();
+    prettyPrint(sb);
+    return sb.toString();
+  }
+}


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


[5/8] hadoop git commit: HADOOP-13345 S3Guard: Improved Consistency for S3A. Contributed by: Chris Nauroth, Aaron Fabbri, Mingliang Liu, Lei (Eddy) Xu, Sean Mackrory, Steve Loughran and others.

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadataDynamoDBTranslation.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..8515bfb
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadataDynamoDBTranslation.java
@@ -0,0 +1,304 @@
+/*
+ * 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.net.URI;
+import java.util.Arrays;
+import java.util.Collection;
+
+import com.amazonaws.services.dynamodbv2.document.Item;
+import com.amazonaws.services.dynamodbv2.document.KeyAttribute;
+import com.amazonaws.services.dynamodbv2.document.PrimaryKey;
+import com.amazonaws.services.dynamodbv2.model.AttributeDefinition;
+import com.amazonaws.services.dynamodbv2.model.KeySchemaElement;
+import com.amazonaws.services.dynamodbv2.model.KeyType;
+import com.amazonaws.services.dynamodbv2.model.ScalarAttributeType;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.Constants;
+import org.apache.hadoop.fs.s3a.Tristate;
+
+/**
+ * Defines methods for translating between domain model objects and their
+ * representations in the DynamoDB schema.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+final class PathMetadataDynamoDBTranslation {
+
+  /** The HASH key name of each item. */
+  @VisibleForTesting
+  static final String PARENT = "parent";
+  /** The RANGE key name of each item. */
+  @VisibleForTesting
+  static final String CHILD = "child";
+  @VisibleForTesting
+  static final String IS_DIR = "is_dir";
+  @VisibleForTesting
+  static final String MOD_TIME = "mod_time";
+  @VisibleForTesting
+  static final String FILE_LENGTH = "file_length";
+  @VisibleForTesting
+  static final String BLOCK_SIZE = "block_size";
+  static final String IS_DELETED = "is_deleted";
+
+  /** Table version field {@value} in version marker item. */
+  @VisibleForTesting
+  static final String TABLE_VERSION = "table_version";
+
+  /** Table creation timestampfield {@value} in version marker item. */
+  @VisibleForTesting
+  static final String TABLE_CREATED = "table_created";
+
+  /** The version marker field is invalid. */
+  static final String E_NOT_VERSION_MARKER = "Not a version marker: ";
+
+  /**
+   * Returns the key schema for the DynamoDB table.
+   *
+   * @return DynamoDB key schema
+   */
+  static Collection<KeySchemaElement> keySchema() {
+    return Arrays.asList(
+        new KeySchemaElement(PARENT, KeyType.HASH),
+        new KeySchemaElement(CHILD, KeyType.RANGE));
+  }
+
+  /**
+   * Returns the attribute definitions for the DynamoDB table.
+   *
+   * @return DynamoDB attribute definitions
+   */
+  static Collection<AttributeDefinition> attributeDefinitions() {
+    return Arrays.asList(
+        new AttributeDefinition(PARENT, ScalarAttributeType.S),
+        new AttributeDefinition(CHILD, ScalarAttributeType.S));
+  }
+
+  /**
+   * Converts a DynamoDB item to a {@link PathMetadata}.
+   *
+   * @param item DynamoDB item to convert
+   * @return {@code item} converted to a {@link PathMetadata}
+   */
+  static PathMetadata itemToPathMetadata(Item item, String username)
+      throws IOException {
+    if (item == null) {
+      return null;
+    }
+
+    String parentStr = item.getString(PARENT);
+    Preconditions.checkNotNull(parentStr, "No parent entry in item %s", item);
+    String childStr = item.getString(CHILD);
+    Preconditions.checkNotNull(childStr, "No child entry in item %s", item);
+
+    // Skip table version markers, which are only non-absolute paths stored.
+    Path rawPath = new Path(parentStr, childStr);
+    if (!rawPath.isAbsoluteAndSchemeAuthorityNull()) {
+      return null;
+    }
+
+    Path parent = new Path(Constants.FS_S3A + ":/" + parentStr + "/");
+    Path path = new Path(parent, childStr);
+
+    boolean isDir = item.hasAttribute(IS_DIR) && item.getBoolean(IS_DIR);
+    final FileStatus fileStatus;
+    if (isDir) {
+      fileStatus = DynamoDBMetadataStore.makeDirStatus(path, username);
+    } else {
+      long len = item.hasAttribute(FILE_LENGTH) ? item.getLong(FILE_LENGTH) : 0;
+      long modTime = item.hasAttribute(MOD_TIME) ? item.getLong(MOD_TIME) : 0;
+      long block = item.hasAttribute(BLOCK_SIZE) ? item.getLong(BLOCK_SIZE) : 0;
+      fileStatus = new FileStatus(len, false, 1, block, modTime, 0, null,
+          username, username, path);
+    }
+    boolean isDeleted =
+        item.hasAttribute(IS_DELETED) && item.getBoolean(IS_DELETED);
+
+    return new PathMetadata(fileStatus, Tristate.UNKNOWN, isDeleted);
+  }
+
+  /**
+   * Converts a {@link PathMetadata} to a DynamoDB item.
+   *
+   * @param meta {@link PathMetadata} to convert
+   * @return {@code meta} converted to DynamoDB item
+   */
+  static Item pathMetadataToItem(PathMetadata meta) {
+    Preconditions.checkNotNull(meta);
+    final FileStatus status = meta.getFileStatus();
+    final Item item = new Item().withPrimaryKey(pathToKey(status.getPath()));
+    if (status.isDirectory()) {
+      item.withBoolean(IS_DIR, true);
+    } else {
+      item.withLong(FILE_LENGTH, status.getLen())
+          .withLong(MOD_TIME, status.getModificationTime())
+          .withLong(BLOCK_SIZE, status.getBlockSize());
+    }
+    item.withBoolean(IS_DELETED, meta.isDeleted());
+    return item;
+  }
+
+  /**
+   * The version marker has a primary key whose PARENT is {@code name};
+   * this MUST NOT be a value which represents an absolute path.
+   * @param name name of the version marker
+   * @param version version number
+   * @param timestamp creation timestamp
+   * @return an item representing a version marker.
+   */
+  static Item createVersionMarker(String name, int version, long timestamp) {
+    return new Item().withPrimaryKey(createVersionMarkerPrimaryKey(name))
+        .withInt(TABLE_VERSION, version)
+        .withLong(TABLE_CREATED, timestamp);
+  }
+
+  /**
+   * Create the primary key of the version marker.
+   * @param name key name
+   * @return the key to use when registering or resolving version markers
+   */
+  static PrimaryKey createVersionMarkerPrimaryKey(String name) {
+    return new PrimaryKey(PARENT, name, CHILD, name);
+  }
+
+  /**
+   * Extract the version from a version marker item.
+   * @param marker version marker item
+   * @return the extracted version field
+   * @throws IOException if the item is not a version marker
+   */
+  static int extractVersionFromMarker(Item marker) throws IOException {
+    if (marker.hasAttribute(TABLE_VERSION)) {
+      return marker.getInt(TABLE_VERSION);
+    } else {
+      throw new IOException(E_NOT_VERSION_MARKER + marker);
+    }
+  }
+
+  /**
+   * Extract the creation time, if present.
+   * @param marker version marker item
+   * @return the creation time, or null
+   * @throws IOException if the item is not a version marker
+   */
+  static Long extractCreationTimeFromMarker(Item marker) throws IOException {
+    if (marker.hasAttribute(TABLE_CREATED)) {
+      return marker.getLong(TABLE_CREATED);
+    } else {
+      return null;
+    }
+  }
+
+  /**
+   * Converts a collection {@link PathMetadata} to a collection DynamoDB items.
+   *
+   * @see #pathMetadataToItem(PathMetadata)
+   */
+  static Item[] pathMetadataToItem(Collection<PathMetadata> metas) {
+    if (metas == null) {
+      return null;
+    }
+
+    final Item[] items = new Item[metas.size()];
+    int i = 0;
+    for (PathMetadata meta : metas) {
+      items[i++] = pathMetadataToItem(meta);
+    }
+    return items;
+  }
+
+  /**
+   * Converts a {@link Path} to a DynamoDB equality condition on that path as
+   * parent, suitable for querying all direct children of the path.
+   *
+   * @param path the path; can not be null
+   * @return DynamoDB equality condition on {@code path} as parent
+   */
+  static KeyAttribute pathToParentKeyAttribute(Path path) {
+    return new KeyAttribute(PARENT, pathToParentKey(path));
+  }
+
+  /**
+   * e.g. {@code pathToParentKey(s3a://bucket/path/a) -> /bucket/path/a}
+   * @param path path to convert
+   * @return string for parent key
+   */
+  static String pathToParentKey(Path path) {
+    Preconditions.checkNotNull(path);
+    Preconditions.checkArgument(path.isUriPathAbsolute(), "Path not absolute");
+    URI uri = path.toUri();
+    String bucket = uri.getHost();
+    Preconditions.checkArgument(!StringUtils.isEmpty(bucket),
+        "Path missing bucket");
+    String pKey = "/" + bucket + uri.getPath();
+
+    // Strip trailing slash
+    if (pKey.endsWith("/")) {
+      pKey = pKey.substring(0, pKey.length() - 1);
+    }
+    return pKey;
+  }
+
+  /**
+   * Converts a {@link Path} to a DynamoDB key, suitable for getting the item
+   * matching the path.
+   *
+   * @param path the path; can not be null
+   * @return DynamoDB key for item matching {@code path}
+   */
+  static PrimaryKey pathToKey(Path path) {
+    Preconditions.checkArgument(!path.isRoot(),
+        "Root path is not mapped to any PrimaryKey");
+    return new PrimaryKey(PARENT, pathToParentKey(path.getParent()), CHILD,
+        path.getName());
+  }
+
+  /**
+   * Converts a collection of {@link Path} to a collection of DynamoDB keys.
+   *
+   * @see #pathToKey(Path)
+   */
+  static PrimaryKey[] pathToKey(Collection<Path> paths) {
+    if (paths == null) {
+      return null;
+    }
+
+    final PrimaryKey[] keys = new PrimaryKey[paths.size()];
+    int i = 0;
+    for (Path p : paths) {
+      keys[i++] = pathToKey(p);
+    }
+    return keys;
+  }
+
+  /**
+   * There is no need to instantiate this class.
+   */
+  private PathMetadataDynamoDBTranslation() {
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..7e4aec1
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java
@@ -0,0 +1,463 @@
+/*
+ * 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.net.URI;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+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.FileStatus;
+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.S3AInstrumentation;
+import org.apache.hadoop.fs.s3a.Tristate;
+import org.apache.hadoop.util.ReflectionUtils;
+
+import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL;
+import static org.apache.hadoop.fs.s3a.Statistic.S3GUARD_METADATASTORE_PUT_PATH_LATENCY;
+import static org.apache.hadoop.fs.s3a.Statistic.S3GUARD_METADATASTORE_PUT_PATH_REQUEST;
+import static org.apache.hadoop.fs.s3a.S3AUtils.createUploadFileStatus;
+
+/**
+ * Logic for integrating MetadataStore with S3A.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public final class S3Guard {
+  private static final Logger LOG = LoggerFactory.getLogger(S3Guard.class);
+
+  @InterfaceAudience.Private
+  @InterfaceStability.Unstable
+  @VisibleForTesting
+  public static final String S3GUARD_DDB_CLIENT_FACTORY_IMPL =
+      "fs.s3a.s3guard.ddb.client.factory.impl";
+
+  static final Class<? extends DynamoDBClientFactory>
+      S3GUARD_DDB_CLIENT_FACTORY_IMPL_DEFAULT =
+      DynamoDBClientFactory.DefaultDynamoDBClientFactory.class;
+  private static final FileStatus[] EMPTY_LISTING = new FileStatus[0];
+
+  // Utility class.  All static functions.
+  private S3Guard() { }
+
+  /* Utility functions. */
+
+  /**
+   * Create a new instance of the configured MetadataStore.
+   * The returned MetadataStore will have been initialized via
+   * {@link MetadataStore#initialize(FileSystem)} by this function before
+   * returning it.  Callers must clean up by calling
+   * {@link MetadataStore#close()} when done using the MetadataStore.
+   *
+   * @param fs  FileSystem whose Configuration specifies which
+   *            implementation to use.
+   * @return Reference to new MetadataStore.
+   * @throws IOException if the metadata store cannot be instantiated
+   */
+  public static MetadataStore getMetadataStore(FileSystem fs)
+      throws IOException {
+    Preconditions.checkNotNull(fs);
+    Configuration conf = fs.getConf();
+    Preconditions.checkNotNull(conf);
+    MetadataStore msInstance;
+    try {
+      Class<? extends MetadataStore> msClass = getMetadataStoreClass(conf);
+      msInstance = ReflectionUtils.newInstance(msClass, conf);
+      LOG.debug("Using {} metadata store for {} filesystem",
+          msClass.getSimpleName(), fs.getScheme());
+      msInstance.initialize(fs);
+      return msInstance;
+    } catch (RuntimeException | IOException e) {
+      String message = "Failed to instantiate metadata store " +
+          conf.get(S3_METADATA_STORE_IMPL)
+          + " defined in " + S3_METADATA_STORE_IMPL
+          + ": " + e;
+      LOG.error(message, e);
+      if (e instanceof IOException) {
+        throw e;
+      } else {
+        throw new IOException(message, e);
+      }
+    }
+  }
+
+  private static Class<? extends MetadataStore> getMetadataStoreClass(
+      Configuration conf) {
+    if (conf == null) {
+      return NullMetadataStore.class;
+    }
+
+    return conf.getClass(S3_METADATA_STORE_IMPL, NullMetadataStore.class,
+            MetadataStore.class);
+  }
+
+
+  /**
+   * 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
+   * @return The same status as passed in
+   * @throws IOException if metadata store update failed
+   */
+  public static S3AFileStatus putAndReturn(MetadataStore ms,
+      S3AFileStatus status,
+      S3AInstrumentation instrumentation) throws IOException {
+    long startTimeNano = System.nanoTime();
+    ms.put(new PathMetadata(status));
+    instrumentation.addValueToQuantiles(S3GUARD_METADATASTORE_PUT_PATH_LATENCY,
+        (System.nanoTime() - startTimeNano));
+    instrumentation.incrementCounter(S3GUARD_METADATASTORE_PUT_PATH_REQUEST, 1);
+    return status;
+  }
+
+  /**
+   * 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.
+   * @param dirMeta directory listing -may be null
+   * @return a possibly-empty array of file status entries
+   */
+  public static FileStatus[] dirMetaToStatuses(DirListingMetadata dirMeta)  {
+    if (dirMeta == null) {
+      return EMPTY_LISTING;
+    }
+
+    Collection<PathMetadata> listing = dirMeta.getListing();
+    List<FileStatus> statuses = new ArrayList<>();
+
+    for (PathMetadata pm : listing) {
+      if (!pm.isDeleted()) {
+        statuses.add(pm.getFileStatus());
+      }
+    }
+
+    return statuses.toArray(new FileStatus[0]);
+  }
+
+  /**
+   * Given directory listing metadata from both the backing store and the
+   * MetadataStore, merge the two sources of truth to create a consistent
+   * view of the current directory contents, which can be returned to clients.
+   *
+   * Also update the MetadataStore to reflect the resulting directory listing.
+   *
+   * @param ms MetadataStore to use.
+   * @param path path to directory
+   * @param backingStatuses Directory listing from the backing store.
+   * @param dirMeta  Directory listing from MetadataStore.  May be null.
+   * @param isAuthoritative State of authoritative mode
+   * @return Final result of directory listing.
+   * @throws IOException if metadata store update failed
+   */
+  public static FileStatus[] dirListingUnion(MetadataStore ms, Path path,
+      List<FileStatus> backingStatuses, DirListingMetadata dirMeta,
+      boolean isAuthoritative) throws IOException {
+
+    // Fast-path for NullMetadataStore
+    if (isNullMetadataStore(ms)) {
+      return backingStatuses.toArray(new FileStatus[backingStatuses.size()]);
+    }
+
+    assertQualified(path);
+
+    if (dirMeta == null) {
+      // The metadataStore had zero state for this directory
+      dirMeta = new DirListingMetadata(path, DirListingMetadata.EMPTY_DIR,
+          false);
+    }
+
+    Set<Path> deleted = dirMeta.listTombstones();
+
+    // Since we treat the MetadataStore as a "fresher" or "consistent" view
+    // of metadata, we always use its metadata first.
+
+    // Since the authoritative case is already handled outside this function,
+    // we will basically start with the set of directory entries in the
+    // DirListingMetadata, and add any that only exist in the backingStatuses.
+
+    boolean changed = false;
+    for (FileStatus s : backingStatuses) {
+      if (deleted.contains(s.getPath())) {
+        continue;
+      }
+
+      // Minor race condition here.  Multiple threads could add to this
+      // mutable DirListingMetadata.  Since it is backed by a
+      // ConcurrentHashMap, the last put() wins.
+      // More concerning is two threads racing on listStatus() and delete().
+      // Any FileSystem has similar race conditions, but we could persist
+      // a stale entry longer.  We could expose an atomic
+      // DirListingMetadata#putIfNotPresent()
+      boolean updated = dirMeta.put(s);
+      changed = changed || updated;
+    }
+
+    if (changed && isAuthoritative) {
+      dirMeta.setAuthoritative(true); // This is the full directory contents
+      ms.put(dirMeta);
+    }
+
+    return dirMetaToStatuses(dirMeta);
+  }
+
+  /**
+   * Although NullMetadataStore does nothing, callers may wish to avoid work
+   * (fast path) when the NullMetadataStore is in use.
+   * @param ms The MetadataStore to test
+   * @return true iff the MetadataStore is the null, or no-op, implementation.
+   */
+  public static boolean isNullMetadataStore(MetadataStore ms) {
+    return (ms instanceof NullMetadataStore);
+  }
+
+  /**
+   * Update MetadataStore to reflect creation of the given  directories.
+   *
+   * If an IOException is raised while trying to update the entry, this
+   * operation catches the exception and returns.
+   * @param ms    MetadataStore to update.
+   * @param dirs  null, or an ordered list of directories from leaf to root.
+   *              E.g. if /a/ exists, and  mkdirs(/a/b/c/d) is called, this
+   *              list will contain [/a/b/c/d, /a/b/c, /a/b].   /a/b/c/d is
+   *              an empty, dir, and the other dirs only contain their child
+   *              dir.
+   * @param owner Hadoop user name.
+   * @param authoritative Whether to mark new directories as authoritative.
+   */
+  public static void makeDirsOrdered(MetadataStore ms, List<Path> dirs,
+      String owner, boolean authoritative) {
+    if (dirs == null) {
+      return;
+    }
+
+    /* We discussed atomicity of this implementation.
+     * The concern is that multiple clients could race to write different
+     * cached directories to the MetadataStore.  Two solutions are proposed:
+     * 1. Move mkdirs() into MetadataStore interface and let implementations
+     *    ensure they are atomic.
+     * 2. Specify that the semantics of MetadataStore#putListStatus() is
+     *    always additive,  That is, if MetadataStore has listStatus() state
+     *    for /a/b that contains [/a/b/file0, /a/b/file1], and we then call
+     *    putListStatus(/a/b -> [/a/b/file2, /a/b/file3], isAuthoritative=true),
+     *    then we will end up with final state of
+     *    [/a/b/file0, /a/b/file1, /a/b/file2, /a/b/file3], isAuthoritative =
+     *    true
+     */
+    FileStatus prevStatus = null;
+
+    // Use new batched put to reduce round trips.
+    List<PathMetadata> pathMetas = new ArrayList<>(dirs.size());
+
+    try {
+      // Iterate from leaf to root
+      for (int i = 0; i < dirs.size(); i++) {
+        boolean isLeaf = (prevStatus == null);
+        Path f = dirs.get(i);
+        assertQualified(f);
+        FileStatus status =
+            createUploadFileStatus(f, true, 0, 0, owner);
+
+        // We only need to put a DirListingMetadata if we are setting
+        // authoritative bit
+        DirListingMetadata dirMeta = null;
+        if (authoritative) {
+          Collection<PathMetadata> children;
+          if (isLeaf) {
+            children = DirListingMetadata.EMPTY_DIR;
+          } else {
+            children = new ArrayList<>(1);
+            children.add(new PathMetadata(prevStatus));
+          }
+          dirMeta = new DirListingMetadata(f, children, authoritative);
+          ms.put(dirMeta);
+        }
+
+        pathMetas.add(new PathMetadata(status));
+        prevStatus = status;
+      }
+
+      // Batched put
+      ms.put(pathMetas);
+    } catch (IOException ioe) {
+      LOG.error("MetadataStore#put() failure:", ioe);
+    }
+  }
+
+  /**
+   * Helper function that records the move of directory paths, adding
+   * resulting metadata to the supplied lists.
+   * Does not store in MetadataStore.
+   * @param ms  MetadataStore, used to make this a no-op, when it is
+   *            NullMetadataStore.
+   * @param srcPaths stores the source path here
+   * @param dstMetas stores destination metadata here
+   * @param srcPath  source path to store
+   * @param dstPath  destination path to store
+   * @param owner file owner to use in created records
+   */
+  public static void addMoveDir(MetadataStore ms, Collection<Path> srcPaths,
+      Collection<PathMetadata> dstMetas, Path srcPath, Path dstPath,
+      String owner) {
+    if (isNullMetadataStore(ms)) {
+      return;
+    }
+    assertQualified(srcPath, dstPath);
+
+    FileStatus dstStatus = createUploadFileStatus(dstPath, true, 0, 0, owner);
+    addMoveStatus(srcPaths, dstMetas, srcPath, dstStatus);
+  }
+
+  /**
+   * Like {@link #addMoveDir(MetadataStore, Collection, Collection, Path,
+   * Path, String)} (), but for files.
+   * @param ms  MetadataStore, used to make this a no-op, when it is
+   *            NullMetadataStore.
+   * @param srcPaths stores the source path here
+   * @param dstMetas stores destination metadata here
+   * @param srcPath  source path to store
+   * @param dstPath  destination path to store
+   * @param size length of file moved
+   * @param blockSize  blocksize to associate with destination file
+   * @param owner file owner to use in created records
+   */
+  public static void addMoveFile(MetadataStore ms, Collection<Path> srcPaths,
+      Collection<PathMetadata> dstMetas, Path srcPath, Path dstPath,
+      long size, long blockSize, String owner) {
+    if (isNullMetadataStore(ms)) {
+      return;
+    }
+    assertQualified(srcPath, dstPath);
+    FileStatus dstStatus = createUploadFileStatus(dstPath, false,
+        size, blockSize, owner);
+    addMoveStatus(srcPaths, dstMetas, srcPath, dstStatus);
+  }
+
+  /**
+   * Helper method that records the move of all ancestors of a path.
+   *
+   * In S3A, an optimization is to delete unnecessary fake directory objects if
+   * the directory is non-empty. In that case, for a nested child to move, S3A
+   * is not listing and thus moving all its ancestors (up to source root). So we
+   * 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.
+   *
+   * @param ms MetadataStore, no-op if it is NullMetadataStore
+   * @param srcPaths stores the source path here
+   * @param dstMetas stores destination metadata here
+   * @param srcRoot source root up to which (exclusive) should we add ancestors
+   * @param srcPath source path of the child to add ancestors
+   * @param dstPath destination path of the child to add ancestors
+   * @param owner Hadoop user name
+   */
+  public static void addMoveAncestors(MetadataStore ms,
+      Collection<Path> srcPaths, Collection<PathMetadata> dstMetas,
+      Path srcRoot, Path srcPath, Path dstPath, String owner) {
+    if (isNullMetadataStore(ms)) {
+      return;
+    }
+
+    assertQualified(srcRoot, srcPath, dstPath);
+
+    if (srcPath.equals(srcRoot)) {
+      LOG.debug("Skip moving ancestors of source root directory {}", srcRoot);
+      return;
+    }
+
+    Path parentSrc = srcPath.getParent();
+    Path parentDst = dstPath.getParent();
+    while (parentSrc != null
+        && !parentSrc.isRoot()
+        && !parentSrc.equals(srcRoot)
+        && !srcPaths.contains(parentSrc)) {
+      LOG.debug("Renaming non-listed parent {} to {}", parentSrc, parentDst);
+      S3Guard.addMoveDir(ms, srcPaths, dstMetas, parentSrc, parentDst, owner);
+      parentSrc = parentSrc.getParent();
+      parentDst = parentDst.getParent();
+    }
+  }
+
+  public static void addAncestors(MetadataStore metadataStore,
+      Path qualifiedPath, String username) throws IOException {
+    Collection<PathMetadata> newDirs = new ArrayList<>();
+    Path parent = qualifiedPath.getParent();
+    while (!parent.isRoot()) {
+      PathMetadata directory = metadataStore.get(parent);
+      if (directory == null || directory.isDeleted()) {
+        FileStatus status = new FileStatus(0, true, 1, 0, 0, 0, null, username,
+            null, parent);
+        PathMetadata meta = new PathMetadata(status, Tristate.FALSE, false);
+        newDirs.add(meta);
+      } else {
+        break;
+      }
+      parent = parent.getParent();
+    }
+    metadataStore.put(newDirs);
+  }
+
+  private static void addMoveStatus(Collection<Path> srcPaths,
+      Collection<PathMetadata> dstMetas,
+      Path srcPath,
+      FileStatus dstStatus) {
+    srcPaths.add(srcPath);
+    dstMetas.add(new PathMetadata(dstStatus));
+  }
+
+  /**
+   * Assert that the path is qualified with a host and scheme.
+   * @param p path to check
+   * @throws NullPointerException if either argument does not hold
+   */
+  public static void assertQualified(Path p) {
+    URI uri = p.toUri();
+    // Paths must include bucket in case MetadataStore is shared between
+    // multiple S3AFileSystem instances
+    Preconditions.checkNotNull(uri.getHost(), "Null host in " + uri);
+
+    // This should never fail, but is retained for completeness.
+    Preconditions.checkNotNull(uri.getScheme(), "Null scheme in " + uri);
+  }
+
+  /**
+   * Assert that all paths are valid.
+   * @param paths path to check
+   * @throws NullPointerException if either argument does not hold
+   */
+  public static void assertQualified(Path...paths) {
+    for (Path path : paths) {
+      assertQualified(path);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..be271ae
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java
@@ -0,0 +1,924 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.FileNotFoundException;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.s3a.Constants;
+import org.apache.hadoop.fs.s3a.S3AFileStatus;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.shell.CommandFormat;
+import org.apache.hadoop.util.GenericOptionsParser;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+import static org.apache.hadoop.fs.s3a.Constants.*;
+
+/**
+ * CLI to manage S3Guard Metadata Store.
+ */
+public abstract class S3GuardTool extends Configured implements Tool {
+  private static final Logger LOG = LoggerFactory.getLogger(S3GuardTool.class);
+
+  private static final String NAME = "s3guard";
+  private static final String COMMON_USAGE =
+      "When possible and not overridden by more specific options, metadata\n" +
+      "repository information will be inferred from the S3A URL (if provided)" +
+      "\n\n" +
+      "Generic options supported are:\n" +
+      "  -conf <config file> - specify an application configuration file\n" +
+      "  -D <property=value> - define a value for a given property\n";
+
+  private static final String USAGE = NAME +
+      " [command] [OPTIONS] [s3a://BUCKET]\n\n" +
+      "Commands: \n" +
+      "\t" + Init.NAME + " - " + Init.PURPOSE + "\n" +
+      "\t" + Destroy.NAME + " - " + Destroy.PURPOSE + "\n" +
+      "\t" + Import.NAME + " - " + Import.PURPOSE + "\n" +
+      "\t" + Diff.NAME + " - " + Diff.PURPOSE + "\n" +
+      "\t" + Prune.NAME + " - " + Prune.PURPOSE + "\n";
+  private static final String DATA_IN_S3_IS_PRESERVED
+      = "(all data in S3 is preserved";
+
+  abstract public String getUsage();
+
+  // Exit codes
+  static final int SUCCESS = 0;
+  static final int INVALID_ARGUMENT = 1;
+  static final int ERROR = 99;
+
+  private S3AFileSystem filesystem;
+  private MetadataStore store;
+  private final CommandFormat commandFormat;
+
+  private static final String META_FLAG = "meta";
+  private static final String DAYS_FLAG = "days";
+  private static final String HOURS_FLAG = "hours";
+  private static final String MINUTES_FLAG = "minutes";
+  private static final String SECONDS_FLAG = "seconds";
+
+  private static final String REGION_FLAG = "region";
+  private static final String READ_FLAG = "read";
+  private static final String WRITE_FLAG = "write";
+
+  /**
+   * Constructor a S3Guard tool with HDFS configuration.
+   * @param conf Configuration.
+   */
+  protected S3GuardTool(Configuration conf) {
+    super(conf);
+
+    commandFormat = new CommandFormat(0, Integer.MAX_VALUE);
+    // For metadata store URI
+    commandFormat.addOptionWithValue(META_FLAG);
+    // DDB region.
+    commandFormat.addOptionWithValue(REGION_FLAG);
+  }
+
+  /**
+   * Return sub-command name.
+   */
+  abstract String getName();
+
+  /**
+   * Parse DynamoDB region from either -m option or a S3 path.
+   *
+   * This function should only be called from {@link Init} or
+   * {@link Destroy}.
+   *
+   * @param paths remaining parameters from CLI.
+   * @return false for invalid parameters.
+   * @throws IOException on I/O errors.
+   */
+  boolean parseDynamoDBRegion(List<String> paths) throws IOException {
+    Configuration conf = getConf();
+    String fromCli = getCommandFormat().getOptValue(REGION_FLAG);
+    String fromConf = conf.get(S3GUARD_DDB_REGION_KEY);
+    boolean hasS3Path = !paths.isEmpty();
+
+    if (fromCli != null) {
+      if (fromCli.isEmpty()) {
+        System.err.println("No region provided with -" + REGION_FLAG + " flag");
+        return false;
+      }
+      if (hasS3Path) {
+        System.err.println("Providing both an S3 path and the -" + REGION_FLAG
+            + " flag is not supported. If you need to specify a different "
+            + "region than the S3 bucket, configure " + S3GUARD_DDB_REGION_KEY);
+        return false;
+      }
+      conf.set(S3GUARD_DDB_REGION_KEY, fromCli);
+      return true;
+    }
+
+    if (fromConf != null) {
+      if (fromConf.isEmpty()) {
+        System.err.printf("No region provided with config %s, %n",
+            S3GUARD_DDB_REGION_KEY);
+        return false;
+      }
+      return true;
+    }
+
+    if (hasS3Path) {
+      String s3Path = paths.get(0);
+      initS3AFileSystem(s3Path);
+      return true;
+    }
+
+    System.err.println("No region found from -" + REGION_FLAG + " flag, " +
+        "config, or S3 bucket");
+    return false;
+  }
+
+  /**
+   * Parse metadata store from command line option or HDFS configuration.
+   *
+   * @param forceCreate override the auto-creation setting to true.
+   * @return a initialized metadata store.
+   */
+  MetadataStore initMetadataStore(boolean forceCreate) throws IOException {
+    if (getStore() != null) {
+      return getStore();
+    }
+    Configuration conf;
+    if (filesystem == null) {
+      conf = getConf();
+    } else {
+      conf = filesystem.getConf();
+    }
+    String metaURI = getCommandFormat().getOptValue(META_FLAG);
+    if (metaURI != null && !metaURI.isEmpty()) {
+      URI uri = URI.create(metaURI);
+      LOG.info("create metadata store: {}", uri + " scheme: "
+          + uri.getScheme());
+      switch (uri.getScheme().toLowerCase(Locale.ENGLISH)) {
+      case "local":
+        setStore(new LocalMetadataStore());
+        break;
+      case "dynamodb":
+        setStore(new DynamoDBMetadataStore());
+        conf.set(S3GUARD_DDB_TABLE_NAME_KEY, uri.getAuthority());
+        if (forceCreate) {
+          conf.setBoolean(S3GUARD_DDB_TABLE_CREATE_KEY, true);
+        }
+        break;
+      default:
+        throw new IOException(
+            String.format("Metadata store %s is not supported", uri));
+      }
+    } else {
+      // CLI does not specify metadata store URI, it uses default metadata store
+      // DynamoDB instead.
+      setStore(new DynamoDBMetadataStore());
+      if (forceCreate) {
+        conf.setBoolean(S3GUARD_DDB_TABLE_CREATE_KEY, true);
+      }
+    }
+
+    if (filesystem == null) {
+      getStore().initialize(conf);
+    } else {
+      getStore().initialize(filesystem);
+    }
+    LOG.info("Metadata store {} is initialized.", getStore());
+    return getStore();
+  }
+
+  /**
+   * Initialize S3A FileSystem instance.
+   *
+   * @param path s3a URI
+   * @throws IOException
+   */
+  void initS3AFileSystem(String path) throws IOException {
+    URI uri;
+    try {
+      uri = new URI(path);
+    } catch (URISyntaxException e) {
+      throw new IOException(e);
+    }
+    // Make sure that S3AFileSystem does not hold an actual MetadataStore
+    // implementation.
+    Configuration conf = getConf();
+    conf.setClass(S3_METADATA_STORE_IMPL, NullMetadataStore.class,
+        MetadataStore.class);
+    FileSystem fs = FileSystem.get(uri, getConf());
+    if (!(fs instanceof S3AFileSystem)) {
+      throw new IOException(
+          String.format("URI %s is not a S3A file system: %s", uri,
+              fs.getClass().getName()));
+    }
+    filesystem = (S3AFileSystem) fs;
+  }
+
+  /**
+   * Parse CLI arguments and returns the position arguments.
+   * The options are stored in {@link #commandFormat}
+   *
+   * @param args command line arguments.
+   * @return the position arguments from CLI.
+   */
+  List<String> parseArgs(String[] args) {
+    return getCommandFormat().parse(args, 1);
+  }
+
+  protected S3AFileSystem getFilesystem() {
+    return filesystem;
+  }
+
+  protected void setFilesystem(S3AFileSystem filesystem) {
+    this.filesystem = filesystem;
+  }
+
+  @VisibleForTesting
+  public MetadataStore getStore() {
+    return store;
+  }
+
+  @VisibleForTesting
+  protected void setStore(MetadataStore store) {
+    Preconditions.checkNotNull(store);
+    this.store = store;
+  }
+
+  protected CommandFormat getCommandFormat() {
+    return commandFormat;
+  }
+
+  /**
+   * Create the metadata store.
+   */
+  static class Init extends S3GuardTool {
+    private static final String NAME = "init";
+    public static final String PURPOSE = "initialize metadata repository";
+    private static final String USAGE = NAME + " [OPTIONS] [s3a://BUCKET]\n" +
+        "\t" + PURPOSE + "\n\n" +
+        "Common options:\n" +
+        "  -" + META_FLAG + " URL - Metadata repository details " +
+          "(implementation-specific)\n" +
+        "\n" +
+        "Amazon DynamoDB-specific options:\n" +
+        "  -" + REGION_FLAG + " REGION - Service region for connections\n" +
+        "  -" + READ_FLAG + " UNIT - Provisioned read throughput units\n" +
+        "  -" + WRITE_FLAG + " UNIT - Provisioned write through put units\n" +
+        "\n" +
+        "  URLs for Amazon DynamoDB are of the form dynamodb://TABLE_NAME.\n" +
+        "  Specifying both the -" + REGION_FLAG + " option and an S3A path\n" +
+        "  is not supported.";
+
+    Init(Configuration conf) {
+      super(conf);
+      // read capacity.
+      getCommandFormat().addOptionWithValue(READ_FLAG);
+      // write capacity.
+      getCommandFormat().addOptionWithValue(WRITE_FLAG);
+    }
+
+    @Override
+    String getName() {
+      return NAME;
+    }
+
+    @Override
+    public String getUsage() {
+      return USAGE;
+    }
+
+    @Override
+    public int run(String[] args) throws IOException {
+      List<String> paths = parseArgs(args);
+
+      String readCap = getCommandFormat().getOptValue(READ_FLAG);
+      if (readCap != null && !readCap.isEmpty()) {
+        int readCapacity = Integer.parseInt(readCap);
+        getConf().setInt(S3GUARD_DDB_TABLE_CAPACITY_READ_KEY, readCapacity);
+      }
+      String writeCap = getCommandFormat().getOptValue(WRITE_FLAG);
+      if (writeCap != null && !writeCap.isEmpty()) {
+        int writeCapacity = Integer.parseInt(writeCap);
+        getConf().setInt(S3GUARD_DDB_TABLE_CAPACITY_WRITE_KEY, writeCapacity);
+      }
+
+      // Validate parameters.
+      if (!parseDynamoDBRegion(paths)) {
+        System.err.println(USAGE);
+        return INVALID_ARGUMENT;
+      }
+      initMetadataStore(true);
+      return SUCCESS;
+    }
+  }
+
+  /**
+   * Destroy a metadata store.
+   */
+  static class Destroy extends S3GuardTool {
+    private static final String NAME = "destroy";
+    public static final String PURPOSE = "destroy Metadata Store data "
+        + DATA_IN_S3_IS_PRESERVED;
+    private static final String USAGE = NAME + " [OPTIONS] [s3a://BUCKET]\n" +
+        "\t" + PURPOSE + "\n\n" +
+        "Common options:\n" +
+        "  -" + META_FLAG + " URL - Metadata repository details " +
+          "(implementation-specific)\n" +
+        "\n" +
+        "Amazon DynamoDB-specific options:\n" +
+        "  -" + REGION_FLAG + " REGION - Service region for connections\n" +
+        "\n" +
+        "  URLs for Amazon DynamoDB are of the form dynamodb://TABLE_NAME.\n" +
+        "  Specifying both the -" + REGION_FLAG + " option and an S3A path\n" +
+        "  is not supported.";
+
+    Destroy(Configuration conf) {
+      super(conf);
+    }
+
+    @Override
+    String getName() {
+      return NAME;
+    }
+
+    @Override
+    public String getUsage() {
+      return USAGE;
+    }
+
+    public int run(String[] args) throws IOException {
+      List<String> paths = parseArgs(args);
+      if (!parseDynamoDBRegion(paths)) {
+        System.err.println(USAGE);
+        return INVALID_ARGUMENT;
+      }
+
+      try {
+        initMetadataStore(false);
+      } catch (FileNotFoundException e) {
+        // indication that the table was not found
+        LOG.debug("Failed to bind to store to be destroyed", e);
+        LOG.info("Metadata Store does not exist.");
+        return SUCCESS;
+      }
+
+      Preconditions.checkState(getStore() != null,
+          "Metadata Store is not initialized");
+
+      getStore().destroy();
+      LOG.info("Metadata store is deleted.");
+      return SUCCESS;
+    }
+  }
+
+  /**
+   * Import s3 metadata to the metadata store.
+   */
+  static class Import extends S3GuardTool {
+    private static final String NAME = "import";
+    public static final String PURPOSE = "import metadata from existing S3 " +
+        "data";
+    private static final String USAGE = NAME + " [OPTIONS] [s3a://BUCKET]\n" +
+        "\t" + PURPOSE + "\n\n" +
+        "Common options:\n" +
+        "  -" + META_FLAG + " URL - Metadata repository details " +
+        "(implementation-specific)\n" +
+        "\n" +
+        "Amazon DynamoDB-specific options:\n" +
+        "  -" + REGION_FLAG + " REGION - Service region for connections\n" +
+        "\n" +
+        "  URLs for Amazon DynamoDB are of the form dynamodb://TABLE_NAME.\n" +
+        "  Specifying both the -" + REGION_FLAG + " option and an S3A path\n" +
+        "  is not supported.";
+
+    private final Set<Path> dirCache = new HashSet<>();
+
+    Import(Configuration conf) {
+      super(conf);
+    }
+
+    @Override
+    String getName() {
+      return NAME;
+    }
+
+    @Override
+    public String getUsage() {
+      return USAGE;
+    }
+
+    /**
+     * Put parents into MS and cache if the parents are not presented.
+     *
+     * @param f the file or an empty directory.
+     * @throws IOException on I/O errors.
+     */
+    private void putParentsIfNotPresent(FileStatus f) throws IOException {
+      Preconditions.checkNotNull(f);
+      Path parent = f.getPath().getParent();
+      while (parent != null) {
+        if (dirCache.contains(parent)) {
+          return;
+        }
+        FileStatus dir = DynamoDBMetadataStore.makeDirStatus(parent,
+            f.getOwner());
+        getStore().put(new PathMetadata(dir));
+        dirCache.add(parent);
+        parent = parent.getParent();
+      }
+    }
+
+    /**
+     * Recursively import every path under path.
+     * @return number of items inserted into MetadataStore
+     * @throws IOException on I/O errors.
+     */
+    private long importDir(FileStatus status) throws IOException {
+      Preconditions.checkArgument(status.isDirectory());
+      RemoteIterator<LocatedFileStatus> it = getFilesystem()
+          .listFilesAndEmptyDirectories(status.getPath(), true);
+      long items = 0;
+
+      while (it.hasNext()) {
+        LocatedFileStatus located = it.next();
+        FileStatus child;
+        if (located.isDirectory()) {
+          child = DynamoDBMetadataStore.makeDirStatus(located.getPath(),
+              located.getOwner());
+          dirCache.add(child.getPath());
+        } else {
+          child = new S3AFileStatus(located.getLen(),
+              located.getModificationTime(),
+              located.getPath(),
+              located.getBlockSize(),
+              located.getOwner());
+        }
+        putParentsIfNotPresent(child);
+        getStore().put(new PathMetadata(child));
+        items++;
+      }
+      return items;
+    }
+
+    @Override
+    public int run(String[] args) throws IOException {
+      List<String> paths = parseArgs(args);
+      if (paths.isEmpty()) {
+        System.err.println(getUsage());
+        return INVALID_ARGUMENT;
+      }
+      String s3Path = paths.get(0);
+      initS3AFileSystem(s3Path);
+
+      URI uri;
+      try {
+        uri = new URI(s3Path);
+      } catch (URISyntaxException e) {
+        throw new IOException(e);
+      }
+      String filePath = uri.getPath();
+      if (filePath.isEmpty()) {
+        // If they specify a naked S3 URI (e.g. s3a://bucket), we'll consider
+        // root to be the path
+        filePath = "/";
+      }
+      Path path = new Path(filePath);
+      FileStatus status = getFilesystem().getFileStatus(path);
+
+      initMetadataStore(false);
+
+      long items = 1;
+      if (status.isFile()) {
+        PathMetadata meta = new PathMetadata(status);
+        getStore().put(meta);
+      } else {
+        items = importDir(status);
+      }
+
+      System.out.printf("Inserted %d items into Metadata Store%n", items);
+
+      return SUCCESS;
+    }
+  }
+
+  /**
+   * Show diffs between the s3 and metadata store.
+   */
+  static class Diff extends S3GuardTool {
+    private static final String NAME = "diff";
+    public static final String PURPOSE = "report on delta between S3 and " +
+        "repository";
+    private static final String USAGE = NAME + " [OPTIONS] s3a://BUCKET\n" +
+        "\t" + PURPOSE + "\n\n" +
+        "Common options:\n" +
+        "  -" + META_FLAG + " URL - Metadata repository details " +
+        "(implementation-specific)\n" +
+        "\n" +
+        "Amazon DynamoDB-specific options:\n" +
+        "  -" + REGION_FLAG + " REGION - Service region for connections\n" +
+        "\n" +
+        "  URLs for Amazon DynamoDB are of the form dynamodb://TABLE_NAME.\n" +
+        "  Specifying both the -" + REGION_FLAG + " option and an S3A path\n" +
+        "  is not supported.";
+
+    private static final String SEP = "\t";
+    static final String S3_PREFIX = "S3";
+    static final String MS_PREFIX = "MS";
+
+    Diff(Configuration conf) {
+      super(conf);
+    }
+
+    @Override
+    String getName() {
+      return NAME;
+    }
+
+    @Override
+    public String getUsage() {
+      return USAGE;
+    }
+
+    /**
+     * Formats the output of printing a FileStatus in S3guard diff tool.
+     * @param status the status to print.
+     * @return the string of output.
+     */
+    private static String formatFileStatus(FileStatus status) {
+      return String.format("%s%s%d%s%s",
+          status.isDirectory() ? "D" : "F",
+          SEP,
+          status.getLen(),
+          SEP,
+          status.getPath().toString());
+    }
+
+    /**
+     * Compares metadata from 2 S3 FileStatus's to see if they differ.
+     * @param thisOne
+     * @param thatOne
+     * @return true if the metadata is not identical
+     */
+    private static boolean differ(FileStatus thisOne, FileStatus thatOne) {
+      Preconditions.checkArgument(!(thisOne == null && thatOne == null));
+      return (thisOne == null || thatOne == null) ||
+          (thisOne.getLen() != thatOne.getLen()) ||
+          (thisOne.isDirectory() != thatOne.isDirectory()) ||
+          (!thisOne.isDirectory() &&
+              thisOne.getModificationTime() != thatOne.getModificationTime());
+    }
+
+    /**
+     * Print difference, if any, between two file statuses to the output stream.
+     *
+     * @param msStatus file status from metadata store.
+     * @param s3Status file status from S3.
+     * @param out output stream.
+     */
+    private static void printDiff(FileStatus msStatus,
+                                  FileStatus s3Status,
+                                  PrintStream out) {
+      Preconditions.checkArgument(!(msStatus == null && s3Status == null));
+      if (msStatus != null && s3Status != null) {
+        Preconditions.checkArgument(
+            msStatus.getPath().equals(s3Status.getPath()),
+            String.format("The path from metadata store and s3 are different:" +
+            " ms=%s s3=%s", msStatus.getPath(), s3Status.getPath()));
+      }
+
+      if (differ(msStatus, s3Status)) {
+        if (s3Status != null) {
+          out.printf("%s%s%s%n", S3_PREFIX, SEP, formatFileStatus(s3Status));
+        }
+        if (msStatus != null) {
+          out.printf("%s%s%s%n", MS_PREFIX, SEP, formatFileStatus(msStatus));
+        }
+      }
+    }
+
+    /**
+     * Compare the metadata of the directory with the same path, on S3 and
+     * the metadata store, respectively. If one of them is null, consider the
+     * metadata of the directory and all its subdirectories are missing from
+     * the source.
+     *
+     * Pass the FileStatus obtained from s3 and metadata store to avoid one
+     * round trip to fetch the same metadata twice, because the FileStatus
+     * hve already been obtained from listStatus() / listChildren operations.
+     *
+     * @param msDir the directory FileStatus obtained from the metadata store.
+     * @param s3Dir the directory FileStatus obtained from S3.
+     * @param out the output stream to generate diff results.
+     * @throws IOException on I/O errors.
+     */
+    private void compareDir(FileStatus msDir, FileStatus s3Dir,
+                            PrintStream out) throws IOException {
+      Preconditions.checkArgument(!(msDir == null && s3Dir == null));
+      if (msDir != null && s3Dir != null) {
+        Preconditions.checkArgument(msDir.getPath().equals(s3Dir.getPath()),
+            String.format("The path from metadata store and s3 are different:" +
+             " ms=%s s3=%s", msDir.getPath(), s3Dir.getPath()));
+      }
+
+      Map<Path, FileStatus> s3Children = new HashMap<>();
+      if (s3Dir != null && s3Dir.isDirectory()) {
+        for (FileStatus status : getFilesystem().listStatus(s3Dir.getPath())) {
+          s3Children.put(status.getPath(), status);
+        }
+      }
+
+      Map<Path, FileStatus> msChildren = new HashMap<>();
+      if (msDir != null && msDir.isDirectory()) {
+        DirListingMetadata dirMeta =
+            getStore().listChildren(msDir.getPath());
+
+        if (dirMeta != null) {
+          for (PathMetadata meta : dirMeta.getListing()) {
+            FileStatus status = meta.getFileStatus();
+            msChildren.put(status.getPath(), status);
+          }
+        }
+      }
+
+      Set<Path> allPaths = new HashSet<>(s3Children.keySet());
+      allPaths.addAll(msChildren.keySet());
+
+      for (Path path : allPaths) {
+        FileStatus s3Status = s3Children.get(path);
+        FileStatus msStatus = msChildren.get(path);
+        printDiff(msStatus, s3Status, out);
+        if ((s3Status != null && s3Status.isDirectory()) ||
+            (msStatus != null && msStatus.isDirectory())) {
+          compareDir(msStatus, s3Status, out);
+        }
+      }
+      out.flush();
+    }
+
+    /**
+     * Compare both metadata store and S3 on the same path.
+     *
+     * @param path the path to be compared.
+     * @param out  the output stream to display results.
+     * @throws IOException on I/O errors.
+     */
+    private void compareRoot(Path path, PrintStream out) throws IOException {
+      Path qualified = getFilesystem().qualify(path);
+      FileStatus s3Status = null;
+      try {
+        s3Status = getFilesystem().getFileStatus(qualified);
+      } catch (FileNotFoundException e) {
+      }
+      PathMetadata meta = getStore().get(qualified);
+      FileStatus msStatus = (meta != null && !meta.isDeleted()) ?
+          meta.getFileStatus() : null;
+      compareDir(msStatus, s3Status, out);
+    }
+
+    @VisibleForTesting
+    public int run(String[] args, PrintStream out) throws IOException {
+      List<String> paths = parseArgs(args);
+      if (paths.isEmpty()) {
+        out.println(USAGE);
+        return INVALID_ARGUMENT;
+      }
+      String s3Path = paths.get(0);
+      initS3AFileSystem(s3Path);
+      initMetadataStore(true);
+
+      URI uri;
+      try {
+        uri = new URI(s3Path);
+      } catch (URISyntaxException e) {
+        throw new IOException(e);
+      }
+      Path root;
+      if (uri.getPath().isEmpty()) {
+        root = new Path("/");
+      } else {
+        root = new Path(uri.getPath());
+      }
+      root = getFilesystem().qualify(root);
+      compareRoot(root, out);
+      out.flush();
+      return SUCCESS;
+    }
+
+    @Override
+    public int run(String[] args) throws IOException {
+      return run(args, System.out);
+    }
+  }
+
+  /**
+   * Prune metadata that has not been modified recently.
+   */
+  static class Prune extends S3GuardTool {
+    private static final String NAME = "prune";
+    public static final String PURPOSE = "truncate older metadata from " +
+        "repository "
+        + DATA_IN_S3_IS_PRESERVED;;
+    private static final String USAGE = NAME + " [OPTIONS] [s3a://BUCKET]\n" +
+        "\t" + PURPOSE + "\n\n" +
+        "Common options:\n" +
+        "  -" + META_FLAG + " URL - Metadata repository details " +
+        "(implementation-specific)\n" +
+        "\n" +
+        "Amazon DynamoDB-specific options:\n" +
+        "  -" + REGION_FLAG + " REGION - Service region for connections\n" +
+        "\n" +
+        "  URLs for Amazon DynamoDB are of the form dynamodb://TABLE_NAME.\n" +
+        "  Specifying both the -" + REGION_FLAG + " option and an S3A path\n" +
+        "  is not supported.";
+
+    Prune(Configuration conf) {
+      super(conf);
+
+      CommandFormat format = getCommandFormat();
+      format.addOptionWithValue(DAYS_FLAG);
+      format.addOptionWithValue(HOURS_FLAG);
+      format.addOptionWithValue(MINUTES_FLAG);
+      format.addOptionWithValue(SECONDS_FLAG);
+    }
+
+    @VisibleForTesting
+    void setMetadataStore(MetadataStore ms) {
+      Preconditions.checkNotNull(ms);
+      this.setStore(ms);
+    }
+
+    @Override
+    String getName() {
+      return NAME;
+    }
+
+    @Override
+    public String getUsage() {
+      return USAGE;
+    }
+
+    private long getDeltaComponent(TimeUnit unit, String arg) {
+      String raw = getCommandFormat().getOptValue(arg);
+      if (raw == null || raw.isEmpty()) {
+        return 0;
+      }
+      Long parsed = Long.parseLong(raw);
+      return unit.toMillis(parsed);
+    }
+
+    @VisibleForTesting
+    public int run(String[] args, PrintStream out) throws
+        InterruptedException, IOException {
+      List<String> paths = parseArgs(args);
+      if (!parseDynamoDBRegion(paths)) {
+        System.err.println(USAGE);
+        return INVALID_ARGUMENT;
+      }
+      initMetadataStore(false);
+
+      Configuration conf = getConf();
+      long confDelta = conf.getLong(Constants.S3GUARD_CLI_PRUNE_AGE, 0);
+
+      long cliDelta = 0;
+      cliDelta += getDeltaComponent(TimeUnit.DAYS, "days");
+      cliDelta += getDeltaComponent(TimeUnit.HOURS, "hours");
+      cliDelta += getDeltaComponent(TimeUnit.MINUTES, "minutes");
+      cliDelta += getDeltaComponent(TimeUnit.SECONDS, "seconds");
+
+      if (confDelta <= 0 && cliDelta <= 0) {
+        System.err.println(
+            "You must specify a positive age for metadata to prune.");
+      }
+
+      // A delta provided on the CLI overrides if one is configured
+      long delta = confDelta;
+      if (cliDelta > 0) {
+        delta = cliDelta;
+      }
+
+      long now = System.currentTimeMillis();
+      long divide = now - delta;
+
+      getStore().prune(divide);
+
+      out.flush();
+      return SUCCESS;
+    }
+
+    @Override
+    public int run(String[] args) throws InterruptedException, IOException {
+      return run(args, System.out);
+    }
+  }
+
+  private static S3GuardTool command;
+
+  private static void printHelp() {
+    if (command == null) {
+      System.err.println("Usage: hadoop " + USAGE);
+      System.err.println("\tperform S3Guard metadata store " +
+          "administrative commands.");
+    } else {
+      System.err.println("Usage: hadoop " + command.getUsage());
+    }
+    System.err.println();
+    System.err.println(COMMON_USAGE);
+  }
+
+  /**
+   * Execute the command with the given arguments.
+   *
+   * @param args command specific arguments.
+   * @param conf Hadoop configuration.
+   * @return exit code.
+   * @throws Exception on I/O errors.
+   */
+  public static int run(String[] args, Configuration conf) throws
+      Exception {
+    /* ToolRunner.run does this too, but we must do it before looking at
+    subCommand or instantiating the cmd object below */
+    String[] otherArgs = new GenericOptionsParser(conf, args)
+        .getRemainingArgs();
+    if (otherArgs.length == 0) {
+      printHelp();
+      return INVALID_ARGUMENT;
+    }
+    final String subCommand = otherArgs[0];
+    switch (subCommand) {
+    case Init.NAME:
+      command = new Init(conf);
+      break;
+    case Destroy.NAME:
+      command = new Destroy(conf);
+      break;
+    case Import.NAME:
+      command = new Import(conf);
+      break;
+    case Diff.NAME:
+      command = new Diff(conf);
+      break;
+    case Prune.NAME:
+      command = new Prune(conf);
+      break;
+    default:
+      printHelp();
+      return INVALID_ARGUMENT;
+    }
+    return ToolRunner.run(conf, command, otherArgs);
+  }
+
+  /**
+   * Main entry point. Calls {@code System.exit()} on all execution paths.
+   * @param args argument list
+   */
+  public static void main(String[] args) {
+    try {
+      int ret = run(args, new Configuration());
+      System.exit(ret);
+    } catch (CommandFormat.UnknownOptionException e) {
+      System.err.println(e.getMessage());
+      printHelp();
+      System.exit(INVALID_ARGUMENT);
+    } catch (Throwable e) {
+      e.printStackTrace(System.err);
+      System.exit(ERROR);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/package-info.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/package-info.java
new file mode 100644
index 0000000..d430315
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/package-info.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * This package contains classes related to S3Guard: a feature of S3A to mask
+ * the eventual consistency behavior of S3 and optimize access patterns by
+ * coordinating with a strongly consistent external store for file system
+ * metadata.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+package org.apache.hadoop.fs.s3a.s3guard;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3xLoginHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3xLoginHelper.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3xLoginHelper.java
index 862ce6b..ce79284 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3xLoginHelper.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3xLoginHelper.java
@@ -105,6 +105,10 @@ public final class S3xLoginHelper {
    * @return a login tuple, possibly empty.
    */
   public static Login extractLoginDetails(URI name) {
+    if (name == null) {
+      return Login.EMPTY;
+    }
+
     try {
       String authority = name.getAuthority();
       if (authority == null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
index 160b148..9318ed1 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
@@ -41,6 +41,7 @@ See also:
 
 * [Testing](testing.html)
 * [Troubleshooting S3a](troubleshooting_s3a.html)
+* [S3Guard](s3guard.html)
 
 ### Warning #1: Object Stores are not filesystems
 
@@ -1595,7 +1596,7 @@ for `fs.s3a.server-side-encryption-algorithm` is `AES256`.
 
 SSE-KMS is where the user specifies a Customer Master Key(CMK) that is used to
 encrypt the objects. The user may specify a specific CMK or leave the
-`fs.s3a.server-side-encryption-key` empty to use the default auto-generated key
+`fs.s3a.server-side-encryption.key` empty to use the default auto-generated key
 in AWS IAM.  Each CMK configured in AWS IAM is region specific, and cannot be
 used in a in a S3 bucket in a different region.  There is can also be policies
 assigned to the CMK that prohibit or restrict its use for users causing S3A

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..fe67d69
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md
@@ -0,0 +1,610 @@
+<!---
+  Licensed 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. See accompanying LICENSE file.
+-->
+
+# S3Guard: Consistency and Metadata Caching for S3A
+
+**Experimental Feature**
+
+<!-- MACRO{toc|fromDepth=0|toDepth=5} -->
+
+## Overview
+
+*S3Guard* is an experimental feature for the S3A client of the S3 object store,
+which can use a (consistent) database as the store of metadata about objects
+in an S3 bucket.
+
+S3Guard
+
+1. May improve performance on directory listing/scanning operations,
+including those which take place during the partitioning period of query
+execution, the process where files are listed and the work divided up amongst
+processes.
+
+1. Permits a consistent view of the object store. Without this, changes in
+objects may not be immediately visible, especially in listing operations.
+
+1. Offers a platform for future performance improvements for running Hadoop
+workloads on top of object stores
+
+The basic idea is that, for each operation in the Hadoop S3 client (s3a) that
+reads or modifies metadata, a shadow copy of that metadata is stored in a
+separate MetadataStore implementation.  Each MetadataStore implementation
+offers HDFS-like consistency for the metadata, and may also provide faster
+lookups for things like file status or directory listings.
+
+For links to early design documents and related patches, see
+[HADOOP-13345](https://issues.apache.org/jira/browse/HADOOP-13345).
+
+*Important*
+
+* S3Guard is experimental and should be considered unstable.
+
+* While all underlying data is persisted in S3, if, for some reason,
+the S3Guard-cached metadata becomes inconsistent with that in S3,
+queries on the data may become incorrect.
+For example, new datasets may be omitted, objects may be overwritten,
+or clients may not be aware that some data has been deleted.
+It is essential for all clients writing to an S3Guard-enabled
+S3 Repository to use the feature. Clients reading the data may work directly
+with the S3A data, in which case the normal S3 consistency guarantees apply.
+
+
+## Setting up S3Guard
+
+The latest configuration parameters are defined in `core-default.xml`.  You
+should consult that file for full information, but a summary is provided here.
+
+
+### 1. Choose the Database
+
+A core concept of S3Guard is that the directory listing data of the object
+store, *the metadata* is replicated in a higher-performance, consistent,
+database. In S3Guard, this database is called *The Metadata Store*
+
+By default, S3Guard is not enabled.
+
+The Metadata Store to use in production is bonded to Amazon's DynamoDB
+database service.  The following setting will enable this Metadata Store:
+
+```xml
+<property>
+    <name>fs.s3a.metadatastore.impl</name>
+    <value>org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStore</value>
+</property>
+```
+
+Note that the `NullMetadataStore` store can be explicitly requested if desired.
+This offers no metadata storage, and effectively disables S3Guard.
+
+```xml
+<property>
+    <name>fs.s3a.metadatastore.impl</name>
+    <value>org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore</value>
+</property>
+```
+
+### 2. Configure S3Guard Settings
+
+More settings will may be added in the future.
+Currently the only Metadata Store-independent setting, besides the
+implementation class above, is the *allow authoritative* flag.
+
+It is recommended that you leave the default setting here:
+
+```xml
+<property>
+    <name>fs.s3a.metadatastore.authoritative</name>
+    <value>false</value>
+</property>
+
+```
+
+Setting this to `true` is currently an experimental feature.  When true, the
+S3A client will avoid round-trips to S3 when getting directory listings, if
+there is a fully-cached version of the directory stored in the Metadata Store.
+
+Note that if this is set to true, it may exacerbate or persist existing race
+conditions around multiple concurrent modifications and listings of a given
+directory tree.
+
+In particular: **If the Metadata Store is declared as authoritative,
+all interactions with the S3 bucket(s) must be through S3A clients sharing
+the same Metadata Store**
+
+
+### 3. Configure the Metadata Store.
+
+Here are the `DynamoDBMetadataStore` settings.  Other Metadata Store
+implementations will have their own configuration parameters.
+
+
+### 4. Name Your Table
+
+First, choose the name of the table you wish to use for the S3Guard metadata
+storage in your DynamoDB instance.  If you leave it unset/empty, a
+separate table will be created for each S3 bucket you access, and that
+bucket's name will be used for the name of the DynamoDB table.  For example,
+this sets the table name to `my-ddb-table-name`
+
+```xml
+<property>
+  <name>fs.s3a.s3guard.ddb.table</name>
+  <value>my-ddb-table-name</value>
+  <description>
+    The DynamoDB table name to operate. Without this property, the respective
+    S3 bucket names will be used.
+  </description>
+</property>
+```
+
+It is good to share a table across multiple buckets for multiple reasons.
+
+1. You are billed for the I/O capacity allocated to the table,
+*even when the table is not used*. Sharing capacity can reduce costs.
+
+1. You can share the "provision burden" across the buckets. That is, rather
+than allocating for the peak load on a single bucket, you can allocate for
+the peak load *across all the buckets*, which is likely to be significantly
+lower.
+
+1. It's easier to measure and tune the load requirements and cost of
+S3Guard, because there is only one table to review and configure in the
+AWS management console.
+
+When wouldn't you want to share a table?
+
+1. When you do explicitly want to provision I/O capacity to a specific bucket
+and table, isolated from others.
+
+1. When you are using separate billing for specific buckets allocated
+to specific projects.
+
+1. When different users/roles have different access rights to different buckets.
+As S3Guard requires all users to have R/W access to the table, all users will
+be able to list the metadata in all buckets, even those to which they lack
+read access.
+
+### 5. Locate your Table
+
+You may also wish to specify the region to use for DynamoDB.  If a region
+is not configured, S3A will assume that it is in the same region as the S3
+bucket. A list of regions for the DynamoDB service can be found in
+[Amazon's documentation](http://docs.aws.amazon.com/general/latest/gr/rande.html#ddb_region).
+In this example, to use the US West 2 region:
+
+```xml
+<property>
+  <name>fs.s3a.s3guard.ddb.region</name>
+  <value>us-west-2</value>
+</property>
+```
+
+When working with S3Guard-managed buckets from EC2 VMs running in AWS
+infrastructure, using a local DynamoDB region ensures the lowest latency
+and highest reliability, as well as avoiding all long-haul network charges.
+The S3Guard tables, and indeed, the S3 buckets, should all be in the same
+region as the VMs.
+
+### 6. Optional: Create your Table
+
+Next, you can choose whether or not the table will be automatically created
+(if it doesn't already exist).  If you want this feature, set the
+`fs.s3a.s3guard.ddb.table.create` option to `true`.
+
+```xml
+<property>
+  <name>fs.s3a.s3guard.ddb.table.create</name>
+  <value>true</value>
+  <description>
+    If true, the S3A client will create the table if it does not already exist.
+  </description>
+</property>
+```
+
+### 7. If creating a table: Set your DynamoDB IO Capacity
+
+Next, you need to set the DynamoDB read and write throughput requirements you
+expect to need for your cluster.  Setting higher values will cost you more
+money.  *Note* that these settings only affect table creation when
+`fs.s3a.s3guard.ddb.table.create` is enabled.  To change the throughput for
+an existing table, use the AWS console or CLI tool.
+
+For more details on DynamoDB capacity units, see the AWS page on [Capacity
+Unit Calculations](http://docs.aws.amazon.com/amazondynamodb/latest/developerguide/WorkingWithTables.html#CapacityUnitCalculations).
+
+The charges are incurred per hour for the life of the table, *even when the
+table and the underlying S3 buckets are not being used*.
+
+There are also charges incurred for data storage and for data IO outside of the
+region of the DynamoDB instance. S3Guard only stores metadata in DynamoDB: path names
+and summary details of objects —the actual data is stored in S3, so billed at S3
+rates.
+
+```xml
+<property>
+  <name>fs.s3a.s3guard.ddb.table.capacity.read</name>
+  <value>500</value>
+  <description>
+    Provisioned throughput requirements for read operations in terms of capacity
+    units for the DynamoDB table.  This config value will only be used when
+    creating a new DynamoDB table, though later you can manually provision by
+    increasing or decreasing read capacity as needed for existing tables.
+    See DynamoDB documents for more information.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.s3guard.ddb.table.capacity.write</name>
+  <value>100</value>
+  <description>
+    Provisioned throughput requirements for write operations in terms of
+    capacity units for the DynamoDB table.  Refer to related config
+    fs.s3a.s3guard.ddb.table.capacity.read before usage.
+  </description>
+</property>
+```
+
+Attempting to perform more IO than the capacity requested simply throttles the
+IO; small capacity numbers are recommended when initially experimenting
+with S3Guard.
+
+## Authenticating with S3Guard
+
+The DynamoDB metadata store takes advantage of the fact that the DynamoDB
+service uses the same authentication mechanisms as S3. S3Guard
+gets all its credentials from the S3A client that is using it.
+
+All existing S3 authentication mechanisms can be used, except for one
+exception. Credentials placed in URIs are not supported for S3Guard, for security
+reasons.
+
+## Per-bucket S3Guard configuration
+
+In production, it is likely only some buckets will have S3Guard enabled;
+those which are read-only may have disabled, for example. Equally importantly,
+buckets in different regions should have different tables, each
+in the relevant region.
+
+These options can be managed through S3A's [per-bucket configuration
+mechanism](./index.html#Configuring_different_S3_buckets).
+All options with the under `fs.s3a.bucket.BUCKETNAME.KEY` are propagated
+to the options `fs.s3a.KEY` *for that bucket only*.
+
+As an example, here is a configuration to use different metadata stores
+and tables for different buckets
+
+First, we define shortcuts for the metadata store classnames
+
+
+```xml
+<property>
+  <name>s3guard.null</name>
+  <value>org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore</value>
+</property>
+
+<property>
+  <name>s3guard.dynamo</name>
+  <value>org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStore</value>
+</property>
+```
+
+Next, Amazon's public landsat database is configured with no
+metadata store
+
+```xml
+<property>
+  <name>fs.s3a.bucket.landsat-pds.metadatastore.impl</name>
+  <value>${s3guard.null}</value>
+  <description>The read-only landsat-pds repository isn't
+  managed by S3Guard</description>
+</property>
+```
+
+Next the `ireland-2` and `ireland-offline` buckets are configured with
+DynamoDB as the store, and a shared table `production-table`
+
+
+```xml
+<property>
+  <name>fs.s3a.bucket.ireland-2.metadatastore.impl</name>
+  <value>${s3guard.dynamo}</value>
+</property>
+
+<property>
+  <name>fs.s3a.bucket.ireland-offline.metadatastore.impl</name>
+  <value>${s3guard.dynamo}</value>
+</property>
+
+<property>
+  <name>fs.s3a.bucket.ireland-2.s3guard.ddb.table</name>
+  <value>production-table</value>
+</property>
+```
+
+The region of this table is automatically set to be that of the buckets,
+here `eu-west-1`; the same table name may actually be used in different
+regions.
+
+Together then, this configuration enables the DynamoDB Metadata Store
+for two buckets with a shared table, while disabling it for the public
+bucket.
+
+
+## S3Guard Command Line Interface (CLI)
+
+Note that in some cases an AWS region or `s3a://` URI can be provided.
+
+Metadata store URIs include a scheme that designates the backing store. For
+example (e.g. `dynamodb://table_name`;). As documented above, the
+AWS region can be inferred if the URI to an existing bucket is provided.
+
+
+The S3A URI must also be provided for per-bucket configuration options
+to be picked up. That is: when an s3a URL is provided on the command line,
+all its "resolved" per-bucket settings are used to connect to, authenticate
+with and configure the S3Guard table. If no such URL is provided, then
+the base settings are picked up.
+
+
+### Create a table: `s3guard init`
+
+```bash
+hadoop s3guard init -meta URI ( -region REGION | s3a://BUCKET )
+```
+
+Creates and initializes an empty metadata store.
+
+A DynamoDB metadata store can be initialized with additional parameters
+pertaining to [Provisioned Throughput](http://docs.aws.amazon.com/amazondynamodb/latest/developerguide/HowItWorks.ProvisionedThroughput.html):
+
+```bash
+[-write PROVISIONED_WRITES] [-read PROVISIONED_READS]
+```
+
+Example 1
+
+```bash
+hadoop s3guard init -meta dynamodb://ireland-team -write 5 -read 10 s3a://ireland-1
+```
+
+Creates a table "ireland-team" with a capacity of 5 for writes, 10 for reads,
+in the same location as the bucket "ireland-1".
+
+
+Example 2
+
+```bash
+hadoop s3guard init -meta dynamodb://ireland-team -region eu-west-1
+```
+
+Creates a table "ireland-team" in the same region "s3-eu-west-1.amazonaws.com"
+
+
+### Import a bucket: `s3guard import`
+
+```bash
+hadoop s3guard import [-meta URI] s3a://BUCKET
+```
+
+Pre-populates a metadata store according to the current contents of an S3
+bucket. If the `-meta` option is omitted, the binding information is taken
+from the `core-site.xml` configuration.
+
+Example
+
+```bash
+hadoop s3guard import s3a://ireland-1
+```
+
+### Audit a table: `s3guard diff`
+
+```bash
+hadoop s3guard diff [-meta URI] s3a://BUCKET
+```
+
+Lists discrepancies between a metadata store and bucket. Note that depending on
+how S3Guard is used, certain discrepancies are to be expected.
+
+Example
+
+```bash
+hadoop s3guard diff s3a://ireland-1
+```
+
+### Delete a table: `s3guard destroy`
+
+
+Deletes a metadata store. With DynamoDB as the store, this means
+the specific DynamoDB table use to store the metadata.
+
+```bash
+hadoop s3guard destroy [-meta URI] ( -region REGION | s3a://BUCKET )
+```
+
+This *does not* delete the bucket, only the S3Guard table which it is bound
+to.
+
+
+Examples
+
+```bash
+hadoop s3guard destroy s3a://ireland-1
+```
+
+Deletes the table which the bucket ireland-1 is configured to use
+as its MetadataStore.
+
+```bash
+hadoop s3guard destroy -meta dynamodb://ireland-team -region eu-west-1
+```
+
+
+
+### Clean up a table, `s3guard prune`
+
+Delete all file entries in the MetadataStore table whose object "modification
+time" is older than the specified age.
+
+```bash
+hadoop s3guard prune [-days DAYS] [-hours HOURS] [-minutes MINUTES]
+    [-seconds SECONDS] [-m URI] ( -region REGION | s3a://BUCKET )
+```
+
+A time value must be supplied.
+
+1. This does not delete the entries in the bucket itself.
+1. The modification time is effectively the creation time of the objects
+in the S3 Bucket.
+1. Even when an S3A URI is supplied, all entries in the table older than
+a specific age are deleted &mdash; even those from other buckets.
+
+Example
+
+```bash
+hadoop s3guard prune -days 7 s3a://ireland-1
+```
+
+Deletes all entries in the S3Guard table for files older than seven days from
+the table associated with `s3a://ireland-1`.
+
+```bash
+hadoop s3guard prune -hours 1 -minutes 30 -meta dynamodb://ireland-team -region eu-west-1
+```
+
+Delete all entries more than 90 minutes old from the table "ireland-team" in
+the region "eu-west-1".
+
+
+
+## Debugging and Error Handling
+
+If you run into network connectivity issues, or have a machine failure in the
+middle of an operation, you may end up with your metadata store having state
+that differs from S3.  The S3Guard CLI commands, covered in the CLI section
+above, can be used to diagnose and repair these issues.
+
+There are some logs whose log level can be increased to provide more
+information.
+
+```properties
+# Log S3Guard classes
+log4j.logger.org.apache.hadoop.fs.s3a.s3guard=DEBUG
+
+# Log all S3A classes
+log4j.logger.org.apache.hadoop.fs.s3a=DEBUG
+
+# Enable debug logging of AWS DynamoDB client
+log4j.logger.com.amazonaws.services.dynamodbv2.AmazonDynamoDB
+
+# Log all HTTP requests made; includes S3 interaction. This may
+# include sensitive information such as account IDs in HTTP headers.
+log4j.logger.com.amazonaws.request=DEBUG
+
+```
+
+If all else fails, S3Guard is designed to allow for easy recovery by deleting
+the metadata store data. In DynamoDB, this can be accomplished by simply
+deleting the table, and allowing S3Guard to recreate it from scratch.  Note
+that S3Guard tracks recent changes to file metadata to implement consistency.
+Deleting the metadata store table will simply result in a period of eventual
+consistency for any file modifications that were made right before the table
+was deleted.
+
+### Failure Semantics
+
+Operations which modify metadata will make changes to S3 first. If, and only
+if, those operations succeed, the equivalent changes will be made to the
+Metadata Store.
+
+These changes to S3 and Metadata Store are not fully-transactional:  If the S3
+operations succeed, and the subsequent Metadata Store updates fail, the S3
+changes will *not* be rolled back.  In this case, an error message will be
+logged.
+
+### Versioning
+
+S3Guard tables are created with a version marker, an entry with the primary
+key and child entry of `../VERSION`; the use of a relative path guarantees
+that it will not be resolved.
+
+#### Versioning policy.
+
+1. The version number of an S3Guard table will only be incremented when
+an incompatible change is made to the table structure —that is, the structure
+has changed so that it is no longer readable by older versions, or because
+it has added new mandatory fields which older versions do not create.
+1. The version number of S3Guard tables will only be changed by incrementing
+the value.
+1. Updated versions of S3Guard MAY continue to support older version tables.
+1. If an incompatible change is made such that existing tables are not compatible,
+then a means shall be provided to update existing tables. For example:
+an option in the Command Line Interface, or an option to upgrade tables
+during S3Guard initialization.
+
+*Note*: this policy does not indicate any intent to upgrade table structures
+in an incompatible manner. The version marker in tables exists to support
+such an option if it ever becomes necessary, by ensuring that all S3Guard
+client can recognise any version mismatch.
+
+### Security
+
+All users of the DynamoDB table must have write access to it. This
+effectively means they must have write access to the entire object store.
+
+There's not been much testing of using a S3Guard Metadata Store
+with a read-only S3 Bucket. It *should* work, provided all users
+have write access to the DynamoDB table. And, as updates to the Metadata Store
+are only made after successful file creation, deletion and rename, the
+store is *unlikely* to get out of sync, it is still something which
+merits more testing before it could be considered reliable.
+
+### Troubleshooting
+
+#### Error: `S3Guard table lacks version marker.`
+
+The table which was intended to be used as a S3guard metadata store
+does not have any version marker indicating that it is a S3Guard table.
+
+It may be that this is not a S3Guard table.
+
+* Make sure that this is the correct table name.
+* Delete the table, so it can be rebuilt.
+
+#### Error: `Database table is from an incompatible S3Guard version`
+
+This indicates that the version of S3Guard which created (or possibly updated)
+the database table is from a different version that that expected by the S3A
+client.
+
+This error will also include the expected and actual version numbers.
+
+If the expected version is lower than the actual version, then the version
+of the S3A client library is too old to interact with this S3Guard-managed
+bucket. Upgrade the application/library.
+
+If the expected version is higher than the actual version, then the table
+itself will need upgrading.
+
+#### Error `"DynamoDB table TABLE does not exist in region REGION; auto-creation is turned off"`
+
+S3Guard could not find the DynamoDB table for the Metadata Store,
+and it was not configured to create it. Either the table was missing,
+or the configuration is preventing S3Guard from finding the table.
+
+1. Verify that the value of `fs.s3a.s3guard.ddb.table` is correct.
+1. If the region for an existing table has been set in
+`fs.s3a.s3guard.ddb.region`, verify that the value is correct.
+1. If the region is not set, verify that the table exists in the same
+region as the bucket being used.
+1. Create the table if necessary.


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


[4/8] hadoop git commit: HADOOP-13345 S3Guard: Improved Consistency for S3A. Contributed by: Chris Nauroth, Aaron Fabbri, Mingliang Liu, Lei (Eddy) Xu, Sean Mackrory, Steve Loughran and others.

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md
index 10e19bd..20a778d 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md
@@ -108,6 +108,10 @@ each filesystem for its testing.
 1. `fs.contract.test.fs.s3n` : the URL of the bucket for S3n filesystem contract tests
 1. `fs.contract.test.fs.s3a` : the URL of the bucket for S3a filesystem contract tests
 
+*Note* that running s3a and s3n tests in parallel mode, against the same bucket
+is unreliable.  We recommend using separate buckets or testing one connector
+at a time.
+
 The contents of each bucket will be destroyed during the test process:
 do not use the bucket for any purpose other than testing. Furthermore, for
 s3a, all in-progress multi-part uploads to the bucket will be aborted at the
@@ -601,7 +605,7 @@ use requires the presence of secret credentials, where tests may be slow,
 and where finding out why something failed from nothing but the test output
 is critical.
 
-#### Subclasses Existing Shared Base Blasses
+#### Subclasses Existing Shared Base Classes
 
 Extend `AbstractS3ATestBase` or `AbstractSTestS3AHugeFiles` unless justifiable.
 These set things up for testing against the object stores, provide good threadnames,
@@ -745,7 +749,7 @@ Example:
 
 ### How to keep your credentials really safe
 
-Although the `auth-keys.xml` file is marged as ignored in git and subversion,
+Although the `auth-keys.xml` file is marked as ignored in git and subversion,
 it is still in your source tree, and there's always that risk that it may
 creep out.
 
@@ -760,3 +764,283 @@ using an absolute XInclude reference to it.
 
 </configuration>
 ```
+
+# Failure Injection
+
+**Warning do not enable any type of failure injection in production.  The
+following settings are for testing only.**
+
+One of the challenges with S3A integration tests is the fact that S3 is an
+eventually-consistent storage system.  In practice, we rarely see delays in
+visibility of recently created objects both in listings (`listStatus()`) and
+when getting a single file's metadata (`getFileStatus()`). Since this behavior
+is rare and non-deterministic, thorough integration testing is challenging.
+
+To address this, S3A supports a shim layer on top of the `AmazonS3Client`
+class which artificially delays certain paths from appearing in listings.
+This is implemented in the class `InconsistentAmazonS3Client`.
+
+## Simulating List Inconsistencies
+
+### Enabling the InconsistentAmazonS3CClient
+
+There are two ways of enabling the `InconsistentAmazonS3Client`: at
+config-time, or programmatically. For an example of programmatic test usage,
+see `ITestS3GuardListConsistency`.
+
+To enable the fault-injecting client via configuration, switch the
+S3A client to use the "Inconsistent S3 Client Factory" when connecting to
+S3:
+
+```xml
+<property>
+  <name>fs.s3a.s3.client.factory.impl</name>
+  <value>org.apache.hadoop.fs.s3a.InconsistentS3ClientFactory</value>
+</property>
+```
+
+The inconsistent client works by:
+
+1. Choosing which objects will be "inconsistent" at the time the object is
+created or deleted.
+2. When `listObjects()` is called, any keys that we have marked as
+inconsistent above will not be returned in the results (until the
+configured delay has elapsed). Similarly, deleted items may be *added* to
+missing results to delay the visibility of the delete.
+
+There are two ways of choosing which keys (filenames) will be affected: By
+substring, and by random probability.
+
+```xml
+<property>
+  <name>fs.s3a.failinject.inconsistency.key.substring</name>
+  <value>DELAY_LISTING_ME</value>
+</property>
+
+<property>
+  <name>fs.s3a.failinject.inconsistency.probability</name>
+  <value>1.0</value>
+</property>
+```
+
+By default, any object which has the substring "DELAY_LISTING_ME" in its key
+will subject to delayed visibility. For example, the path
+`s3a://my-bucket/test/DELAY_LISTING_ME/file.txt` would match this condition.
+To match all keys use the value "\*" (a single asterisk). This is a special
+value: *We don't support arbitrary wildcards.*
+
+The default probability of delaying an object is 1.0. This means that *all*
+keys that match the substring will get delayed visibility. Note that we take
+the logical *and* of the two conditions (substring matches *and* probability
+random chance occurs). Here are some example configurations:
+
+```
+| substring | probability |  behavior                                  |
+|-----------|-------------|--------------------------------------------|
+|           | 0.001       | An empty <value> tag in .xml config will   |
+|           |             | be interpreted as unset and revert to the  |
+|           |             | default value, "DELAY_LISTING_ME"          |
+|           |             |                                            |
+| *         | 0.001       | 1/1000 chance of *any* key being delayed.  |
+|           |             |                                            |
+| delay     | 0.01        | 1/100 chance of any key containing "delay" |
+|           |             |                                            |
+| delay     | 1.0         | All keys containing substring "delay" ..   |
+```
+
+You can also configure how long you want the delay in visibility to last.
+The default is 5000 milliseconds (five seconds).
+
+```xml
+<property>
+  <name>fs.s3a.failinject.inconsistency.msec</name>
+  <value>5000</value>
+</property>
+```
+
+Future versions of this client will introduce new failure modes,
+with simulation of S3 throttling exceptions the next feature under
+development.
+
+### Limitations of Inconsistency Injection
+
+Although `InconsistentAmazonS3Client` can delay the visibility of an object
+or parent directory, it does not prevent the key of that object from
+appearing in all prefix searches. For example, if we create the following
+object with the default configuration above, in an otherwise empty bucket:
+
+```
+s3a://bucket/a/b/c/DELAY_LISTING_ME
+```
+
+Then the following paths will still be visible as directories (ignoring
+possible real-world inconsistencies):
+
+```
+s3a://bucket/a
+s3a://bucket/a/b
+```
+
+Whereas `getFileStatus()` on the following *will* be subject to delayed
+visibility (`FileNotFoundException` until delay has elapsed):
+
+```
+s3a://bucket/a/b/c
+s3a://bucket/a/b/c/DELAY_LISTING_ME
+```
+
+In real-life S3 inconsistency, however, we expect that all the above paths
+(including `a` and `b`) will be subject to delayed visiblity.
+
+### Using the `InconsistentAmazonS3CClient` in downstream integration tests
+
+The inconsistent client is shipped in the `hadoop-aws` JAR, so it can
+be used in applications which work with S3 to see how they handle
+inconsistent directory listings.
+
+## Testing S3Guard
+
+The basic strategy for testing S3Guard correctness consists of:
+
+1. MetadataStore Contract tests.
+
+    The MetadataStore contract tests are inspired by the Hadoop FileSystem and
+    `FileContext` contract tests.  Each implementation of the `MetadataStore` interface
+    subclasses the `MetadataStoreTestBase` class and customizes it to initialize
+    their MetadataStore.  This test ensures that the different implementations
+    all satisfy the semantics of the MetadataStore API.
+
+2. Running existing S3A unit and integration tests with S3Guard enabled.
+
+    You can run the S3A integration tests on top of S3Guard by configuring your
+    `MetadataStore` in your
+    `hadoop-tools/hadoop-aws/src/test/resources/core-site.xml` or
+    `hadoop-tools/hadoop-aws/src/test/resources/auth-keys.xml` files.
+    Next run the S3A integration tests as outlined in the *Running the Tests* section
+    of the [S3A documentation](./index.html)
+
+3. Running fault-injection tests that test S3Guard's consistency features.
+
+    The `ITestS3GuardListConsistency` uses failure injection to ensure
+    that list consistency logic is correct even when the underlying storage is
+    eventually consistent.
+
+    The integration test adds a shim above the Amazon S3 Client layer that injects
+    delays in object visibility.
+
+    All of these tests will be run if you follow the steps listed in step 2 above.
+
+    No charges are incurred for using this store, and its consistency
+    guarantees are that of the underlying object store instance. <!-- :) -->
+
+## Testing S3A with S3Guard Enabled
+
+All the S3A tests which work with a private repository can be configured to
+run with S3Guard by using the `s3guard` profile. When set, this will run
+all the tests with local memory for the metadata set to "non-authoritative" mode.
+
+```bash
+mvn -T 1C verify -Dparallel-tests -DtestsThreadCount=6 -Ds3guard
+```
+
+When the `s3guard` profile is enabled, following profiles can be specified:
+
+* `dynamo`: use an AWS-hosted DynamoDB table; creating the table if it does
+  not exist. You will have to pay the bills for DynamoDB web service.
+* `dynamodblocal`: use an in-memory DynamoDBLocal server instead of real AWS
+  DynamoDB web service; launch the server and creating the table.
+  You won't be charged bills for using DynamoDB in test. As it runs in-JVM,
+  the table isn't shared across other tests running in parallel.
+* `non-auth`: treat the S3Guard metadata as authorative.
+
+```bash
+mvn -T 1C verify -Dparallel-tests -DtestsThreadCount=6 -Ds3guard -Ddynamo -Dauth
+```
+
+When experimenting with options, it is usually best to run a single test suite
+at a time until the operations appear to be working.
+
+```bash
+mvn -T 1C verify -Dtest=skip -Dit.test=ITestS3AMiscOperations -Ds3guard -Ddynamo
+```
+
+### Notes
+
+1. If the `s3guard` profile is not set, then the S3Guard properties are those
+of the test configuration set in `contract-test-options.xml` or `auth-keys.xml`
+
+If the `s3guard` profile *is* set,
+1. The S3Guard options from maven (the dynamo and authoritative flags)
+  overwrite any previously set in the configuration files.
+1. DynamoDB will be configured to create any missing tables.
+
+### Warning About Concurrent Tests
+
+You must not run S3A and S3N tests in parallel on the same bucket.  This is
+especially true when S3Guard is enabled.  S3Guard requires that all clients
+that are modifying the bucket have S3Guard enabled, so having S3N
+integration tests running in parallel with S3A tests will cause strange
+failures.
+
+### Scale Testing MetadataStore Directly
+
+There are some scale tests that exercise Metadata Store implementations
+directly. These ensure that S3Guard is are robust to things like DynamoDB
+throttling, and compare performance for different implementations. These
+are included in the scale tests executed when `-Dscale` is passed to
+the maven command line.
+
+The two S3Guard scale testse are `ITestDynamoDBMetadataStoreScale` and
+`ITestLocalMetadataStoreScale`.  To run the DynamoDB test, you will need to
+define your table name and region in your test configuration.  For example,
+the following settings allow us to run `ITestDynamoDBMetadataStoreScale` with
+artificially low read and write capacity provisioned, so we can judge the
+effects of being throttled by the DynamoDB service:
+
+```xml
+<property>
+  <name>scale.test.operation.count</name>
+  <value>10</value>
+</property>
+<property>
+  <name>scale.test.directory.count</name>
+  <value>3</value>
+</property>
+<property>
+  <name>fs.s3a.scale.test.enabled</name>
+  <value>true</value>
+</property>
+<property>
+  <name>fs.s3a.s3guard.ddb.table</name>
+  <value>my-scale-test</value>
+</property>
+<property>
+  <name>fs.s3a.s3guard.ddb.region</name>
+  <value>us-west-2</value>
+</property>
+<property>
+  <name>fs.s3a.s3guard.ddb.table.create</name>
+  <value>true</value>
+</property>
+<property>
+  <name>fs.s3a.s3guard.ddb.table.capacity.read</name>
+  <value>10</value>
+</property>
+<property>
+  <name>fs.s3a.s3guard.ddb.table.capacity.write</name>
+  <value>10</value>
+</property>
+```
+
+### Testing only: Local Metadata Store
+
+There is an in-memory Metadata Store for testing.
+
+```xml
+<property>
+  <name>fs.s3a.metadatastore.impl</name>
+  <value>org.apache.hadoop.fs.s3a.s3guard.LocalMetadataStore</value>
+</property>
+```
+
+This is not for use in production.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractCreate.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractCreate.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractCreate.java
index d2a858f..fd9497b 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractCreate.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractCreate.java
@@ -22,11 +22,25 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractCreateTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
 
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeEnableS3Guard;
+
 /**
  * S3A contract tests creating files.
  */
 public class ITestS3AContractCreate extends AbstractContractCreateTest {
 
+  /**
+   * Create a configuration, possibly patching in S3Guard options.
+   * @return a configuration
+   */
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    // patch in S3Guard options
+    maybeEnableS3Guard(conf);
+    return conf;
+  }
+
   @Override
   protected AbstractFSContract createContract(Configuration conf) {
     return new S3AContract(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDelete.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDelete.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDelete.java
index a47dcaef..95ea410 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDelete.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDelete.java
@@ -22,11 +22,25 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractDeleteTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
 
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeEnableS3Guard;
+
 /**
  * S3A contract tests covering deletes.
  */
 public class ITestS3AContractDelete extends AbstractContractDeleteTest {
 
+  /**
+   * Create a configuration, possibly patching in S3Guard options.
+   * @return a configuration
+   */
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    // patch in S3Guard options
+    maybeEnableS3Guard(conf);
+    return conf;
+  }
+
   @Override
   protected AbstractFSContract createContract(Configuration conf) {
     return new S3AContract(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java
index 50ce0c2..587dbbc 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.fs.contract.s3a;
 
 import static org.apache.hadoop.fs.s3a.Constants.*;
 import static org.apache.hadoop.fs.s3a.S3ATestConstants.SCALE_TEST_TIMEOUT_MILLIS;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeEnableS3Guard;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.tools.contract.AbstractContractDistCpTest;
@@ -38,12 +39,18 @@ public class ITestS3AContractDistCp extends AbstractContractDistCpTest {
     return SCALE_TEST_TIMEOUT_MILLIS;
   }
 
+  /**
+   * Create a configuration, possibly patching in S3Guard options.
+   * @return a configuration
+   */
   @Override
   protected Configuration createConfiguration() {
     Configuration newConf = super.createConfiguration();
     newConf.setLong(MULTIPART_SIZE, MULTIPART_SETTING);
     newConf.setBoolean(FAST_UPLOAD, true);
     newConf.set(FAST_UPLOAD_BUFFER, FAST_UPLOAD_BUFFER_DISK);
+    // patch in S3Guard options
+    maybeEnableS3Guard(newConf);
     return newConf;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractGetFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractGetFileStatus.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractGetFileStatus.java
index c7ed5a3..cb9819c 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractGetFileStatus.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractGetFileStatus.java
@@ -23,6 +23,8 @@ import org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest;
 import org.apache.hadoop.fs.s3a.Constants;
 import org.apache.hadoop.fs.s3a.S3ATestUtils;
 
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeEnableS3Guard;
+
 /**
  * S3A contract tests covering getFileStatus.
  */
@@ -46,6 +48,8 @@ public class ITestS3AContractGetFileStatus
     S3ATestUtils.disableFilesystemCaching(conf);
     // aggressively low page size forces tests to go multipage
     conf.setInt(Constants.MAX_PAGING_KEYS, 2);
+    // patch in S3Guard options
+    maybeEnableS3Guard(conf);
     return conf;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractMkdir.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractMkdir.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractMkdir.java
index d953e7e..dba52e1 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractMkdir.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractMkdir.java
@@ -22,11 +22,25 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractMkdirTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
 
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeEnableS3Guard;
+
 /**
  * Test dir operations on S3A.
  */
 public class ITestS3AContractMkdir extends AbstractContractMkdirTest {
 
+  /**
+   * Create a configuration, possibly patching in S3Guard options.
+   * @return a configuration
+   */
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    // patch in S3Guard options
+    maybeEnableS3Guard(conf);
+    return conf;
+  }
+
   @Override
   protected AbstractFSContract createContract(Configuration conf) {
     return new S3AContract(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractOpen.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractOpen.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractOpen.java
index a7bdc0d..8e338b7 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractOpen.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractOpen.java
@@ -22,11 +22,25 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractOpenTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
 
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeEnableS3Guard;
+
 /**
  * S3A contract tests opening files.
  */
 public class ITestS3AContractOpen extends AbstractContractOpenTest {
 
+  /**
+   * Create a configuration, possibly patching in S3Guard options.
+   * @return a configuration
+   */
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    // patch in S3Guard options
+    maybeEnableS3Guard(conf);
+    return conf;
+  }
+
   @Override
   protected AbstractFSContract createContract(Configuration conf) {
     return new S3AContract(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRename.java
----------------------------------------------------------------------
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 5dba03d..4339649 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
@@ -26,12 +26,25 @@ import org.apache.hadoop.fs.Path;
 
 import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeEnableS3Guard;
 
 /**
  * S3A contract tests covering rename.
  */
 public class ITestS3AContractRename extends AbstractContractRenameTest {
 
+  /**
+   * Create a configuration, possibly patching in S3Guard options.
+   * @return a configuration
+   */
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    // patch in S3Guard options
+    maybeEnableS3Guard(conf);
+    return conf;
+  }
+
   @Override
   protected AbstractFSContract createContract(Configuration conf) {
     return new S3AContract(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRootDir.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRootDir.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRootDir.java
index 8383a77..5c2e2cd 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRootDir.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRootDir.java
@@ -28,6 +28,8 @@ import org.apache.hadoop.fs.contract.AbstractFSContract;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeEnableS3Guard;
+
 /**
  * root dir operations against an S3 bucket.
  */
@@ -37,6 +39,18 @@ public class ITestS3AContractRootDir extends
   private static final Logger LOG =
       LoggerFactory.getLogger(ITestS3AContractRootDir.class);
 
+  /**
+   * Create a configuration, possibly patching in S3Guard options.
+   * @return a configuration
+   */
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    // patch in S3Guard options
+    maybeEnableS3Guard(conf);
+    return conf;
+  }
+
   @Override
   protected AbstractFSContract createContract(Configuration conf) {
     return new S3AContract(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java
index 1572fbc..379ace8 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java
@@ -22,11 +22,25 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractSeekTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
 
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeEnableS3Guard;
+
 /**
  * S3A contract tests covering file seek.
  */
 public class ITestS3AContractSeek extends AbstractContractSeekTest {
 
+  /**
+   * Create a configuration, possibly patching in S3Guard options.
+   * @return a configuration
+   */
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    // patch in S3Guard options
+    maybeEnableS3Guard(conf);
+    return conf;
+  }
+
   @Override
   protected AbstractFSContract createContract(Configuration conf) {
     return new S3AContract(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java
index 6734947..0c7f7df 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java
@@ -26,6 +26,8 @@ import com.amazonaws.services.s3.AmazonS3;
 import java.net.URI;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
+import org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore;
 
 import org.junit.After;
 import org.junit.Before;
@@ -33,7 +35,8 @@ import org.junit.Rule;
 import org.junit.rules.ExpectedException;
 
 /**
- * Abstract base class for S3A unit tests using a mock S3 client.
+ * Abstract base class for S3A unit tests using a mock S3 client and a null
+ * metadata store.
  */
 public abstract class AbstractS3AMockTest {
 
@@ -55,6 +58,10 @@ public abstract class AbstractS3AMockTest {
     Configuration conf = new Configuration();
     conf.setClass(S3_CLIENT_FACTORY_IMPL, MockS3ClientFactory.class,
         S3ClientFactory.class);
+    // We explicitly disable MetadataStore even if it's configured. For unit
+    // test we don't issue request to AWS DynamoDB service.
+    conf.setClass(S3_METADATA_STORE_IMPL, NullMetadataStore.class,
+        MetadataStore.class);
     fs = new S3AFileSystem();
     URI uri = URI.create(FS_S3A + "://" + BUCKET);
     fs.initialize(uri, conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java
index c19b72c..f0c389d 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java
@@ -33,6 +33,7 @@ import java.io.IOException;
 
 import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeEnableS3Guard;
 
 /**
  * An extension of the contract test base set up for S3A tests.
@@ -65,6 +66,18 @@ public abstract class AbstractS3ATestBase extends AbstractFSContractTestBase
     return S3A_TEST_TIMEOUT;
   }
 
+  /**
+   * Create a configuration, possibly patching in S3Guard options.
+   * @return a configuration
+   */
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    // patch in S3Guard options
+    maybeEnableS3Guard(conf);
+    return conf;
+  }
+
   protected Configuration getConfiguration() {
     return getContract().getConf();
   }
@@ -99,10 +112,21 @@ public abstract class AbstractS3ATestBase extends AbstractFSContractTestBase
    */
   protected Path writeThenReadFile(String name, int len) throws IOException {
     Path path = path(name);
+    writeThenReadFile(path, len);
+    return path;
+  }
+
+  /**
+   * Write a file, read it back, validate the dataset. Overwrites the file
+   * if it is present
+   * @param path path to file
+   * @param len length of file
+   * @throws IOException any IO problem
+   */
+  protected void writeThenReadFile(Path path, int len) throws IOException {
     byte[] data = dataset(len, 'a', 'z');
     writeDataset(getFileSystem(), path, data, data.length, 1024 * 1024, true);
     ContractTestUtils.verifyFileContents(getFileSystem(), path, data);
-    return path;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AAWSCredentialsProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AAWSCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AAWSCredentialsProvider.java
index 22c4f7e..6601233 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AAWSCredentialsProvider.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AAWSCredentialsProvider.java
@@ -140,6 +140,10 @@ public class ITestS3AAWSCredentialsProvider {
       createFailingFS(conf);
     } catch (AccessDeniedException e) {
       // expected
+    } catch (AWSServiceIOException e) {
+      GenericTestUtils.assertExceptionContains(
+          "UnrecognizedClientException", e);
+      // expected
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
index 602fc35..b5284c0 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
@@ -25,6 +25,7 @@ import com.amazonaws.services.s3.S3ClientOptions;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang.reflect.FieldUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.s3native.S3xLoginHelper;
@@ -516,7 +517,7 @@ public class ITestS3AConfiguration {
       }
     });
     assertEquals("username", alice, fs.getUsername());
-    S3AFileStatus status = fs.getFileStatus(new Path("/"));
+    FileStatus status = fs.getFileStatus(new Path("/"));
     assertEquals("owner in " + status, alice, status.getOwner());
     assertEquals("group in " + status, alice, status.getGroup());
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACredentialsInURL.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACredentialsInURL.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACredentialsInURL.java
index 3b0ae26..95d44cc 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACredentialsInURL.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACredentialsInURL.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.fs.s3a;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
 import org.junit.After;
@@ -37,6 +38,7 @@ import java.net.URLEncoder;
 import java.nio.file.AccessDeniedException;
 
 import static org.apache.hadoop.fs.s3a.S3ATestConstants.TEST_FS_S3A_NAME;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.assumeS3GuardState;
 
 /**
  * Tests that credentials can go into the URL. This includes a valid
@@ -63,6 +65,11 @@ public class ITestS3ACredentialsInURL extends Assert {
   public void testInstantiateFromURL() throws Throwable {
 
     Configuration conf = new Configuration();
+
+    // Skip in the case of S3Guard with DynamoDB because it cannot get
+    // credentials for its own use if they're only in S3 URLs
+    assumeS3GuardState(false, conf);
+
     String accessKey = conf.get(Constants.ACCESS_KEY);
     String secretKey = conf.get(Constants.SECRET_KEY);
     String fsname = conf.getTrimmed(TEST_FS_S3A_NAME, "");
@@ -84,6 +91,7 @@ public class ITestS3ACredentialsInURL extends Assert {
     conf.unset(Constants.ACCESS_KEY);
     conf.unset(Constants.SECRET_KEY);
     fs = S3ATestUtils.createTestFileSystem(conf);
+
     String fsURI = fs.getUri().toString();
     assertFalse("FS URI contains a @ symbol", fsURI.contains("@"));
     assertFalse("FS URI contains a % symbol", fsURI.contains("%"));
@@ -119,13 +127,14 @@ public class ITestS3ACredentialsInURL extends Assert {
     Configuration conf = new Configuration();
     String fsname = conf.getTrimmed(TEST_FS_S3A_NAME, "");
     Assume.assumeNotNull(fsname);
+    assumeS3GuardState(false, conf);
     URI original = new URI(fsname);
     URI testURI = createUriWithEmbeddedSecrets(original, "user", "//");
 
     conf.set(TEST_FS_S3A_NAME, testURI.toString());
     try {
       fs = S3ATestUtils.createTestFileSystem(conf);
-      S3AFileStatus status = fs.getFileStatus(new Path("/"));
+      FileStatus status = fs.getFileStatus(new Path("/"));
       fail("Expected an AccessDeniedException, got " + status);
     } catch (AccessDeniedException e) {
       // expected

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ADelayedFNF.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ADelayedFNF.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ADelayedFNF.java
new file mode 100644
index 0000000..7abd474
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ADelayedFNF.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.junit.Test;
+
+import java.io.FileNotFoundException;
+import java.util.concurrent.Callable;
+
+/**
+ * Tests behavior of a FileNotFound error that happens after open(), i.e. on
+ * the first read.
+ */
+public class ITestS3ADelayedFNF extends AbstractS3ATestBase {
+
+
+  /**
+   * See debugging documentation
+   * <a href="https://cwiki.apache.org/confluence/display/HADOOP/S3A%3A+FileNotFound+Exception+on+Read">here</a>.
+   * @throws Exception
+   */
+  @Test
+  public void testNotFoundFirstRead() throws Exception {
+    FileSystem fs = getFileSystem();
+    Path p = path("some-file");
+    ContractTestUtils.createFile(fs, p, false, new byte[] {20, 21, 22});
+
+    final FSDataInputStream in = fs.open(p);
+    assertDeleted(p, false);
+
+    // This should fail since we deleted after the open.
+    LambdaTestUtils.intercept(FileNotFoundException.class,
+        new Callable<Integer>() {
+          @Override
+          public Integer call() throws Exception {
+            return in.read();
+          }
+        });
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEmptyDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEmptyDirectory.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEmptyDirectory.java
new file mode 100644
index 0000000..c55be5b
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEmptyDirectory.java
@@ -0,0 +1,83 @@
+/*
+ * 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;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.junit.Test;
+
+import java.io.IOException;
+
+/**
+ * Tests which exercise treatment of empty/non-empty directories.
+ */
+public class ITestS3AEmptyDirectory extends AbstractS3ATestBase {
+
+  @Test
+  public void testDirectoryBecomesEmpty() throws Exception {
+    S3AFileSystem fs = getFileSystem();
+
+    // 1. set up non-empty dir
+    Path dir = path("testEmptyDir");
+    Path child = path("testEmptyDir/dir2");
+    mkdirs(child);
+
+    S3AFileStatus status = getS3AFileStatus(fs, dir);
+    assertEmptyDirectory(false, status);
+
+    // 2. Make testEmptyDir empty
+    assertDeleted(child, false);
+    status = getS3AFileStatus(fs, dir);
+
+    assertEmptyDirectory(true, status);
+  }
+
+  private static void assertEmptyDirectory(boolean isEmpty, S3AFileStatus s) {
+    String msg = "dir is empty";
+    // Should *not* be Tristate.UNKNOWN since we request a definitive value
+    // in getS3AFileStatus() below
+    Tristate expected = Tristate.fromBool(isEmpty);
+    assertEquals(msg, expected, s.isEmptyDirectory());
+  }
+
+  @Test
+  public void testDirectoryBecomesNonEmpty() throws Exception {
+    S3AFileSystem fs = getFileSystem();
+
+    // 1. create empty dir
+    Path dir = path("testEmptyDir");
+    mkdirs(dir);
+
+    S3AFileStatus status = getS3AFileStatus(fs, dir);
+    assertEmptyDirectory(true, status);
+
+    // 2. Make testEmptyDir non-empty
+
+    ContractTestUtils.touch(fs, path("testEmptyDir/file1"));
+    status = getS3AFileStatus(fs, dir);
+
+    assertEmptyDirectory(false, status);
+  }
+
+  private S3AFileStatus getS3AFileStatus(S3AFileSystem fs, Path p) throws
+      IOException {
+    return fs.innerGetFileStatus(p, true /* want isEmptyDirectory value */);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java
index 3d73cac..e452288 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java
@@ -19,21 +19,23 @@
 package org.apache.hadoop.fs.s3a;
 
 import java.io.IOException;
+import java.nio.file.AccessDeniedException;
 import java.util.concurrent.Callable;
 
-import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.ExpectedException;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.contract.s3a.S3AContract;
+import org.apache.hadoop.io.IOUtils;
 
 import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
-import static org.apache.hadoop.fs.contract.ContractTestUtils.rm;
-import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfEncryptionTestsDisabled;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 
 /**
@@ -42,22 +44,39 @@ import static org.apache.hadoop.test.LambdaTestUtils.intercept;
  */
 public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
 
+  private static final String SERVICE_AMAZON_S3_STATUS_CODE_403
+      = "Service: Amazon S3; Status Code: 403;";
+  private static final String KEY_1
+      = "4niV/jPK5VFRHY+KNb6wtqYd4xXyMgdJ9XQJpcQUVbs=";
+  private static final String KEY_2
+      = "G61nz31Q7+zpjJWbakxfTOZW4VS0UmQWAq2YXhcTXoo=";
+  private static final String KEY_3
+      = "NTx0dUPrxoo9+LbNiT/gqf3z9jILqL6ilismFmJO50U=";
+  private static final String KEY_4
+      = "msdo3VvvZznp66Gth58a91Hxe/UpExMkwU9BHkIjfW8=";
+  private static final int TEST_FILE_LEN = 2048;
 
-  @Rule
-  public ExpectedException expectedException = ExpectedException.none();
-
+  /**
+   * Filesystem created with a different key.
+   */
+  private FileSystem fsKeyB;
 
   @Override
   protected Configuration createConfiguration() {
     Configuration conf = super.createConfiguration();
-    S3ATestUtils.disableFilesystemCaching(conf);
+    disableFilesystemCaching(conf);
     conf.set(Constants.SERVER_SIDE_ENCRYPTION_ALGORITHM,
         getSSEAlgorithm().getMethod());
-    conf.set(Constants.SERVER_SIDE_ENCRYPTION_KEY,
-        "4niV/jPK5VFRHY+KNb6wtqYd4xXyMgdJ9XQJpcQUVbs=");
+    conf.set(Constants.SERVER_SIDE_ENCRYPTION_KEY, KEY_1);
     return conf;
   }
 
+  @Override
+  public void teardown() throws Exception {
+    super.teardown();
+    IOUtils.closeStream(fsKeyB);
+  }
+
   /**
    * This will create and write to a file using encryption key A, then attempt
    * to read from it again with encryption key B.  This will not work as it
@@ -73,29 +92,28 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
     assumeEnabled();
     skipIfEncryptionTestsDisabled(getConfiguration());
 
-    final Path[] path = new Path[1];
-    intercept(java.nio.file.AccessDeniedException.class,
-        "Service: Amazon S3; Status Code: 403;",
-        new Callable<Void>() {
+    intercept(AccessDeniedException.class,
+        SERVICE_AMAZON_S3_STATUS_CODE_403,
+        new Callable<FileStatus>() {
           @Override
-          public Void call() throws Exception {
-            int len = 2048;
+          public FileStatus call() throws Exception {
+            int len = TEST_FILE_LEN;
             describe("Create an encrypted file of size " + len);
-            String src = createFilename(len);
-            path[0] = writeThenReadFile(src, len);
+            Path src = path("testCreateFileAndReadWithDifferentEncryptionKey");
+            writeThenReadFile(src, len);
 
             //extract the test FS
-            FileSystem fileSystem = createNewFileSystemWithSSECKey(
+            fsKeyB = createNewFileSystemWithSSECKey(
                 "kX7SdwVc/1VXJr76kfKnkQ3ONYhxianyL2+C3rPVT9s=");
             byte[] data = dataset(len, 'a', 'z');
-            ContractTestUtils.verifyFileContents(fileSystem, path[0], data);
-            throw new Exception("Fail");
+            ContractTestUtils.verifyFileContents(fsKeyB, src, data);
+            return fsKeyB.getFileStatus(src);
           }
         });
   }
 
   /**
-   * While each object has it's own key and should be distinct, this verifies
+   * While each object has its own key and should be distinct, this verifies
    * that hadoop treats object keys as a filesystem path.  So if a top level
    * dir is encrypted with keyA, a sublevel dir cannot be accessed with a
    * different keyB.
@@ -108,29 +126,23 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
   public void testCreateSubdirWithDifferentKey() throws Exception {
     assumeEnabled();
     skipIfEncryptionTestsDisabled(getConfiguration());
+    assumeS3GuardState(false, getConfiguration());
 
-    final Path[] path = new Path[1];
-    intercept(java.nio.file.AccessDeniedException.class,
+    intercept(AccessDeniedException.class,
         "Service: Amazon S3; Status Code: 403;",
-        new Callable<Void>() {
+        new Callable<FileStatus>() {
           @Override
-          public Void call() throws Exception {
-
-            path[0] = S3ATestUtils.createTestPath(
-                new Path(createFilename("dir/"))
-            );
-            Path nestedDirectory = S3ATestUtils.createTestPath(
-                new Path(createFilename("dir/nestedDir/"))
-            );
-            FileSystem fsKeyB = createNewFileSystemWithSSECKey(
-                "G61nz31Q7+zpjJWbakxfTOZW4VS0UmQWAq2YXhcTXoo=");
-            getFileSystem().mkdirs(path[0]);
+          public FileStatus call() throws Exception {
+            Path base = path("testCreateSubdirWithDifferentKey");
+            Path nestedDirectory = new Path(base, "nestedDir");
+            fsKeyB = createNewFileSystemWithSSECKey(
+                KEY_2);
+            getFileSystem().mkdirs(base);
             fsKeyB.mkdirs(nestedDirectory);
-
-            throw new Exception("Exception should be thrown.");
+            // expected to fail
+            return fsKeyB.getFileStatus(nestedDirectory);
           }
         });
-    rm(getFileSystem(), path[0], true, false);
   }
 
   /**
@@ -146,23 +158,18 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
     assumeEnabled();
     skipIfEncryptionTestsDisabled(getConfiguration());
 
-    final Path[] path = new Path[1];
-    intercept(java.nio.file.AccessDeniedException.class,
-        "Service: Amazon S3; Status Code: 403;",
-        new Callable<Void>() {
+    intercept(AccessDeniedException.class,
+        SERVICE_AMAZON_S3_STATUS_CODE_403,
+        new Callable<Boolean>() {
           @Override
-          public Void call() throws Exception {
-
-            int len = 2048;
-            String src = createFilename(len);
-            path[0] = writeThenReadFile(src, len);
-
-            FileSystem fsKeyB = createNewFileSystemWithSSECKey(
-                "NTx0dUPrxoo9+LbNiT/gqf3z9jILqL6ilismFmJO50U=");
-            fsKeyB.rename(path[0],
-                new Path(createFilename("different-path.txt")));
-
-            throw new Exception("Exception should be thrown.");
+          public Boolean call() throws Exception {
+            int len = TEST_FILE_LEN;
+            Path src = path(createFilename(len));
+            writeThenReadFile(src, len);
+            fsKeyB = createNewFileSystemWithSSECKey(KEY_3);
+            Path dest = path(createFilename("different-path.txt"));
+            getFileSystem().mkdirs(dest.getParent());
+            return fsKeyB.rename(src, dest);
           }
         });
   }
@@ -178,11 +185,11 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
     assumeEnabled();
     skipIfEncryptionTestsDisabled(getConfiguration());
 
-    String src = createFilename("original-path.txt");
-    Path path = writeThenReadFile(src, 2048);
-    Path newPath = path(createFilename("different-path.txt"));
-    getFileSystem().rename(path, newPath);
-    byte[] data = dataset(2048, 'a', 'z');
+    Path src = path("original-path.txt");
+    writeThenReadFile(src, TEST_FILE_LEN);
+    Path newPath = path("different-path.txt");
+    getFileSystem().rename(src, newPath);
+    byte[] data = dataset(TEST_FILE_LEN, 'a', 'z');
     ContractTestUtils.verifyFileContents(getFileSystem(), newPath, data);
   }
 
@@ -196,32 +203,27 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
   public void testListEncryptedDir() throws Exception {
     assumeEnabled();
     skipIfEncryptionTestsDisabled(getConfiguration());
+    assumeS3GuardState(false, getConfiguration());
 
-    Path nestedDirectory = S3ATestUtils.createTestPath(
-         path(createFilename("/a/b/c/"))
-    );
+    final Path pathABC = path("testListEncryptedDir/a/b/c/");
+    final Path pathAB = pathABC.getParent();
+    final Path pathA = pathAB.getParent();
+
+    final Path nestedDirectory = createTestPath(pathABC);
     assertTrue(getFileSystem().mkdirs(nestedDirectory));
 
-    final FileSystem fsKeyB = createNewFileSystemWithSSECKey(
-        "msdo3VvvZznp66Gth58a91Hxe/UpExMkwU9BHkIjfW8=");
+    fsKeyB = createNewFileSystemWithSSECKey(KEY_4);
 
-    fsKeyB.listFiles(S3ATestUtils.createTestPath(
-        path(createFilename("/a/"))
-    ), true);
-    fsKeyB.listFiles(S3ATestUtils.createTestPath(
-        path(createFilename("/a/b/"))
-    ), true);
+    fsKeyB.listFiles(pathA, true);
+    fsKeyB.listFiles(pathAB, true);
 
     //Until this point, no exception is thrown about access
-    intercept(java.nio.file.AccessDeniedException.class,
-        "Service: Amazon S3; Status Code: 403;",
-        new Callable<Void>() {
+    intercept(AccessDeniedException.class,
+        SERVICE_AMAZON_S3_STATUS_CODE_403,
+        new Callable<RemoteIterator<LocatedFileStatus>>() {
           @Override
-          public Void call() throws Exception {
-            fsKeyB.listFiles(S3ATestUtils.createTestPath(
-                path(createFilename("/a/b/c/"))
-            ), false);
-            throw new Exception("Exception should be thrown.");
+          public RemoteIterator<LocatedFileStatus> call() throws Exception {
+            return fsKeyB.listFiles(pathABC, false);
           }
         });
 
@@ -234,25 +236,16 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
     final FileSystem unencryptedFileSystem = contract.getTestFileSystem();
 
     //unencrypted can access until the final directory
-    unencryptedFileSystem.listFiles(S3ATestUtils.createTestPath(
-        path(createFilename("/a/"))
-    ), true);
-    unencryptedFileSystem.listFiles(S3ATestUtils.createTestPath(
-        path(createFilename("/a/b/"))
-    ), true);
-    intercept(org.apache.hadoop.fs.s3a.AWSS3IOException.class,
-        "Bad Request (Service: Amazon S3; Status Code: 400; Error" +
-            " Code: 400 Bad Request;",
-        new Callable<Void>() {
+    unencryptedFileSystem.listFiles(pathA, true);
+    unencryptedFileSystem.listFiles(pathAB, true);
+    AWSS3IOException ex = intercept(AWSS3IOException.class,
+        new Callable<RemoteIterator<LocatedFileStatus>>() {
           @Override
-          public Void call() throws Exception {
-            unencryptedFileSystem.listFiles(S3ATestUtils.createTestPath(
-                path(createFilename("/a/b/c/"))
-            ), false);
-            throw new Exception("Exception should be thrown.");
+          public RemoteIterator<LocatedFileStatus> call() throws Exception {
+            return unencryptedFileSystem.listFiles(pathABC, false);
           }
         });
-    rm(getFileSystem(), path(createFilename("/")), true, false);
+    assertStatusCode(ex, 400);
   }
 
   /**
@@ -264,35 +257,30 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
   public void testListStatusEncryptedDir() throws Exception {
     assumeEnabled();
     skipIfEncryptionTestsDisabled(getConfiguration());
+    assumeS3GuardState(false, getConfiguration());
 
-    Path nestedDirectory = S3ATestUtils.createTestPath(
-         path(createFilename("/a/b/c/"))
-    );
-    assertTrue(getFileSystem().mkdirs(nestedDirectory));
+    final Path pathABC = path("testListStatusEncryptedDir/a/b/c/");
+    final Path pathAB = pathABC.getParent();
+    final Path pathA = pathAB.getParent();
+    assertTrue(getFileSystem().mkdirs(pathABC));
 
-    final FileSystem fsKeyB = createNewFileSystemWithSSECKey(
-        "msdo3VvvZznp66Gth58a91Hxe/UpExMkwU9BHkIjfW8=");
+    fsKeyB = createNewFileSystemWithSSECKey(KEY_4);
 
-    fsKeyB.listStatus(S3ATestUtils.createTestPath(
-        path(createFilename("/a/"))));
-    fsKeyB.listStatus(S3ATestUtils.createTestPath(
-        path(createFilename("/a/b/"))));
+    fsKeyB.listStatus(pathA);
+    fsKeyB.listStatus(pathAB);
 
     //Until this point, no exception is thrown about access
-    intercept(java.nio.file.AccessDeniedException.class,
-        "Service: Amazon S3; Status Code: 403;",
-        new Callable<Void>() {
+    intercept(AccessDeniedException.class,
+        SERVICE_AMAZON_S3_STATUS_CODE_403,
+        new Callable<FileStatus[]>() {
           @Override
-          public Void call() throws Exception {
-            fsKeyB.listStatus(S3ATestUtils.createTestPath(
-                path(createFilename("/a/b/c/"))));
-
-            throw new Exception("Exception should be thrown.");
+          public FileStatus[] call() throws Exception {
+            return fsKeyB.listStatus(pathABC);
           }
         });
 
     //Now try it with an unencrypted filesystem.
-    Configuration conf = this.createConfiguration();
+    Configuration conf = createConfiguration();
     conf.unset(Constants.SERVER_SIDE_ENCRYPTION_ALGORITHM);
     conf.unset(Constants.SERVER_SIDE_ENCRYPTION_KEY);
 
@@ -301,22 +289,17 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
     final FileSystem unencryptedFileSystem = contract.getTestFileSystem();
 
     //unencrypted can access until the final directory
-    unencryptedFileSystem.listStatus(S3ATestUtils.createTestPath(
-        path(createFilename("/a/"))));
-    unencryptedFileSystem.listStatus(S3ATestUtils.createTestPath(
-        path(createFilename("/a/b/"))));
-    intercept(org.apache.hadoop.fs.s3a.AWSS3IOException.class,
-        "Bad Request (Service: Amazon S3; Status Code: 400; Error Code: 400" +
-            " Bad Request;", new Callable<Void>() {
-          @Override
-          public Void call() throws Exception {
+    unencryptedFileSystem.listStatus(pathA);
+    unencryptedFileSystem.listStatus(pathAB);
 
-            unencryptedFileSystem.listStatus(S3ATestUtils.createTestPath(
-                path(createFilename("/a/b/c/"))));
-            throw new Exception("Exception should be thrown.");
+    AWSS3IOException ex = intercept(AWSS3IOException.class,
+        new Callable<FileStatus[]>() {
+          @Override
+          public FileStatus[] call() throws Exception {
+            return unencryptedFileSystem.listStatus(pathABC);
           }
         });
-    rm(getFileSystem(), path(createFilename("/")), true, false);
+    assertStatusCode(ex, 400);
   }
 
   /**
@@ -328,34 +311,26 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
   public void testListStatusEncryptedFile() throws Exception {
     assumeEnabled();
     skipIfEncryptionTestsDisabled(getConfiguration());
+    assumeS3GuardState(false, getConfiguration());
+    final Path pathABC = path("testListStatusEncryptedFile/a/b/c/");
+    assertTrue(getFileSystem().mkdirs(pathABC));
 
-    Path nestedDirectory = S3ATestUtils.createTestPath(
-        path(createFilename("/a/b/c/"))
-    );
-    assertTrue(getFileSystem().mkdirs(nestedDirectory));
-
-    String src = createFilename("/a/b/c/fileToStat.txt");
-    final Path fileToStat =  writeThenReadFile(src, 2048);
+    final Path fileToStat = new Path(pathABC, "fileToStat.txt");
+    writeThenReadFile(fileToStat, TEST_FILE_LEN);
 
-    final FileSystem fsKeyB = createNewFileSystemWithSSECKey(
-        "msdo3VvvZznp66Gth58a91Hxe/UpExMkwU9BHkIjfW8=");
+    fsKeyB = createNewFileSystemWithSSECKey(KEY_4);
 
     //Until this point, no exception is thrown about access
-    intercept(java.nio.file.AccessDeniedException.class,
-        "Service: Amazon S3; Status Code: 403;",
-        new Callable<Void>() {
+    intercept(AccessDeniedException.class,
+        SERVICE_AMAZON_S3_STATUS_CODE_403,
+        new Callable<FileStatus[]>() {
           @Override
-          public Void call() throws Exception {
-        fsKeyB.listStatus(S3ATestUtils.createTestPath(fileToStat));
-
-        throw new Exception("Exception should be thrown.");
+          public FileStatus[] call() throws Exception {
+            return fsKeyB.listStatus(S3ATestUtils.createTestPath(fileToStat));
       }});
-    rm(getFileSystem(), path(createFilename("/")), true, false);
   }
 
 
-
-
   /**
    * It is possible to delete directories without the proper encryption key and
    * the hierarchy above it.
@@ -366,35 +341,29 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
   public void testDeleteEncryptedObjectWithDifferentKey() throws Exception {
     assumeEnabled();
     skipIfEncryptionTestsDisabled(getConfiguration());
-
-    Path nestedDirectory = S3ATestUtils.createTestPath(
-        path(createFilename("/a/b/c/"))
-    );
-    assertTrue(getFileSystem().mkdirs(nestedDirectory));
-    String src = createFilename("/a/b/c/filetobedeleted.txt");
-    final Path fileToDelete =  writeThenReadFile(src, 2048);
-
-    final FileSystem fsKeyB = createNewFileSystemWithSSECKey(
-        "msdo3VvvZznp66Gth58a91Hxe/UpExMkwU9BHkIjfW8=");
-    intercept(java.nio.file.AccessDeniedException.class,
-        "Forbidden (Service: Amazon S3; Status Code: 403; Error Code: " +
-        "403 Forbidden",
-          new Callable<Void>() {
+    assumeS3GuardState(false, getConfiguration());
+    final Path pathABC = path("testDeleteEncryptedObjectWithDifferentKey/a/b/c/");
+
+    final Path pathAB = pathABC.getParent();
+    final Path pathA = pathAB.getParent();
+    assertTrue(getFileSystem().mkdirs(pathABC));
+    final Path fileToDelete = new Path(pathABC, "filetobedeleted.txt");
+    writeThenReadFile(fileToDelete, TEST_FILE_LEN);
+    fsKeyB = createNewFileSystemWithSSECKey(KEY_4);
+    intercept(AccessDeniedException.class,
+        SERVICE_AMAZON_S3_STATUS_CODE_403,
+          new Callable<Boolean>() {
             @Override
-            public Void call() throws Exception {
-
-              fsKeyB.delete(fileToDelete, false);
-              throw new Exception("Exception should be thrown.");
+            public Boolean call() throws Exception {
+              return fsKeyB.delete(fileToDelete, false);
             }
           });
 
-  //This is possible
-    fsKeyB.delete(S3ATestUtils.createTestPath(
-        path(createFilename("/a/b/c/"))), true);
-    fsKeyB.delete(S3ATestUtils.createTestPath(
-        path(createFilename("/a/b/"))), true);
-    fsKeyB.delete(S3ATestUtils.createTestPath(
-        path(createFilename("/a/"))), true);
+    //This is possible
+    fsKeyB.delete(pathABC, true);
+    fsKeyB.delete(pathAB, true);
+    fsKeyB.delete(pathA, true);
+    assertPathDoesNotExist("expected recursive delete", fileToDelete);
   }
 
   private FileSystem createNewFileSystemWithSSECKey(String sseCKey) throws
@@ -412,4 +381,5 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
   protected S3AEncryptionMethods getSSEAlgorithm() {
     return S3AEncryptionMethods.SSE_C;
   }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java
index 00171f0..3e293f7 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.fs.s3a;
 
+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;
@@ -32,8 +33,8 @@ import java.net.URI;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
 import static org.apache.hadoop.fs.s3a.Statistic.*;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
-import static org.apache.hadoop.fs.s3a.S3ATestUtils.MetricDiff;
 import static org.apache.hadoop.test.GenericTestUtils.getTestDir;
+import static org.junit.Assume.assumeFalse;
 
 /**
  * Use metrics to assert about the cost of file status queries.
@@ -62,9 +63,11 @@ public class ITestS3AFileOperationCost extends AbstractS3ATestBase {
     S3AFileSystem fs = getFileSystem();
     touch(fs, simpleFile);
     resetMetricDiffs();
-    S3AFileStatus status = fs.getFileStatus(simpleFile);
+    FileStatus status = fs.getFileStatus(simpleFile);
     assertTrue("not a file: " + status, status.isFile());
-    metadataRequests.assertDiffEquals(1);
+    if (!fs.hasMetadataStore()) {
+      metadataRequests.assertDiffEquals(1);
+    }
     listRequests.assertDiffEquals(0);
   }
 
@@ -79,9 +82,13 @@ public class ITestS3AFileOperationCost extends AbstractS3ATestBase {
     Path dir = path("empty");
     fs.mkdirs(dir);
     resetMetricDiffs();
-    S3AFileStatus status = fs.getFileStatus(dir);
-    assertTrue("not empty: " + status, status.isEmptyDirectory());
-    metadataRequests.assertDiffEquals(2);
+    S3AFileStatus status = fs.innerGetFileStatus(dir, true);
+    assertTrue("not empty: " + status,
+        status.isEmptyDirectory() == Tristate.TRUE);
+
+    if (!fs.hasMetadataStore()) {
+      metadataRequests.assertDiffEquals(2);
+    }
     listRequests.assertDiffEquals(0);
   }
 
@@ -92,7 +99,7 @@ public class ITestS3AFileOperationCost extends AbstractS3ATestBase {
     Path path = path("missing");
     resetMetricDiffs();
     try {
-      S3AFileStatus status = fs.getFileStatus(path);
+      FileStatus status = fs.getFileStatus(path);
       fail("Got a status back from a missing file path " + status);
     } catch (FileNotFoundException expected) {
       // expected
@@ -108,7 +115,7 @@ public class ITestS3AFileOperationCost extends AbstractS3ATestBase {
     Path path = path("missingdir/missingpath");
     resetMetricDiffs();
     try {
-      S3AFileStatus status = fs.getFileStatus(path);
+      FileStatus status = fs.getFileStatus(path);
       fail("Got a status back from a missing file path " + status);
     } catch (FileNotFoundException expected) {
       // expected
@@ -126,16 +133,18 @@ public class ITestS3AFileOperationCost extends AbstractS3ATestBase {
     Path simpleFile = new Path(dir, "simple.txt");
     touch(fs, simpleFile);
     resetMetricDiffs();
-    S3AFileStatus status = fs.getFileStatus(dir);
-    if (status.isEmptyDirectory()) {
+    S3AFileStatus status = fs.innerGetFileStatus(dir, true);
+    if (status.isEmptyDirectory() == Tristate.TRUE) {
       // erroneous state
       String fsState = fs.toString();
       fail("FileStatus says directory isempty: " + status
           + "\n" + ContractTestUtils.ls(fs, dir)
           + "\n" + fsState);
     }
-    metadataRequests.assertDiffEquals(2);
-    listRequests.assertDiffEquals(1);
+    if (!fs.hasMetadataStore()) {
+      metadataRequests.assertDiffEquals(2);
+      listRequests.assertDiffEquals(1);
+    }
   }
 
   @Test
@@ -187,6 +196,13 @@ public class ITestS3AFileOperationCost extends AbstractS3ATestBase {
         + "In S3, rename deletes any fake directories as a part of "
         + "clean up activity");
     S3AFileSystem fs = getFileSystem();
+
+    // As this test uses the s3 metrics to count the number of fake directory
+    // operations, it depends on side effects happening internally. With
+    // metadata store enabled, it is brittle to change. We disable this test
+    // before the internal behavior w/ or w/o metadata store.
+    assumeFalse(fs.hasMetadataStore());
+
     Path srcBaseDir = path("src");
     mkdirs(srcBaseDir);
     MetricDiff deleteRequests =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileSystemContract.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileSystemContract.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileSystemContract.java
index 6fcf4c7..ef99b80 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileSystemContract.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileSystemContract.java
@@ -27,6 +27,9 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystemContractBaseTest;
 import org.apache.hadoop.fs.Path;
 
+import static org.junit.Assume.*;
+import static org.junit.Assert.*;
+
 /**
  *  Tests a live S3 system. If your keys and bucket aren't specified, all tests
  *  are marked as passed.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AInconsistency.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AInconsistency.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AInconsistency.java
new file mode 100644
index 0000000..eb4f70b
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AInconsistency.java
@@ -0,0 +1,100 @@
+/*
+ * 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;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.contract.s3a.S3AContract;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.junit.Test;
+
+import java.io.FileNotFoundException;
+import java.util.concurrent.Callable;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
+import static org.apache.hadoop.fs.s3a.Constants.*;
+import static org.apache.hadoop.fs.s3a.InconsistentAmazonS3Client.*;
+
+/**
+ * Tests S3A behavior under forced inconsistency via {@link
+ * InconsistentAmazonS3Client}.
+ *
+ * These tests are for validating expected behavior *without* S3Guard, but
+ * may also run with S3Guard enabled.  For tests that validate S3Guard's
+ * consistency features, see {@link ITestS3GuardListConsistency}.
+ */
+public class ITestS3AInconsistency extends AbstractS3ATestBase {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    conf.setClass(S3_CLIENT_FACTORY_IMPL, InconsistentS3ClientFactory.class,
+        S3ClientFactory.class);
+    conf.set(FAIL_INJECT_INCONSISTENCY_KEY, DEFAULT_DELAY_KEY_SUBSTRING);
+    conf.setFloat(FAIL_INJECT_INCONSISTENCY_PROBABILITY, 1.0f);
+    conf.setLong(FAIL_INJECT_INCONSISTENCY_MSEC, DEFAULT_DELAY_KEY_MSEC);
+    return new S3AContract(conf);
+  }
+
+  @Test
+  public void testGetFileStatus() throws Exception {
+    S3AFileSystem fs = getFileSystem();
+
+    // 1. Make sure no ancestor dirs exist
+    Path dir = path("ancestor");
+    fs.delete(dir, true);
+    waitUntilDeleted(dir);
+
+    // 2. Create a descendant file, which implicitly creates ancestors
+    // This file has delayed visibility.
+    touch(getFileSystem(),
+        path("ancestor/file-" + DEFAULT_DELAY_KEY_SUBSTRING));
+
+    // 3. Assert expected behavior.  If S3Guard is enabled, we should be able
+    // to get status for ancestor.  If S3Guard is *not* enabled, S3A will
+    // fail to infer the existence of the ancestor since visibility of the
+    // child file is delayed, and its key prefix search will return nothing.
+    try {
+      FileStatus status = fs.getFileStatus(dir);
+      if (fs.hasMetadataStore()) {
+        assertTrue("Ancestor is dir", status.isDirectory());
+      } else {
+        fail("getFileStatus should fail due to delayed visibility.");
+      }
+    } catch (FileNotFoundException e) {
+      if (fs.hasMetadataStore()) {
+        fail("S3Guard failed to list parent of inconsistent child.");
+      }
+      LOG.info("File not found, as expected.");
+    }
+  }
+
+  private void waitUntilDeleted(final Path p) throws Exception {
+    LambdaTestUtils.eventually(30 * 1000, 1000,
+        new Callable<Void>() {
+          @Override
+          public Void call() throws Exception {
+            assertPathDoesNotExist("Dir should be deleted", p);
+            return null;
+          }
+        }
+    );
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java
index 59fcb05..869d64c 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java
@@ -22,10 +22,17 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.test.LambdaTestUtils;
+
+import com.amazonaws.services.s3.model.ObjectMetadata;
+import com.amazonaws.services.s3.model.PutObjectRequest;
+import com.amazonaws.services.s3.model.PutObjectResult;
 import org.junit.Test;
 
+import java.io.ByteArrayInputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.util.concurrent.Callable;
 
 /**
  * Tests of the S3A FileSystem which don't have a specific home and can share
@@ -55,6 +62,26 @@ public class ITestS3AMiscOperations extends AbstractS3ATestBase {
     createNonRecursive(new Path(parent, "fail"));
   }
 
+  @Test
+  public void testPutObjectDirect() throws Throwable {
+    final S3AFileSystem fs = getFileSystem();
+    ObjectMetadata metadata = fs.newObjectMetadata(-1);
+    metadata.setContentLength(-1);
+    Path path = path("putDirect");
+    final PutObjectRequest put = new PutObjectRequest(fs.getBucket(),
+        path.toUri().getPath(),
+        new ByteArrayInputStream("PUT".getBytes()),
+        metadata);
+    LambdaTestUtils.intercept(IllegalStateException.class,
+        new Callable<PutObjectResult>() {
+          @Override
+          public PutObjectResult call() throws Exception {
+            return fs.putObjectDirect(put);
+          }
+        });
+    assertPathDoesNotExist("put object was created", path);
+  }
+
   private FSDataOutputStream createNonRecursive(Path path) throws IOException {
     return getFileSystem().createNonRecursive(path, false, 4096,
         (short) 3, (short) 4096,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardCreate.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardCreate.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardCreate.java
new file mode 100644
index 0000000..dcc2538
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardCreate.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.s3guard.DirListingMetadata;
+import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
+import org.junit.Assume;
+import org.junit.Test;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
+
+/**
+ * Home for testing the creation of new files and directories with S3Guard
+ * enabled.
+ */
+public class ITestS3GuardCreate extends AbstractS3ATestBase {
+
+  /**
+   * Test that ancestor creation during S3AFileSystem#create() is properly
+   * accounted for in the MetadataStore.  This should be handled by the
+   * FileSystem, and be a FS contract test, but S3A does not handle ancestors on
+   * create(), so we need to take care in the S3Guard code to do the right
+   * thing.  This may change: See HADOOP-13221 for more detail.
+   */
+  @Test
+  public void testCreatePopulatesFileAncestors() throws Exception {
+    final S3AFileSystem fs = getFileSystem();
+    Assume.assumeTrue(fs.hasMetadataStore());
+    final MetadataStore ms = fs.getMetadataStore();
+    final Path parent = path("testCreatePopulatesFileAncestors");
+
+    try {
+      fs.mkdirs(parent);
+      final Path nestedFile = new Path(parent, "dir1/dir2/file4");
+      touch(fs, nestedFile);
+
+      DirListingMetadata list = ms.listChildren(parent);
+      assertFalse("MetadataStore falsely reports authoritative empty list",
+          list.isEmpty() == Tristate.TRUE);
+    } finally {
+      fs.delete(parent, true);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardEmptyDirs.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardEmptyDirs.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardEmptyDirs.java
new file mode 100644
index 0000000..fb6e370
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardEmptyDirs.java
@@ -0,0 +1,85 @@
+/*
+ * 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;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
+import org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore;
+import org.junit.Assume;
+import org.junit.Test;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
+
+/**
+ * Test logic around whether or not a directory is empty, with S3Guard enabled.
+ * The fact that S3AFileStatus has an isEmptyDirectory flag in it makes caching
+ * S3AFileStatus's really tricky, as the flag can change as a side effect of
+ * changes to other paths.
+ * After S3Guard is merged to trunk, we should try to remove the
+ * isEmptyDirectory flag from S3AFileStatus, or maintain it outside
+ * of the MetadataStore.
+ */
+public class ITestS3GuardEmptyDirs extends AbstractS3ATestBase {
+
+  @Test
+  public void testEmptyDirs() throws Exception {
+    S3AFileSystem fs = getFileSystem();
+    Assume.assumeTrue(fs.hasMetadataStore());
+    MetadataStore configuredMs = fs.getMetadataStore();
+    Path existingDir = path("existing-dir");
+    Path existingFile = path("existing-dir/existing-file");
+    try {
+      // 1. Simulate files already existing in the bucket before we started our
+      // cluster.  Temporarily disable the MetadataStore so it doesn't witness
+      // us creating these files.
+
+      fs.setMetadataStore(new NullMetadataStore());
+      assertTrue(fs.mkdirs(existingDir));
+      touch(fs, existingFile);
+
+
+      // 2. Simulate (from MetadataStore's perspective) starting our cluster and
+      // creating a file in an existing directory.
+      fs.setMetadataStore(configuredMs);  // "start cluster"
+      Path newFile = path("existing-dir/new-file");
+      touch(fs, newFile);
+
+      S3AFileStatus status = fs.innerGetFileStatus(existingDir, true);
+      assertEquals("Should not be empty dir", Tristate.FALSE,
+          status.isEmptyDirectory());
+
+      // 3. Assert that removing the only file the MetadataStore witnessed
+      // being created doesn't cause it to think the directory is now empty.
+      fs.delete(newFile, false);
+      status = fs.innerGetFileStatus(existingDir, true);
+      assertEquals("Should not be empty dir", Tristate.FALSE,
+          status.isEmptyDirectory());
+
+      // 4. Assert that removing the final file, that existed "before"
+      // MetadataStore started, *does* cause the directory to be marked empty.
+      fs.delete(existingFile, false);
+      status = fs.innerGetFileStatus(existingDir, true);
+      assertEquals("Should be empty dir now", Tristate.TRUE,
+          status.isEmptyDirectory());
+    } finally {
+      configuredMs.forgetMetadata(existingFile);
+      configuredMs.forgetMetadata(existingDir);
+    }
+  }
+}


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


[8/8] hadoop git commit: HADOOP-13345 S3Guard: Improved Consistency for S3A. Contributed by: Chris Nauroth, Aaron Fabbri, Mingliang Liu, Lei (Eddy) Xu, Sean Mackrory, Steve Loughran and others.

Posted by st...@apache.org.
HADOOP-13345 S3Guard: Improved Consistency for S3A.
Contributed by: Chris Nauroth, Aaron Fabbri, Mingliang Liu, Lei (Eddy) Xu,
Sean Mackrory, Steve Loughran and others.

(cherry-picked from/based on commit 621b43e254afaff708cd6fc4698b29628f6abc33)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/a1afc6aa
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a1afc6aa
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a1afc6aa

Branch: refs/heads/branch-2
Commit: a1afc6aa59188d32133bfa0367ff11219428d568
Parents: a641bce
Author: Steve Loughran <st...@apache.org>
Authored: Fri Sep 29 11:29:22 2017 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Sep 29 11:29:22 2017 +0100

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/pom.xml     |    5 +
 .../hadoop-common/src/main/bin/hadoop           |    3 +
 .../apache/hadoop/fs/AbstractFileSystem.java    |    8 +
 .../java/org/apache/hadoop/fs/FileContext.java  |    9 +
 .../src/main/resources/core-default.xml         |  108 ++
 .../fs/contract/AbstractContractRenameTest.java |   63 ++
 .../org/apache/hadoop/test/LambdaTestUtils.java |  112 ++
 hadoop-project/pom.xml                          |   13 +
 .../hadoop-aws/dev-support/findbugs-exclude.xml |    6 +
 hadoop-tools/hadoop-aws/pom.xml                 |  147 +++
 .../org/apache/hadoop/fs/s3a/Constants.java     |  133 ++-
 .../hadoop/fs/s3a/DefaultS3ClientFactory.java   |  233 ++++
 .../fs/s3a/InconsistentAmazonS3Client.java      |  434 ++++++++
 .../fs/s3a/InconsistentS3ClientFactory.java     |   40 +
 .../java/org/apache/hadoop/fs/s3a/Listing.java  |  263 ++++-
 .../hadoop/fs/s3a/S3ABlockOutputStream.java     |   17 +-
 .../org/apache/hadoop/fs/s3a/S3AFileStatus.java |   45 +-
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java |  572 ++++++++--
 .../hadoop/fs/s3a/S3AInstrumentation.java       |   81 +-
 .../apache/hadoop/fs/s3a/S3AOutputStream.java   |   14 +-
 .../java/org/apache/hadoop/fs/s3a/S3AUtils.java |   36 +-
 .../apache/hadoop/fs/s3a/S3ClientFactory.java   |  195 +---
 .../org/apache/hadoop/fs/s3a/Statistic.java     |   13 +-
 .../java/org/apache/hadoop/fs/s3a/Tristate.java |   32 +
 .../org/apache/hadoop/fs/s3a/UploadInfo.java    |   43 +
 .../fs/s3a/s3guard/DescendantsIterator.java     |  142 +++
 .../fs/s3a/s3guard/DirListingMetadata.java      |  322 ++++++
 .../fs/s3a/s3guard/DynamoDBClientFactory.java   |  132 +++
 .../fs/s3a/s3guard/DynamoDBMetadataStore.java   | 1010 ++++++++++++++++++
 .../fs/s3a/s3guard/LocalMetadataStore.java      |  435 ++++++++
 .../hadoop/fs/s3a/s3guard/LruHashMap.java       |   50 +
 .../hadoop/fs/s3a/s3guard/MetadataStore.java    |  221 ++++
 .../s3guard/MetadataStoreListFilesIterator.java |  169 +++
 .../fs/s3a/s3guard/NullMetadataStore.java       |  104 ++
 .../hadoop/fs/s3a/s3guard/PathMetadata.java     |  143 +++
 .../PathMetadataDynamoDBTranslation.java        |  304 ++++++
 .../apache/hadoop/fs/s3a/s3guard/S3Guard.java   |  463 ++++++++
 .../hadoop/fs/s3a/s3guard/S3GuardTool.java      |  924 ++++++++++++++++
 .../hadoop/fs/s3a/s3guard/package-info.java     |   30 +
 .../hadoop/fs/s3native/S3xLoginHelper.java      |    4 +
 .../src/site/markdown/tools/hadoop-aws/index.md |    3 +-
 .../site/markdown/tools/hadoop-aws/s3guard.md   |  610 +++++++++++
 .../site/markdown/tools/hadoop-aws/testing.md   |  288 ++++-
 .../fs/contract/s3a/ITestS3AContractCreate.java |   14 +
 .../fs/contract/s3a/ITestS3AContractDelete.java |   14 +
 .../fs/contract/s3a/ITestS3AContractDistCp.java |    7 +
 .../s3a/ITestS3AContractGetFileStatus.java      |    4 +
 .../fs/contract/s3a/ITestS3AContractMkdir.java  |   14 +
 .../fs/contract/s3a/ITestS3AContractOpen.java   |   14 +
 .../fs/contract/s3a/ITestS3AContractRename.java |   13 +
 .../contract/s3a/ITestS3AContractRootDir.java   |   14 +
 .../fs/contract/s3a/ITestS3AContractSeek.java   |   14 +
 .../hadoop/fs/s3a/AbstractS3AMockTest.java      |    9 +-
 .../hadoop/fs/s3a/AbstractS3ATestBase.java      |   26 +-
 .../fs/s3a/ITestS3AAWSCredentialsProvider.java  |    4 +
 .../hadoop/fs/s3a/ITestS3AConfiguration.java    |    3 +-
 .../hadoop/fs/s3a/ITestS3ACredentialsInURL.java |   11 +-
 .../hadoop/fs/s3a/ITestS3ADelayedFNF.java       |   62 ++
 .../hadoop/fs/s3a/ITestS3AEmptyDirectory.java   |   83 ++
 .../hadoop/fs/s3a/ITestS3AEncryptionSSEC.java   |  312 +++---
 .../fs/s3a/ITestS3AFileOperationCost.java       |   40 +-
 .../fs/s3a/ITestS3AFileSystemContract.java      |    3 +
 .../hadoop/fs/s3a/ITestS3AInconsistency.java    |  100 ++
 .../hadoop/fs/s3a/ITestS3AMiscOperations.java   |   27 +
 .../hadoop/fs/s3a/ITestS3GuardCreate.java       |   61 ++
 .../hadoop/fs/s3a/ITestS3GuardEmptyDirs.java    |   85 ++
 .../fs/s3a/ITestS3GuardListConsistency.java     |  544 ++++++++++
 .../hadoop/fs/s3a/ITestS3GuardWriteBack.java    |  141 +++
 .../hadoop/fs/s3a/MockS3ClientFactory.java      |    3 +
 .../apache/hadoop/fs/s3a/S3ATestConstants.java  |   12 +
 .../org/apache/hadoop/fs/s3a/S3ATestUtils.java  |  197 +++-
 .../org/apache/hadoop/fs/s3a/TestListing.java   |  118 ++
 .../ITestS3AFileContextStatistics.java          |    4 +-
 .../s3a/fileContext/ITestS3AFileContextURI.java |   19 +-
 .../fs/s3a/s3guard/AbstractMSContract.java      |   33 +
 .../s3guard/AbstractS3GuardToolTestBase.java    |  161 +++
 .../s3a/s3guard/DynamoDBLocalClientFactory.java |  157 +++
 .../s3a/s3guard/ITestS3GuardConcurrentOps.java  |  160 +++
 .../s3a/s3guard/ITestS3GuardToolDynamoDB.java   |  134 +++
 .../fs/s3a/s3guard/ITestS3GuardToolLocal.java   |  149 +++
 .../fs/s3a/s3guard/MetadataStoreTestBase.java   |  887 +++++++++++++++
 .../fs/s3a/s3guard/TestDirListingMetadata.java  |  303 ++++++
 .../s3a/s3guard/TestDynamoDBMetadataStore.java  |  594 ++++++++++
 .../fs/s3a/s3guard/TestLocalMetadataStore.java  |  140 +++
 .../fs/s3a/s3guard/TestNullMetadataStore.java   |   58 +
 .../TestPathMetadataDynamoDBTranslation.java    |  238 +++++
 .../hadoop/fs/s3a/s3guard/TestS3Guard.java      |   93 ++
 .../AbstractITestS3AMetadataStoreScale.java     |  250 +++++
 .../fs/s3a/scale/AbstractSTestS3AHugeFiles.java |   13 +-
 .../scale/ITestDynamoDBMetadataStoreScale.java  |   48 +
 .../s3a/scale/ITestLocalMetadataStoreScale.java |   37 +
 .../fs/s3a/scale/ITestS3AConcurrentOps.java     |    3 +-
 .../fs/s3a/scale/ITestS3ACreatePerformance.java |   86 ++
 .../s3a/scale/ITestS3ADirectoryPerformance.java |    5 +-
 .../scale/ITestS3AInputStreamPerformance.java   |    4 +-
 .../hadoop/fs/s3a/scale/S3AScaleTestBase.java   |    2 +-
 .../hadoop-aws/src/test/resources/core-site.xml |   26 +
 .../src/test/resources/log4j.properties         |   15 +-
 98 files changed, 13007 insertions(+), 540 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-common-project/hadoop-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/pom.xml b/hadoop-common-project/hadoop-common/pom.xml
index 8972e71..6b4ace5 100644
--- a/hadoop-common-project/hadoop-common/pom.xml
+++ b/hadoop-common-project/hadoop-common/pom.xml
@@ -163,6 +163,11 @@
       <scope>compile</scope>
     </dependency>
     <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-lang3</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-api</artifactId>
       <scope>compile</scope>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-common-project/hadoop-common/src/main/bin/hadoop
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/bin/hadoop b/hadoop-common-project/hadoop-common/src/main/bin/hadoop
index 1575996..74bf2d9 100755
--- a/hadoop-common-project/hadoop-common/src/main/bin/hadoop
+++ b/hadoop-common-project/hadoop-common/src/main/bin/hadoop
@@ -134,6 +134,9 @@ case $COMMAND in
         echo $CLASSPATH
         exit
       fi
+    elif [ "$COMMAND" = "s3guard" ] ; then
+      CLASS=org.apache.hadoop.fs.s3a.s3guard.S3GuardTool
+      CLASSPATH=${CLASSPATH}:${TOOL_PATH}
     elif [[ "$COMMAND" = -*  ]] ; then
         # class and package names cannot begin with a -
         echo "Error: No command named \`$COMMAND' was found. Perhaps you meant \`hadoop ${COMMAND#-}'"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java
index 9bea8f9..df14ee8 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.fs;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.ArrayList;
@@ -132,6 +133,13 @@ public abstract class AbstractFileSystem {
         CONSTRUCTOR_CACHE.put(theClass, meth);
       }
       result = meth.newInstance(uri, conf);
+    } catch (InvocationTargetException e) {
+      Throwable cause = e.getCause();
+      if (cause instanceof RuntimeException) {
+        throw (RuntimeException) cause;
+      } else {
+        throw new RuntimeException(cause);
+      }
     } catch (Exception e) {
       throw new RuntimeException(e);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
index 341d11f..039dbad 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
@@ -342,6 +342,15 @@ public class FileContext {
           return AbstractFileSystem.get(uri, conf);
         }
       });
+    } catch (RuntimeException ex) {
+      // RTEs can wrap other exceptions; if there is an IOException inner,
+      // throw it direct.
+      Throwable cause = ex.getCause();
+      if (cause instanceof IOException) {
+        throw (IOException) cause;
+      } else {
+        throw ex;
+      }
     } catch (InterruptedException ex) {
       LOG.error(ex.toString());
       throw new IOException("Failed to get the AbstractFileSystem for path: "

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
----------------------------------------------------------------------
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 041a912..ed5a3a5 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
@@ -1312,12 +1312,120 @@
 </property>
 
 <property>
+    <name>fs.s3a.metadatastore.authoritative</name>
+    <value>false</value>
+    <description>
+        When true, allow MetadataStore implementations to act as source of
+        truth for getting file status and directory listings.  Even if this
+        is set to true, MetadataStore implementations may choose not to
+        return authoritative results.  If the configured MetadataStore does
+        not support being authoritative, this setting will have no effect.
+    </description>
+</property>
+
+<property>
+    <name>fs.s3a.metadatastore.impl</name>
+    <value>org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore</value>
+    <description>
+        Fully-qualified name of the class that implements the MetadataStore
+        to be used by s3a.  The default class, NullMetadataStore, has no
+        effect: s3a will continue to treat the backing S3 service as the one
+        and only source of truth for file and directory metadata.
+    </description>
+</property>
+
+<property>
+    <name>fs.s3a.s3guard.cli.prune.age</name>
+    <value>86400000</value>
+    <description>
+        Default age (in milliseconds) after which to prune metadata from the
+        metadatastore when the prune command is run.  Can be overridden on the
+        command-line.
+    </description>
+</property>
+
+
+<property>
   <name>fs.s3a.impl</name>
   <value>org.apache.hadoop.fs.s3a.S3AFileSystem</value>
   <description>The implementation class of the S3A Filesystem</description>
 </property>
 
 <property>
+  <name>fs.s3a.s3guard.ddb.region</name>
+  <value></value>
+  <description>
+    AWS DynamoDB region to connect to. An up-to-date list is
+    provided in the AWS Documentation: regions and endpoints. Without this
+    property, the S3Guard will operate table in the associated S3 bucket region.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.s3guard.ddb.table</name>
+  <value></value>
+  <description>
+    The DynamoDB table name to operate. Without this property, the respective
+    S3 bucket name will be used.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.s3guard.ddb.table.create</name>
+  <value>false</value>
+  <description>
+    If true, the S3A client will create the table if it does not already exist.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.s3guard.ddb.table.capacity.read</name>
+  <value>500</value>
+  <description>
+    Provisioned throughput requirements for read operations in terms of capacity
+    units for the DynamoDB table.  This config value will only be used when
+    creating a new DynamoDB table, though later you can manually provision by
+    increasing or decreasing read capacity as needed for existing tables.
+    See DynamoDB documents for more information.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.s3guard.ddb.table.capacity.write</name>
+  <value>100</value>
+  <description>
+    Provisioned throughput requirements for write operations in terms of
+    capacity units for the DynamoDB table.  Refer to related config
+    fs.s3a.s3guard.ddb.table.capacity.read before usage.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.s3guard.ddb.max.retries</name>
+  <value>9</value>
+    <description>
+      Max retries on batched DynamoDB operations before giving up and
+      throwing an IOException.  Each retry is delayed with an exponential
+      backoff timer which starts at 100 milliseconds and approximately
+      doubles each time.  The minimum wait before throwing an exception is
+      sum(100, 200, 400, 800, .. 100*2^N-1 ) == 100 * ((2^N)-1)
+      So N = 9 yields at least 51.1 seconds (51,100) milliseconds of blocking
+      before throwing an IOException.
+    </description>
+</property>
+
+<property>
+  <name>fs.s3a.s3guard.ddb.background.sleep</name>
+  <value>25</value>
+  <description>
+    Length (in milliseconds) of pause between each batch of deletes when
+    pruning metadata.  Prevents prune operations (which can typically be low
+    priority background operations) from overly interfering with other I/O
+    operations.
+  </description>
+</property>
+
+<property>
   <name>fs.AbstractFileSystem.s3a.impl</name>
   <value>org.apache.hadoop.fs.s3a.S3A</value>
   <description>The implementation class of the S3A AbstractFileSystem.</description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRenameTest.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRenameTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRenameTest.java
index b0dcb93..b6d0a49 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
@@ -222,4 +222,67 @@ public abstract class AbstractContractRenameTest extends
     assertPathDoesNotExist("not deleted",
         new Path(srcDir, "source.txt"));
   }
+
+  /**
+   * Test that after renaming, the nested subdirectory is moved along with all
+   * its ancestors.
+   */
+  @Test
+  public void testRenamePopulatesDirectoryAncestors() throws IOException {
+    final FileSystem fs = getFileSystem();
+    final Path src = path("testRenamePopulatesDirectoryAncestors/source");
+    fs.mkdirs(src);
+    final String nestedDir = "/dir1/dir2/dir3/dir4";
+    fs.mkdirs(path(src + nestedDir));
+
+    Path dst = path("testRenamePopulatesDirectoryAncestorsNew");
+
+    fs.rename(src, dst);
+    validateAncestorsMoved(src, dst, nestedDir);
+  }
+
+  /**
+   * Test that after renaming, the nested file is moved along with all its
+   * ancestors. It is similar to {@link #testRenamePopulatesDirectoryAncestors}.
+   */
+  @Test
+  public void testRenamePopulatesFileAncestors() throws IOException {
+    final FileSystem fs = getFileSystem();
+    final Path src = path("testRenamePopulatesFileAncestors/source");
+    fs.mkdirs(src);
+    final String nestedFile = "/dir1/dir2/dir3/file4";
+    byte[] srcDataset = dataset(256, 'a', 'z');
+    writeDataset(fs, path(src + nestedFile), srcDataset, srcDataset.length,
+        1024, false);
+
+    Path dst = path("testRenamePopulatesFileAncestorsNew");
+
+    fs.rename(src, dst);
+    validateAncestorsMoved(src, dst, nestedFile);
+  }
+
+  /**
+   * Validate that the nested path and its ancestors should have been moved.
+   *
+   * @param src the source root to move
+   * @param dst the destination root to move
+   * @param nestedPath the nested path to move
+   */
+  private void validateAncestorsMoved(Path src, Path dst, String nestedPath)
+      throws IOException {
+    assertIsDirectory(dst);
+    assertPathDoesNotExist("src path should not exist", path(src + nestedPath));
+    assertPathExists("dst path should exist", path(dst + nestedPath));
+
+    Path path = new Path(nestedPath).getParent();
+    while (path != null && !path.isRoot()) {
+      final Path parentSrc = path(src + path.toString());
+      assertPathDoesNotExist(parentSrc + " is not deleted", parentSrc);
+      final Path parentDst = path(dst + path.toString());
+      assertPathExists(parentDst + " should exist after rename", parentDst);
+      assertIsDirectory(parentDst);
+      path = path.getParent();
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/LambdaTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/LambdaTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/LambdaTestUtils.java
index f5be132..1f906be 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/LambdaTestUtils.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/LambdaTestUtils.java
@@ -278,6 +278,23 @@ public final class LambdaTestUtils {
   }
 
   /**
+   * Variant of {@link #eventually(int, Callable, Callable)} method for
+   * void lambda expressions.
+   * @param timeoutMillis timeout in milliseconds.
+   * Can be zero, in which case only one attempt is made before failing.
+   * @param eval expression to evaluate
+   * @param retry retry interval generator
+   * @throws Exception the last exception thrown before timeout was triggered
+   * @throws FailFastException if raised -without any retry attempt.
+   * @throws InterruptedException if interrupted during the sleep operation.
+   */
+  public static void eventually(int timeoutMillis,
+      VoidCallable eval,
+      Callable<Integer> retry) throws Exception {
+    eventually(timeoutMillis, new VoidCaller(eval), retry);
+  }
+
+  /**
    * Simplified {@link #eventually(int, Callable, Callable)} method
    * with a fixed interval.
    * <p>
@@ -306,6 +323,25 @@ public final class LambdaTestUtils {
   }
 
   /**
+   /**
+   * Variant of {@link #eventually(int, int, Callable)} method for
+   * void lambda expressions.
+   * @param timeoutMillis timeout in milliseconds.
+   * Can be zero, in which case only one attempt is made before failing.
+   * @param intervalMillis interval in milliseconds
+   * @param eval expression to evaluate
+   * @throws Exception the last exception thrown before timeout was triggered
+   * @throws FailFastException if raised -without any retry attempt.
+   * @throws InterruptedException if interrupted during the sleep operation.
+   */
+  public static void eventually(int timeoutMillis,
+      int intervalMillis,
+      VoidCallable eval) throws Exception {
+    eventually(timeoutMillis, eval,
+        new FixedRetryInterval(intervalMillis));
+  }
+
+  /**
    * Intercept an exception; throw an {@code AssertionError} if one not raised.
    * The caught exception is rethrown if it is of the wrong class or
    * does not contain the text defined in {@code contained}.
@@ -348,6 +384,32 @@ public final class LambdaTestUtils {
   }
 
   /**
+   * Variant of {@link #intercept(Class, Callable)} to simplify void
+   * invocations.
+   * @param clazz class of exception; the raised exception must be this class
+   * <i>or a subclass</i>.
+   * @param eval expression to eval
+   * @param <E> exception class
+   * @return the caught exception if it was of the expected type
+   * @throws Exception any other exception raised
+   * @throws AssertionError if the evaluation call didn't raise an exception.
+   */
+  public static <E extends Throwable> E intercept(
+      Class<E> clazz,
+      VoidCallable eval)
+      throws Exception {
+    try {
+      eval.call();
+      throw new AssertionError("Expected an exception");
+    } catch (Throwable e) {
+      if (clazz.isAssignableFrom(e.getClass())) {
+        return (E)e;
+      }
+      throw e;
+    }
+  }
+
+  /**
    * Intercept an exception; throw an {@code AssertionError} if one not raised.
    * The caught exception is rethrown if it is of the wrong class or
    * does not contain the text defined in {@code contained}.
@@ -388,6 +450,29 @@ public final class LambdaTestUtils {
   }
 
   /**
+   * Variant of {@link #intercept(Class, Callable)} to simplify void
+   * invocations.
+   * @param clazz class of exception; the raised exception must be this class
+   * <i>or a subclass</i>.
+   * @param contained string which must be in the {@code toString()} value
+   * of the exception
+   * @param eval expression to eval
+   * @param <E> exception class
+   * @return the caught exception if it was of the expected type
+   * @throws Exception any other exception raised
+   * @throws AssertionError if the evaluation call didn't raise an exception.
+   */
+  public static <E extends Throwable> E intercept(
+      Class<E> clazz,
+      String contained,
+      VoidCallable eval)
+      throws Exception {
+    E ex = intercept(clazz, eval);
+    GenericTestUtils.assertExceptionContains(contained, ex);
+    return ex;
+  }
+
+  /**
    * Robust string converter for exception messages; if the {@code toString()}
    * method throws an exception then that exception is caught and logged,
    * then a simple string of the classname logged.
@@ -547,4 +632,31 @@ public final class LambdaTestUtils {
       return new FailFastException(String.format(format, args));
     }
   }
+
+  /**
+   * A simple interface for lambdas, which returns nothing; this exists
+   * to simplify lambda tests on operations with no return value.
+   */
+  public interface VoidCallable {
+    void call() throws Exception;
+  }
+
+  /**
+   * Bridge class to make {@link VoidCallable} something to use in anything
+   * which takes an {@link Callable}.
+   */
+  public static class VoidCaller implements Callable<Void> {
+    private final VoidCallable callback;
+
+    public VoidCaller(VoidCallable callback) {
+      this.callback = callback;
+    }
+
+    @Override
+    public Void call() throws Exception {
+      callback.call();
+      return null;
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 62e9fb4..e41c4c1 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -766,6 +766,11 @@
         <version>1.6</version>
       </dependency>
       <dependency>
+        <groupId>org.apache.commons</groupId>
+        <artifactId>commons-lang3</artifactId>
+        <version>3.4</version>
+      </dependency>
+      <dependency>
         <groupId>org.slf4j</groupId>
         <artifactId>slf4j-api</artifactId>
         <version>${slf4j.version}</version>
@@ -1509,4 +1514,12 @@
       </build>
     </profile>
   </profiles>
+
+  <repositories>
+    <repository>
+      <id>dynamodb-local-oregon</id>
+      <name>DynamoDB Local Release Repository</name>
+      <url>https://s3-us-west-2.amazonaws.com/dynamodb-local/release</url>
+    </repository>
+  </repositories>
 </project>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml b/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml
index ffb0a79..82ec16e 100644
--- a/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml
+++ b/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml
@@ -26,4 +26,10 @@
   <Match>
     <Class name="org.apache.hadoop.fs.s3.INode" />
   </Match>
+  <!-- Redundant null check makes code clearer, future-proof here. -->
+  <Match>
+    <Class name="org.apache.hadoop.fs.s3a.S3AFileSystem" />
+    <Method name="s3Exists" />
+    <Bug pattern="RCN_REDUNDANT_NULLCHECK_OF_NONNULL_VALUE" />
+  </Match>
 </FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml
index 59b0c82..aa7403b 100644
--- a/hadoop-tools/hadoop-aws/pom.xml
+++ b/hadoop-tools/hadoop-aws/pom.xml
@@ -36,6 +36,7 @@
     <downloadSources>true</downloadSources>
     <hadoop.tmp.dir>${project.build.directory}/test</hadoop.tmp.dir>
 
+    <dynamodb.local.version>1.11.86</dynamodb.local.version>
     <!-- are scale tests enabled ? -->
     <fs.s3a.scale.test.enabled>unset</fs.s3a.scale.test.enabled>
     <!-- Size in MB of huge files. -->
@@ -44,6 +45,11 @@
     <fs.s3a.scale.test.huge.partitionsize>unset</fs.s3a.scale.test.huge.partitionsize>
     <!-- Timeout in seconds for scale tests.-->
     <fs.s3a.scale.test.timeout>3600</fs.s3a.scale.test.timeout>
+    <!-- are scale tests enabled ? -->
+    <fs.s3a.s3guard.test.enabled>false</fs.s3a.s3guard.test.enabled>
+    <fs.s3a.s3guard.test.authoritative>false</fs.s3a.s3guard.test.authoritative>
+    <fs.s3a.s3guard.test.implementation>local</fs.s3a.s3guard.test.implementation>
+
   </properties>
 
   <profiles>
@@ -164,6 +170,11 @@
                     <fs.s3a.scale.test.huge.filesize>${fs.s3a.scale.test.huge.filesize}</fs.s3a.scale.test.huge.filesize>
                     <fs.s3a.scale.test.huge.huge.partitionsize>${fs.s3a.scale.test.huge.partitionsize}</fs.s3a.scale.test.huge.huge.partitionsize>
                     <fs.s3a.scale.test.timeout>${fs.s3a.scale.test.timeout}</fs.s3a.scale.test.timeout>
+                    <!-- S3Guard -->
+                    <fs.s3a.s3guard.test.enabled>${fs.s3a.s3guard.test.enabled}</fs.s3a.s3guard.test.enabled>
+                    <fs.s3a.s3guard.test.authoritative>${fs.s3a.s3guard.test.authoritative}</fs.s3a.s3guard.test.authoritative>
+                    <fs.s3a.s3guard.test.implementation>${fs.s3a.s3guard.test.implementation}</fs.s3a.s3guard.test.implementation>
+
                   </systemPropertyVariables>
                   <!-- Some tests cannot run in parallel.  Tests that cover -->
                   <!-- access to the root directory must run in isolation -->
@@ -206,6 +217,10 @@
                     <fs.s3a.scale.test.huge.filesize>${fs.s3a.scale.test.huge.filesize}</fs.s3a.scale.test.huge.filesize>
                     <fs.s3a.scale.test.huge.huge.partitionsize>${fs.s3a.scale.test.huge.partitionsize}</fs.s3a.scale.test.huge.huge.partitionsize>
                     <fs.s3a.scale.test.timeout>${fs.s3a.scale.test.timeout}</fs.s3a.scale.test.timeout>
+                    <!-- S3Guard -->
+                    <fs.s3a.s3guard.test.enabled>${fs.s3a.s3guard.test.enabled}</fs.s3a.s3guard.test.enabled>
+                    <fs.s3a.s3guard.test.implementation>${fs.s3a.s3guard.test.implementation}</fs.s3a.s3guard.test.implementation>
+                    <fs.s3a.s3guard.test.authoritative>${fs.s3a.s3guard.test.authoritative}</fs.s3a.s3guard.test.authoritative>
                   </systemPropertyVariables>
                   <!-- Do a sequential run for tests that cannot handle -->
                   <!-- parallel execution. -->
@@ -249,6 +264,10 @@
                     <fs.s3a.scale.test.enabled>${fs.s3a.scale.test.enabled}</fs.s3a.scale.test.enabled>
                     <fs.s3a.scale.test.huge.filesize>${fs.s3a.scale.test.huge.filesize}</fs.s3a.scale.test.huge.filesize>
                     <fs.s3a.scale.test.timeout>${fs.s3a.scale.test.timeout}</fs.s3a.scale.test.timeout>
+                    <!-- S3Guard -->
+                    <fs.s3a.s3guard.test.enabled>${fs.s3a.s3guard.test.enabled}</fs.s3a.s3guard.test.enabled>
+                    <fs.s3a.s3guard.test.implementation>${fs.s3a.s3guard.test.implementation}</fs.s3a.s3guard.test.implementation>
+                    <fs.s3a.s3guard.test.authoritative>${fs.s3a.s3guard.test.authoritative}</fs.s3a.s3guard.test.authoritative>
                   </systemPropertyVariables>
                   <forkedProcessTimeoutInSeconds>${fs.s3a.scale.test.timeout}</forkedProcessTimeoutInSeconds>
                 </configuration>
@@ -271,6 +290,60 @@
         <fs.s3a.scale.test.enabled>true</fs.s3a.scale.test.enabled>
       </properties>
     </profile>
+
+    <!-- Turn on S3Guard tests-->
+    <profile>
+      <id>s3guard</id>
+      <activation>
+        <property>
+          <name>s3guard</name>
+        </property>
+      </activation>
+      <properties >
+        <fs.s3a.s3guard.test.enabled>true</fs.s3a.s3guard.test.enabled>
+      </properties>
+    </profile>
+
+    <!-- Switch to DynamoDB for S3Guard. Has no effect unless S3Guard is enabled -->
+    <profile>
+      <id>dynamo</id>
+      <activation>
+        <property>
+          <name>dynamo</name>
+        </property>
+      </activation>
+      <properties >
+        <fs.s3a.s3guard.test.implementation>dynamo</fs.s3a.s3guard.test.implementation>
+      </properties>
+    </profile>
+
+    <!-- Switch to DynamoDBLocal for S3Guard. Has no effect unless S3Guard is enabled -->
+    <profile>
+      <id>dynamodblocal</id>
+      <activation>
+        <property>
+          <name>dynamodblocal</name>
+        </property>
+      </activation>
+      <properties>
+        <fs.s3a.s3guard.test.implementation>dynamodblocal</fs.s3a.s3guard.test.implementation>
+      </properties>
+    </profile>
+
+    <!-- Switch S3Guard from Authoritative=false to true
+     Has no effect unless S3Guard is enabled -->
+    <profile>
+      <id>non-auth</id>
+      <activation>
+        <property>
+          <name>auth</name>
+        </property>
+      </activation>
+      <properties >
+        <fs.s3a.s3guard.test.authoritative>true</fs.s3a.s3guard.test.authoritative>
+      </properties>
+    </profile>
+
   </profiles>
 
   <build>
@@ -301,6 +374,33 @@
           <forkedProcessTimeoutInSeconds>3600</forkedProcessTimeoutInSeconds>
         </configuration>
       </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>copy</id>
+            <phase>test-compile</phase>
+            <goals>
+              <goal>copy-dependencies</goal>
+            </goals>
+            <configuration>
+              <includeScope>test</includeScope>
+              <includeTypes>so,dll,dylib</includeTypes>
+              <outputDirectory>${project.build.directory}/native-libs</outputDirectory>
+            </configuration>
+          </execution>
+          <execution>
+            <phase>package</phase>
+            <goals>
+              <goal>copy-dependencies</goal>
+            </goals>
+            <configuration>
+              <outputDirectory>${project.build.directory}/lib</outputDirectory>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
   </build>
 
@@ -309,12 +409,32 @@
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-common</artifactId>
       <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <artifactId>servlet-api</artifactId>
+          <groupId>javax.servlet</groupId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.commons</groupId>
+          <artifactId>commons-lang3</artifactId>
+        </exclusion>
+      </exclusions>
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-common</artifactId>
       <scope>test</scope>
       <type>test-jar</type>
+      <exclusions>
+        <exclusion>
+          <artifactId>servlet-api</artifactId>
+          <groupId>javax.servlet</groupId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.commons</groupId>
+          <artifactId>commons-lang3</artifactId>
+        </exclusion>
+      </exclusions>
     </dependency>
     <dependency>
       <groupId>com.amazonaws</groupId>
@@ -322,6 +442,33 @@
       <scope>compile</scope>
     </dependency>
     <dependency>
+      <groupId>com.amazonaws</groupId>
+      <artifactId>DynamoDBLocal</artifactId>
+      <version>${dynamodb.local.version}</version>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.hamcrest</groupId>
+          <artifactId>hamcrest-core</artifactId>
+        </exclusion>
+<!--
+        <exclusion>
+          <groupId>org.eclipse.jetty</groupId>
+          <artifactId>jetty-http</artifactId>
+        </exclusion>
+-->
+        <exclusion>
+          <groupId>org.apache.commons</groupId>
+          <artifactId>commons-lang3</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-lang3</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
index 091e691..c3fc034 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
@@ -272,6 +272,11 @@ public final class Constants {
 
   public static final String USER_AGENT_PREFIX = "fs.s3a.user.agent.prefix";
 
+  /** Whether or not to allow MetadataStore to be source of truth. */
+  public static final String METADATASTORE_AUTHORITATIVE =
+      "fs.s3a.metadatastore.authoritative";
+  public static final boolean DEFAULT_METADATASTORE_AUTHORITATIVE = false;
+
   /** read ahead buffer size to prevent connection re-establishments. */
   public static final String READAHEAD_RANGE = "fs.s3a.readahead.range";
   public static final long DEFAULT_READAHEAD_RANGE = 64 * 1024;
@@ -317,7 +322,7 @@ public final class Constants {
   @InterfaceStability.Unstable
   public static final Class<? extends S3ClientFactory>
       DEFAULT_S3_CLIENT_FACTORY_IMPL =
-          S3ClientFactory.DefaultS3ClientFactory.class;
+          DefaultS3ClientFactory.class;
 
   /**
    * Maximum number of partitions in a multipart upload: {@value}.
@@ -325,4 +330,130 @@ public final class Constants {
   @InterfaceAudience.Private
   public static final int MAX_MULTIPART_COUNT = 10000;
 
+  /**
+   * Classname of the S3A-specific output committer factory. This
+   * is what must be declared when attempting to use
+   */
+  @InterfaceStability.Unstable
+  public static final String S3A_OUTPUT_COMMITTER_FACTORY =
+      "org.apache.hadoop.fs.s3a.commit.S3AOutputCommitterFactory";
+
+  /* Constants. */
+  public static final String S3_METADATA_STORE_IMPL =
+      "fs.s3a.metadatastore.impl";
+
+  /** Minimum period of time (in milliseconds) to keep metadata (may only be
+   * applied when a prune command is manually run).
+   */
+  @InterfaceStability.Unstable
+  public static final String S3GUARD_CLI_PRUNE_AGE =
+      "fs.s3a.s3guard.cli.prune.age";
+
+  /**
+   * The region of the DynamoDB service.
+   *
+   * This config has no default value. If the user does not set this, the
+   * S3Guard will operate table in the associated S3 bucket region.
+   */
+  @InterfaceStability.Unstable
+  public static final String S3GUARD_DDB_REGION_KEY =
+      "fs.s3a.s3guard.ddb.region";
+
+  /**
+   * The DynamoDB table name to use.
+   *
+   * This config has no default value. If the user does not set this, the
+   * S3Guard implementation will use the respective S3 bucket name.
+   */
+  @InterfaceStability.Unstable
+  public static final String S3GUARD_DDB_TABLE_NAME_KEY =
+      "fs.s3a.s3guard.ddb.table";
+
+  /**
+   * Whether to create the DynamoDB table if the table does not exist.
+   */
+  @InterfaceStability.Unstable
+  public static final String S3GUARD_DDB_TABLE_CREATE_KEY =
+      "fs.s3a.s3guard.ddb.table.create";
+
+  @InterfaceStability.Unstable
+  public static final String S3GUARD_DDB_TABLE_CAPACITY_READ_KEY =
+      "fs.s3a.s3guard.ddb.table.capacity.read";
+  public static final long S3GUARD_DDB_TABLE_CAPACITY_READ_DEFAULT = 500;
+  @InterfaceStability.Unstable
+  public static final String S3GUARD_DDB_TABLE_CAPACITY_WRITE_KEY =
+      "fs.s3a.s3guard.ddb.table.capacity.write";
+  public static final long S3GUARD_DDB_TABLE_CAPACITY_WRITE_DEFAULT = 100;
+
+  /**
+   * The maximum put or delete requests per BatchWriteItem request.
+   *
+   * Refer to Amazon API reference for this limit.
+   */
+  public static final int S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT = 25;
+
+  @InterfaceStability.Unstable
+  public static final String S3GUARD_DDB_MAX_RETRIES =
+      "fs.s3a.s3guard.ddb.max.retries";
+  /**
+   * Max retries on batched DynamoDB operations before giving up and
+   * throwing an IOException.  Default is {@value}. See core-default.xml for
+   * more detail.
+   */
+  public static final int S3GUARD_DDB_MAX_RETRIES_DEFAULT = 9;
+
+  /**
+   * Period of time (in milliseconds) to sleep between batches of writes.
+   * Currently only applies to prune operations, as they are naturally a
+   * lower priority than other operations.
+   */
+  @InterfaceStability.Unstable
+  public static final String S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY =
+      "fs.s3a.s3guard.ddb.background.sleep";
+  public static final int S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_DEFAULT = 25;
+
+  /**
+   * V1 committer.
+   */
+  @InterfaceStability.Unstable
+  public static final String S3A_OUTPUT_COMMITTER_MRV1 =
+      "org.apache.hadoop.fs.s3a.commit.S3OutputCommitterMRv1";
+
+  /**
+   * The default "Null" metadata store: {@value}.
+   */
+  @InterfaceStability.Unstable
+  public static final String S3GUARD_METASTORE_NULL
+      = "org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore";
+
+  /**
+   * Use Local memory for the metadata: {@value}.
+   * This is not coherent across processes and must be used for testing only.
+   */
+  @InterfaceStability.Unstable
+  public static final String S3GUARD_METASTORE_LOCAL
+      = "org.apache.hadoop.fs.s3a.s3guard.LocalMetadataStore";
+
+  /**
+   * Use DynamoDB for the metadata: {@value}.
+   */
+  @InterfaceStability.Unstable
+  public static final String S3GUARD_METASTORE_DYNAMO
+      = "org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStore";
+
+  /**
+   * Inconsistency (visibility delay) injection settings.
+   */
+  @InterfaceStability.Unstable
+  public static final String FAIL_INJECT_INCONSISTENCY_KEY =
+      "fs.s3a.failinject.inconsistency.key.substring";
+
+  @InterfaceStability.Unstable
+  public static final String FAIL_INJECT_INCONSISTENCY_MSEC =
+      "fs.s3a.failinject.inconsistency.msec";
+
+  @InterfaceStability.Unstable
+  public static final String FAIL_INJECT_INCONSISTENCY_PROBABILITY =
+      "fs.s3a.failinject.inconsistency.probability";
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java
new file mode 100644
index 0000000..f33b25e
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java
@@ -0,0 +1,233 @@
+/*
+ * 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;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.Protocol;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.s3.AmazonS3;
+import com.amazonaws.services.s3.AmazonS3Client;
+import com.amazonaws.services.s3.S3ClientOptions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.util.VersionInfo;
+import org.slf4j.Logger;
+
+import java.io.IOException;
+import java.net.URI;
+
+import static org.apache.hadoop.fs.s3a.Constants.*;
+import static org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProviderSet;
+import static org.apache.hadoop.fs.s3a.S3AUtils.intOption;
+
+/**
+ * The default factory implementation, which calls the AWS SDK to configure
+ * and create an {@link AmazonS3Client} that communicates with the S3 service.
+ */
+public class DefaultS3ClientFactory extends Configured implements
+    S3ClientFactory {
+
+  protected static final Logger LOG = S3AFileSystem.LOG;
+
+  @Override
+  public AmazonS3 createS3Client(URI name) throws IOException {
+    Configuration conf = getConf();
+    AWSCredentialsProvider credentials =
+        createAWSCredentialProviderSet(name, conf);
+    final ClientConfiguration awsConf = createAwsConf(getConf());
+    AmazonS3 s3 = newAmazonS3Client(credentials, awsConf);
+    return createAmazonS3Client(s3, conf, credentials, awsConf);
+  }
+
+  /**
+   * Create a new {@link ClientConfiguration}.
+   * @param conf The Hadoop configuration
+   * @return new AWS client configuration
+   */
+  public static ClientConfiguration createAwsConf(Configuration conf) {
+    final ClientConfiguration awsConf = new ClientConfiguration();
+    initConnectionSettings(conf, awsConf);
+    initProxySupport(conf, awsConf);
+    initUserAgent(conf, awsConf);
+    return awsConf;
+  }
+
+  /**
+   * Wrapper around constructor for {@link AmazonS3} client.  Override this to
+   * provide an extended version of the client
+   * @param credentials credentials to use
+   * @param awsConf  AWS configuration
+   * @return  new AmazonS3 client
+   */
+  protected AmazonS3 newAmazonS3Client(
+      AWSCredentialsProvider credentials, ClientConfiguration awsConf) {
+    return new AmazonS3Client(credentials, awsConf);
+  }
+
+  /**
+   * Initializes all AWS SDK settings related to connection management.
+   *
+   * @param conf Hadoop configuration
+   * @param awsConf AWS SDK configuration
+   */
+  private static void initConnectionSettings(Configuration conf,
+      ClientConfiguration awsConf) {
+    awsConf.setMaxConnections(intOption(conf, MAXIMUM_CONNECTIONS,
+        DEFAULT_MAXIMUM_CONNECTIONS, 1));
+    boolean secureConnections = conf.getBoolean(SECURE_CONNECTIONS,
+        DEFAULT_SECURE_CONNECTIONS);
+    awsConf.setProtocol(secureConnections ?  Protocol.HTTPS : Protocol.HTTP);
+    awsConf.setMaxErrorRetry(intOption(conf, MAX_ERROR_RETRIES,
+        DEFAULT_MAX_ERROR_RETRIES, 0));
+    awsConf.setConnectionTimeout(intOption(conf, ESTABLISH_TIMEOUT,
+        DEFAULT_ESTABLISH_TIMEOUT, 0));
+    awsConf.setSocketTimeout(intOption(conf, SOCKET_TIMEOUT,
+        DEFAULT_SOCKET_TIMEOUT, 0));
+    int sockSendBuffer = intOption(conf, SOCKET_SEND_BUFFER,
+        DEFAULT_SOCKET_SEND_BUFFER, 2048);
+    int sockRecvBuffer = intOption(conf, SOCKET_RECV_BUFFER,
+        DEFAULT_SOCKET_RECV_BUFFER, 2048);
+    awsConf.setSocketBufferSizeHints(sockSendBuffer, sockRecvBuffer);
+    String signerOverride = conf.getTrimmed(SIGNING_ALGORITHM, "");
+    if (!signerOverride.isEmpty()) {
+      LOG.debug("Signer override = {}", signerOverride);
+      awsConf.setSignerOverride(signerOverride);
+    }
+  }
+
+  /**
+   * Initializes AWS SDK proxy support if configured.
+   *
+   * @param conf Hadoop configuration
+   * @param awsConf AWS SDK configuration
+   * @throws IllegalArgumentException if misconfigured
+   */
+  private static void initProxySupport(Configuration conf,
+      ClientConfiguration awsConf) throws IllegalArgumentException {
+    String proxyHost = conf.getTrimmed(PROXY_HOST, "");
+    int proxyPort = conf.getInt(PROXY_PORT, -1);
+    if (!proxyHost.isEmpty()) {
+      awsConf.setProxyHost(proxyHost);
+      if (proxyPort >= 0) {
+        awsConf.setProxyPort(proxyPort);
+      } else {
+        if (conf.getBoolean(SECURE_CONNECTIONS, DEFAULT_SECURE_CONNECTIONS)) {
+          LOG.warn("Proxy host set without port. Using HTTPS default 443");
+          awsConf.setProxyPort(443);
+        } else {
+          LOG.warn("Proxy host set without port. Using HTTP default 80");
+          awsConf.setProxyPort(80);
+        }
+      }
+      String proxyUsername = conf.getTrimmed(PROXY_USERNAME);
+      String proxyPassword = conf.getTrimmed(PROXY_PASSWORD);
+      if ((proxyUsername == null) != (proxyPassword == null)) {
+        String msg = "Proxy error: " + PROXY_USERNAME + " or " +
+            PROXY_PASSWORD + " set without the other.";
+        LOG.error(msg);
+        throw new IllegalArgumentException(msg);
+      }
+      awsConf.setProxyUsername(proxyUsername);
+      awsConf.setProxyPassword(proxyPassword);
+      awsConf.setProxyDomain(conf.getTrimmed(PROXY_DOMAIN));
+      awsConf.setProxyWorkstation(conf.getTrimmed(PROXY_WORKSTATION));
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Using proxy server {}:{} as user {} with password {} on " +
+                "domain {} as workstation {}", awsConf.getProxyHost(),
+            awsConf.getProxyPort(),
+            String.valueOf(awsConf.getProxyUsername()),
+            awsConf.getProxyPassword(), awsConf.getProxyDomain(),
+            awsConf.getProxyWorkstation());
+      }
+    } else if (proxyPort >= 0) {
+      String msg =
+          "Proxy error: " + PROXY_PORT + " set without " + PROXY_HOST;
+      LOG.error(msg);
+      throw new IllegalArgumentException(msg);
+    }
+  }
+
+  /**
+   * Initializes the User-Agent header to send in HTTP requests to the S3
+   * back-end.  We always include the Hadoop version number.  The user also
+   * may set an optional custom prefix to put in front of the Hadoop version
+   * number.  The AWS SDK interally appends its own information, which seems
+   * to include the AWS SDK version, OS and JVM version.
+   *
+   * @param conf Hadoop configuration
+   * @param awsConf AWS SDK configuration
+   */
+  private static void initUserAgent(Configuration conf,
+      ClientConfiguration awsConf) {
+    String userAgent = "Hadoop " + VersionInfo.getVersion();
+    String userAgentPrefix = conf.getTrimmed(USER_AGENT_PREFIX, "");
+    if (!userAgentPrefix.isEmpty()) {
+      userAgent = userAgentPrefix + ", " + userAgent;
+    }
+    LOG.debug("Using User-Agent: {}", userAgent);
+    awsConf.setUserAgentPrefix(userAgent);
+  }
+
+  /**
+   * Creates an {@link AmazonS3Client} from the established configuration.
+   *
+   * @param conf Hadoop configuration
+   * @param credentials AWS credentials
+   * @param awsConf AWS SDK configuration
+   * @return S3 client
+   * @throws IllegalArgumentException if misconfigured
+   */
+  private static AmazonS3 createAmazonS3Client(AmazonS3 s3, Configuration conf,
+      AWSCredentialsProvider credentials, ClientConfiguration awsConf)
+      throws IllegalArgumentException {
+    String endPoint = conf.getTrimmed(ENDPOINT, "");
+    if (!endPoint.isEmpty()) {
+      try {
+        s3.setEndpoint(endPoint);
+      } catch (IllegalArgumentException e) {
+        String msg = "Incorrect endpoint: "  + e.getMessage();
+        LOG.error(msg);
+        throw new IllegalArgumentException(msg, e);
+      }
+    }
+    enablePathStyleAccessIfRequired(s3, conf);
+    return s3;
+  }
+
+  /**
+   * Enables path-style access to S3 buckets if configured.  By default, the
+   * behavior is to use virtual hosted-style access with URIs of the form
+   * http://bucketname.s3.amazonaws.com.  Enabling path-style access and a
+   * region-specific endpoint switches the behavior to use URIs of the form
+   * http://s3-eu-west-1.amazonaws.com/bucketname.
+   *
+   * @param s3 S3 client
+   * @param conf Hadoop configuration
+   */
+  private static void enablePathStyleAccessIfRequired(AmazonS3 s3,
+      Configuration conf) {
+    final boolean pathStyleAccess = conf.getBoolean(PATH_STYLE_ACCESS, false);
+    if (pathStyleAccess) {
+      LOG.debug("Enabling path style access!");
+      s3.setS3ClientOptions(S3ClientOptions.builder()
+          .setPathStyleAccess(true)
+          .build());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentAmazonS3Client.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentAmazonS3Client.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentAmazonS3Client.java
new file mode 100644
index 0000000..5e9cb3f
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentAmazonS3Client.java
@@ -0,0 +1,434 @@
+/*
+ * 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;
+
+import com.amazonaws.AmazonClientException;
+import com.amazonaws.AmazonServiceException;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.s3.AmazonS3;
+import com.amazonaws.services.s3.AmazonS3Client;
+import com.amazonaws.services.s3.model.DeleteObjectRequest;
+import com.amazonaws.services.s3.model.DeleteObjectsRequest;
+import com.amazonaws.services.s3.model.DeleteObjectsResult;
+import com.amazonaws.services.s3.model.ListObjectsRequest;
+import com.amazonaws.services.s3.model.ObjectListing;
+import com.amazonaws.services.s3.model.PutObjectRequest;
+import com.amazonaws.services.s3.model.PutObjectResult;
+import com.amazonaws.services.s3.model.S3ObjectSummary;
+import com.google.common.base.Preconditions;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.hadoop.fs.s3a.Constants.*;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A wrapper around {@link com.amazonaws.services.s3.AmazonS3} that injects
+ * inconsistency and/or errors.  Used for testing S3Guard.
+ * Currently only delays listing visibility, not affecting GET.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class InconsistentAmazonS3Client extends AmazonS3Client {
+
+  /**
+   * Keys containing this substring will be subject to delayed visibility.
+   */
+  public static final String DEFAULT_DELAY_KEY_SUBSTRING = "DELAY_LISTING_ME";
+
+  /**
+   * How many seconds affected keys will be delayed from appearing in listing.
+   * This should probably be a config value.
+   */
+  public static final long DEFAULT_DELAY_KEY_MSEC = 5 * 1000;
+
+  public static final float DEFAULT_DELAY_KEY_PROBABILITY = 1.0f;
+
+  /** Special config value since we can't store empty strings in XML. */
+  public static final String MATCH_ALL_KEYS = "*";
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(InconsistentAmazonS3Client.class);
+
+  /** Empty string matches all keys. */
+  private String delayKeySubstring;
+
+  /** Probability to delay visibility of a matching key. */
+  private float delayKeyProbability;
+
+  /** Time in milliseconds to delay visibility of newly modified object. */
+  private long delayKeyMsec;
+
+  /**
+   * Composite of data we need to track about recently deleted objects:
+   * when it was deleted (same was with recently put objects) and the object
+   * summary (since we should keep returning it for sometime after its
+   * deletion).
+   */
+  private static class Delete {
+    private Long time;
+    private S3ObjectSummary summary;
+
+    Delete(Long time, S3ObjectSummary summary) {
+      this.time = time;
+      this.summary = summary;
+    }
+
+    public Long time() {
+      return time;
+    }
+
+    public S3ObjectSummary summary() {
+      return summary;
+    }
+  }
+
+  /** Map of key to delay -> time it was deleted + object summary (object
+   * summary is null for prefixes. */
+  private Map<String, Delete> delayedDeletes = new HashMap<>();
+
+  /** Map of key to delay -> time it was created. */
+  private Map<String, Long> delayedPutKeys = new HashMap<>();
+
+  public InconsistentAmazonS3Client(AWSCredentialsProvider credentials,
+      ClientConfiguration clientConfiguration, Configuration conf) {
+    super(credentials, clientConfiguration);
+    setupConfig(conf);
+  }
+
+  protected void setupConfig(Configuration conf) {
+
+    delayKeySubstring = conf.get(FAIL_INJECT_INCONSISTENCY_KEY,
+        DEFAULT_DELAY_KEY_SUBSTRING);
+    // "" is a substring of all strings, use it to match all keys.
+    if (delayKeySubstring.equals(MATCH_ALL_KEYS)) {
+      delayKeySubstring = "";
+    }
+    delayKeyProbability = conf.getFloat(FAIL_INJECT_INCONSISTENCY_PROBABILITY,
+        DEFAULT_DELAY_KEY_PROBABILITY);
+    delayKeyMsec = conf.getLong(FAIL_INJECT_INCONSISTENCY_MSEC,
+        DEFAULT_DELAY_KEY_MSEC);
+    LOG.info("Enabled with {} msec delay, substring {}, probability {}",
+        delayKeyMsec, delayKeySubstring, delayKeyProbability);
+  }
+
+  /**
+   * Clear all oustanding inconsistent keys.  After calling this function,
+   * listings should behave normally (no failure injection), until additional
+   * keys are matched for delay, e.g. via putObject(), deleteObject().
+   */
+  public void clearInconsistency() {
+    LOG.info("clearing all delayed puts / deletes");
+    delayedDeletes.clear();
+    delayedPutKeys.clear();
+  }
+
+  /**
+   * Convenience function for test code to cast from supertype.
+   * @param c supertype to cast from
+   * @return subtype, not null
+   * @throws Exception on error
+   */
+  public static InconsistentAmazonS3Client castFrom(AmazonS3 c) throws
+      Exception {
+    InconsistentAmazonS3Client ic = null;
+    if (c instanceof InconsistentAmazonS3Client) {
+      ic = (InconsistentAmazonS3Client) c;
+    }
+    Preconditions.checkNotNull(ic, "Not an instance of " +
+        "InconsistentAmazonS3Client");
+    return ic;
+  }
+
+  @Override
+  public DeleteObjectsResult deleteObjects(DeleteObjectsRequest
+      deleteObjectsRequest)
+      throws AmazonClientException, AmazonServiceException {
+    for (DeleteObjectsRequest.KeyVersion keyVersion :
+        deleteObjectsRequest.getKeys()) {
+      registerDeleteObject(keyVersion.getKey(), deleteObjectsRequest
+          .getBucketName());
+    }
+    return super.deleteObjects(deleteObjectsRequest);
+  }
+
+  @Override
+  public void deleteObject(DeleteObjectRequest deleteObjectRequest)
+      throws AmazonClientException, AmazonServiceException {
+    String key = deleteObjectRequest.getKey();
+    LOG.debug("key {}", key);
+    registerDeleteObject(key, deleteObjectRequest.getBucketName());
+    super.deleteObject(deleteObjectRequest);
+  }
+
+  /* We should only need to override this version of putObject() */
+  @Override
+  public PutObjectResult putObject(PutObjectRequest putObjectRequest)
+      throws AmazonClientException, AmazonServiceException {
+    LOG.debug("key {}", putObjectRequest.getKey());
+    registerPutObject(putObjectRequest);
+    return super.putObject(putObjectRequest);
+  }
+
+  /* We should only need to override this version of listObjects() */
+  @Override
+  public ObjectListing listObjects(ListObjectsRequest listObjectsRequest)
+      throws AmazonClientException, AmazonServiceException {
+    LOG.debug("prefix {}", listObjectsRequest.getPrefix());
+    ObjectListing listing = super.listObjects(listObjectsRequest);
+    listing = filterListObjects(listObjectsRequest, listing);
+    listing = restoreListObjects(listObjectsRequest, listing);
+    return listing;
+  }
+
+  private void addSummaryIfNotPresent(List<S3ObjectSummary> list,
+      S3ObjectSummary item) {
+    // Behavior of S3ObjectSummary
+    String key = item.getKey();
+    for (S3ObjectSummary member : list) {
+      if (member.getKey().equals(key)) {
+        return;
+      }
+    }
+    list.add(item);
+  }
+
+  /**
+   * Add prefix of child to given list.  The added prefix will be equal to
+   * ancestor plus one directory past ancestor.  e.g.:
+   * if ancestor is "/a/b/c" and child is "/a/b/c/d/e/file" then "a/b/c/d" is
+   * added to list.
+   * @param prefixes list to add to
+   * @param ancestor path we are listing in
+   * @param child full path to get prefix from
+   */
+  private void addPrefixIfNotPresent(List<String> prefixes, String ancestor,
+      String child) {
+    Path prefixCandidate = new Path(child).getParent();
+    Path ancestorPath = new Path(ancestor);
+    Preconditions.checkArgument(child.startsWith(ancestor), "%s does not " +
+        "start with %s", child, ancestor);
+    while (!prefixCandidate.isRoot()) {
+      Path nextParent = prefixCandidate.getParent();
+      if (nextParent.equals(ancestorPath)) {
+        String prefix = prefixCandidate.toString();
+        if (!prefixes.contains(prefix)) {
+          prefixes.add(prefix);
+        }
+        return;
+      }
+      prefixCandidate = nextParent;
+    }
+  }
+
+  /**
+   * Checks that the parent key is an ancestor of the child key.
+   * @param parent key that may be the parent.
+   * @param child key that may be the child.
+   * @param recursive if false, only return true for direct children.  If
+   *                  true, any descendant will count.
+   * @return true if parent is an ancestor of child
+   */
+  private boolean isDescendant(String parent, String child, boolean recursive) {
+    if (recursive) {
+      if (!parent.endsWith("/")) {
+        parent = parent + "/";
+      }
+      return child.startsWith(parent);
+    } else {
+      Path actualParentPath = new Path(child).getParent();
+      Path expectedParentPath = new Path(parent);
+      return actualParentPath.equals(expectedParentPath);
+    }
+  }
+
+  /**
+   * Simulate eventual consistency of delete for this list operation:  Any
+   * recently-deleted keys will be added.
+   * @param request List request
+   * @param rawListing listing returned from underlying S3
+   * @return listing with recently-deleted items restored
+   */
+  private ObjectListing restoreListObjects(ListObjectsRequest request,
+      ObjectListing rawListing) {
+    List<S3ObjectSummary> outputList = rawListing.getObjectSummaries();
+    List<String> outputPrefixes = rawListing.getCommonPrefixes();
+    // recursive list has no delimiter, returns everything that matches a
+    // prefix.
+    boolean recursiveObjectList = !("/".equals(request.getDelimiter()));
+
+    // Go through all deleted keys
+    for (String key : new HashSet<>(delayedDeletes.keySet())) {
+      Delete delete = delayedDeletes.get(key);
+      if (isKeyDelayed(delete.time(), key)) {
+        if (isDescendant(request.getPrefix(), key, recursiveObjectList)) {
+          if (delete.summary() != null) {
+            addSummaryIfNotPresent(outputList, delete.summary());
+          }
+        }
+        // Non-recursive list has delimiter: will return rolled-up prefixes for
+        // all keys that are not direct children
+        if (!recursiveObjectList) {
+          if (isDescendant(request.getPrefix(), key, true)) {
+            addPrefixIfNotPresent(outputPrefixes, request.getPrefix(), key);
+          }
+        }
+      } else {
+        // Clean up any expired entries
+        delayedDeletes.remove(key);
+      }
+    }
+
+    return new CustomObjectListing(rawListing, outputList, outputPrefixes);
+  }
+
+  private ObjectListing filterListObjects(ListObjectsRequest request,
+      ObjectListing rawListing) {
+
+    // Filter object listing
+    List<S3ObjectSummary> outputList = new ArrayList<>();
+    for (S3ObjectSummary s : rawListing.getObjectSummaries()) {
+      String key = s.getKey();
+      if (!isKeyDelayed(delayedPutKeys.get(key), key)) {
+        outputList.add(s);
+      }
+    }
+
+    // Filter prefixes (directories)
+    List<String> outputPrefixes = new ArrayList<>();
+    for (String key : rawListing.getCommonPrefixes()) {
+      if (!isKeyDelayed(delayedPutKeys.get(key), key)) {
+        outputPrefixes.add(key);
+      }
+    }
+
+    return new CustomObjectListing(rawListing, outputList, outputPrefixes);
+  }
+
+  private boolean isKeyDelayed(Long enqueueTime, String key) {
+    if (enqueueTime == null) {
+      LOG.debug("no delay for key {}", key);
+      return false;
+    }
+    long currentTime = System.currentTimeMillis();
+    long deadline = enqueueTime + delayKeyMsec;
+    if (currentTime >= deadline) {
+      delayedDeletes.remove(key);
+      LOG.debug("no longer delaying {}", key);
+      return false;
+    } else  {
+      LOG.info("delaying {}", key);
+      return true;
+    }
+  }
+
+  private void registerDeleteObject(String key, String bucket) {
+    if (shouldDelay(key)) {
+      // Record summary so we can add it back for some time post-deletion
+      S3ObjectSummary summary = null;
+      ObjectListing list = listObjects(bucket, key);
+      for (S3ObjectSummary result : list.getObjectSummaries()) {
+        if (result.getKey().equals(key)) {
+          summary = result;
+          break;
+        }
+      }
+      delayedDeletes.put(key, new Delete(System.currentTimeMillis(), summary));
+    }
+  }
+
+  private void registerPutObject(PutObjectRequest req) {
+    String key = req.getKey();
+    if (shouldDelay(key)) {
+      enqueueDelayedPut(key);
+    }
+  }
+
+  /**
+   * Should we delay listing visibility for this key?
+   * @param key key which is being put
+   * @return true if we should delay
+   */
+  private boolean shouldDelay(String key) {
+    boolean delay = key.contains(delayKeySubstring);
+    delay = delay && trueWithProbability(delayKeyProbability);
+    LOG.debug("{} -> {}", key, delay);
+    return delay;
+  }
+
+
+  private boolean trueWithProbability(float p) {
+    return Math.random() < p;
+  }
+
+  /**
+   * Record this key as something that should not become visible in
+   * listObject replies for a while, to simulate eventual list consistency.
+   * @param key key to delay visibility of
+   */
+  private void enqueueDelayedPut(String key) {
+    LOG.debug("delaying put of {}", key);
+    delayedPutKeys.put(key, System.currentTimeMillis());
+  }
+
+  /** Since ObjectListing is immutable, we just override it with wrapper. */
+  private static class CustomObjectListing extends ObjectListing {
+
+    private final List<S3ObjectSummary> customListing;
+    private final List<String> customPrefixes;
+
+    CustomObjectListing(ObjectListing rawListing,
+        List<S3ObjectSummary> customListing,
+        List<String> customPrefixes) {
+      super();
+      this.customListing = customListing;
+      this.customPrefixes = customPrefixes;
+
+      this.setBucketName(rawListing.getBucketName());
+      this.setCommonPrefixes(rawListing.getCommonPrefixes());
+      this.setDelimiter(rawListing.getDelimiter());
+      this.setEncodingType(rawListing.getEncodingType());
+      this.setMarker(rawListing.getMarker());
+      this.setMaxKeys(rawListing.getMaxKeys());
+      this.setNextMarker(rawListing.getNextMarker());
+      this.setPrefix(rawListing.getPrefix());
+      this.setTruncated(rawListing.isTruncated());
+    }
+
+    @Override
+    public List<S3ObjectSummary> getObjectSummaries() {
+      return customListing;
+    }
+
+    @Override
+    public List<String> getCommonPrefixes() {
+      return customPrefixes;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java
new file mode 100644
index 0000000..17d268b
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.s3.AmazonS3;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * S3 Client factory used for testing with eventual consistency fault injection.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class InconsistentS3ClientFactory extends DefaultS3ClientFactory {
+
+  @Override
+  protected AmazonS3 newAmazonS3Client(AWSCredentialsProvider credentials,
+      ClientConfiguration awsConf) {
+    LOG.warn("** FAILURE INJECTION ENABLED.  Do not run in production! **");
+    return new InconsistentAmazonS3Client(credentials, awsConf, getConf());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java
index 30d8e6f..8efa218 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java
@@ -22,18 +22,25 @@ import com.amazonaws.AmazonClientException;
 import com.amazonaws.services.s3.model.ListObjectsRequest;
 import com.amazonaws.services.s3.model.ObjectListing;
 import com.amazonaws.services.s3.model.S3ObjectSummary;
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.RemoteIterator;
+
+import com.google.common.base.Preconditions;
 import org.slf4j.Logger;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
 import java.util.List;
 import java.util.ListIterator;
 import java.util.NoSuchElementException;
+import java.util.Set;
 
 import static org.apache.hadoop.fs.s3a.Constants.S3N_FOLDER_SUFFIX;
 import static org.apache.hadoop.fs.s3a.S3AUtils.createFileStatus;
@@ -54,6 +61,43 @@ public class Listing {
   }
 
   /**
+   * Create a FileStatus iterator against a provided list of file status, with
+   * a given status filter.
+   *
+   * @param fileStatuses the provided list of file status. NO remote calls.
+   * @param filter file path filter on which paths to accept
+   * @param acceptor the file status acceptor
+   * @return the file status iterator
+   */
+  ProvidedFileStatusIterator createProvidedFileStatusIterator(
+      FileStatus[] fileStatuses,
+      PathFilter filter,
+      FileStatusAcceptor acceptor) {
+    return new ProvidedFileStatusIterator(fileStatuses, filter, acceptor);
+  }
+
+  /**
+   * Create a FileStatus iterator against a path, with a given list object
+   * request.
+   *
+   * @param listPath path of the listing
+   * @param request initial request to make
+   * @param filter the filter on which paths to accept
+   * @param acceptor the class/predicate to decide which entries to accept
+   * in the listing based on the full file status.
+   * @return the iterator
+   * @throws IOException IO Problems
+   */
+  FileStatusListingIterator createFileStatusListingIterator(
+      Path listPath,
+      ListObjectsRequest request,
+      PathFilter filter,
+      Listing.FileStatusAcceptor acceptor) throws IOException {
+    return createFileStatusListingIterator(listPath, request, filter, acceptor,
+        null);
+  }
+
+  /**
    * Create a FileStatus iterator against a path, with a given
    * list object request.
    * @param listPath path of the listing
@@ -61,6 +105,8 @@ public class Listing {
    * @param filter the filter on which paths to accept
    * @param acceptor the class/predicate to decide which entries to accept
    * in the listing based on the full file status.
+   * @param providedStatus the provided list of file status, which may contain
+   *                       items that are not listed from source.
    * @return the iterator
    * @throws IOException IO Problems
    */
@@ -68,11 +114,13 @@ public class Listing {
       Path listPath,
       ListObjectsRequest request,
       PathFilter filter,
-      Listing.FileStatusAcceptor acceptor) throws IOException {
+      Listing.FileStatusAcceptor acceptor,
+      RemoteIterator<FileStatus> providedStatus) throws IOException {
     return new FileStatusListingIterator(
         new ObjectListingIterator(listPath, request),
         filter,
-        acceptor);
+        acceptor,
+        providedStatus);
   }
 
   /**
@@ -80,12 +128,27 @@ public class Listing {
    * @param statusIterator an iterator over the remote status entries
    * @return a new remote iterator
    */
+  @VisibleForTesting
   LocatedFileStatusIterator createLocatedFileStatusIterator(
       RemoteIterator<FileStatus> statusIterator) {
     return new LocatedFileStatusIterator(statusIterator);
   }
 
   /**
+   * Create an located status iterator that wraps another to filter out a set
+   * of recently deleted items.
+   * @param iterator an iterator over the remote located status entries.
+   * @param tombstones set of paths that are recently deleted and should be
+   *                   filtered.
+   * @return a new remote iterator.
+   */
+  @VisibleForTesting
+  TombstoneReconcilingIterator createTombstoneReconcilingIterator(
+      RemoteIterator<LocatedFileStatus> iterator, Set<Path> tombstones) {
+    return new TombstoneReconcilingIterator(iterator, tombstones);
+  }
+
+  /**
    * Interface to implement by the logic deciding whether to accept a summary
    * entry or path as a valid file or directory.
    */
@@ -108,6 +171,13 @@ public class Listing {
      * should be generated.)
      */
     boolean accept(Path keyPath, String commonPrefix);
+
+    /**
+     * Predicate to decide whether or not to accept a file status.
+     * @param status file status containing file path information
+     * @return true if the status is accepted else false
+     */
+    boolean accept(FileStatus status);
   }
 
   /**
@@ -115,9 +185,9 @@ public class Listing {
    * value.
    *
    * If the status value is null, the iterator declares that it has no data.
-   * This iterator is used to handle {@link listStatus()} calls where the path
-   * handed in refers to a file, not a directory: this is the iterator
-   * returned.
+   * This iterator is used to handle {@link S3AFileSystem#listStatus} calls
+   * where the path handed in refers to a file, not a directory: this is the
+   * iterator returned.
    */
   static final class SingleStatusRemoteIterator
       implements RemoteIterator<LocatedFileStatus> {
@@ -169,6 +239,47 @@ public class Listing {
   }
 
   /**
+   * This wraps up a provided non-null list of file status as a remote iterator.
+   *
+   * It firstly filters the provided list and later {@link #next} call will get
+   * from the filtered list. This suffers from scalability issues if the
+   * provided list is too large.
+   *
+   * There is no remote data to fetch.
+   */
+  static class ProvidedFileStatusIterator
+      implements RemoteIterator<FileStatus> {
+    private final ArrayList<FileStatus> filteredStatusList;
+    private int index = 0;
+
+    ProvidedFileStatusIterator(FileStatus[] fileStatuses, PathFilter filter,
+        FileStatusAcceptor acceptor) {
+      Preconditions.checkArgument(fileStatuses != null, "Null status list!");
+
+      filteredStatusList = new ArrayList<>(fileStatuses.length);
+      for (FileStatus status : fileStatuses) {
+        if (filter.accept(status.getPath()) && acceptor.accept(status)) {
+          filteredStatusList.add(status);
+        }
+      }
+      filteredStatusList.trimToSize();
+    }
+
+    @Override
+    public boolean hasNext() throws IOException {
+      return index < filteredStatusList.size();
+    }
+
+    @Override
+    public FileStatus next() throws IOException {
+      if (!hasNext()) {
+        throw new NoSuchElementException();
+      }
+      return filteredStatusList.get(index++);
+    }
+  }
+
+  /**
    * Wraps up object listing into a remote iterator which will ask for more
    * listing data if needed.
    *
@@ -179,7 +290,7 @@ public class Listing {
    * iterator can declare that there is more data available.
    *
    * The need to filter the results precludes the iterator from simply
-   * declaring that if the {@link S3AFileSystem.ObjectListingIterator#hasNext()}
+   * declaring that if the {@link ObjectListingIterator#hasNext()}
    * is true then there are more results. Instead the next batch of results must
    * be retrieved and filtered.
    *
@@ -208,20 +319,33 @@ public class Listing {
     /** Iterator over the current set of results. */
     private ListIterator<FileStatus> statusBatchIterator;
 
+    private final Set<FileStatus> providedStatus;
+    private Iterator<FileStatus> providedStatusIterator;
+
     /**
      * Create an iterator over file status entries.
      * @param source the listing iterator from a listObjects call.
      * @param filter the filter on which paths to accept
      * @param acceptor the class/predicate to decide which entries to accept
      * in the listing based on the full file status.
+     * @param providedStatus the provided list of file status, which may contain
+     *                       items that are not listed from source.
      * @throws IOException IO Problems
      */
     FileStatusListingIterator(ObjectListingIterator source,
         PathFilter filter,
-        FileStatusAcceptor acceptor) throws IOException {
+        FileStatusAcceptor acceptor,
+        RemoteIterator<FileStatus> providedStatus) throws IOException {
       this.source = source;
       this.filter = filter;
       this.acceptor = acceptor;
+      this.providedStatus = new HashSet<>();
+      for (; providedStatus != null && providedStatus.hasNext();) {
+        final FileStatus status = providedStatus.next();
+        if (filter.accept(status.getPath()) && acceptor.accept(status)) {
+          this.providedStatus.add(status);
+        }
+      }
       // build the first set of results. This will not trigger any
       // remote IO, assuming the source iterator is in its initial
       // iteration
@@ -233,26 +357,53 @@ public class Listing {
      * If there is data in the local filtered list, return true.
      * Else: request more data util that condition is met, or there
      * is no more remote listing data.
+     * Lastly, return true if the {@code providedStatusIterator}
+     * has left items.
      * @return true if a call to {@link #next()} will succeed.
      * @throws IOException
      */
     @Override
     public boolean hasNext() throws IOException {
-      return statusBatchIterator.hasNext() || requestNextBatch();
+      return sourceHasNext() || providedStatusIterator.hasNext();
+    }
+
+    private boolean sourceHasNext() throws IOException {
+      if (statusBatchIterator.hasNext() || requestNextBatch()) {
+        return true;
+      } else {
+        // turn to file status that are only in provided list
+        if (providedStatusIterator == null) {
+          LOG.debug("Start iterating the provided status.");
+          providedStatusIterator = providedStatus.iterator();
+        }
+        return false;
+      }
     }
 
     @Override
     public FileStatus next() throws IOException {
-      if (!hasNext()) {
-        throw new NoSuchElementException();
+      final FileStatus status;
+      if (sourceHasNext()) {
+        status = statusBatchIterator.next();
+        // We remove from provided list the file status listed by S3 so that
+        // this does not return duplicate items.
+        LOG.debug("Removing the status from provided file status {}", status);
+        providedStatus.remove(status);
+      } else {
+        if (providedStatusIterator.hasNext()) {
+          status = providedStatusIterator.next();
+          LOG.debug("Returning provided file status {}", status);
+        } else {
+          throw new NoSuchElementException();
+        }
       }
-      return statusBatchIterator.next();
+      return status;
     }
 
     /**
      * Try to retrieve another batch.
      * Note that for the initial batch,
-     * {@link S3AFileSystem.ObjectListingIterator} does not generate a request;
+     * {@link ObjectListingIterator} does not generate a request;
      * it simply returns the initial set.
      *
      * @return true if a new batch was created.
@@ -312,7 +463,7 @@ public class Listing {
       for (String prefix : objects.getCommonPrefixes()) {
         Path keyPath = owner.keyToQualifiedPath(prefix);
         if (acceptor.accept(keyPath, prefix) && filter.accept(keyPath)) {
-          FileStatus status = new S3AFileStatus(false, keyPath,
+          FileStatus status = new S3AFileStatus(Tristate.FALSE, keyPath,
               owner.getUsername());
           LOG.debug("Adding directory: {}", status);
           added++;
@@ -352,7 +503,7 @@ public class Listing {
    * instance.
    *
    * 2. Second and later invocations will continue the ongoing listing,
-   * calling {@link #continueListObjects(ObjectListing)} to request the next
+   * calling {@link S3AFileSystem#continueListObjects} to request the next
    * batch of results.
    *
    * 3. The {@link #hasNext()} predicate returns true for the initial call,
@@ -504,6 +655,11 @@ public class Listing {
     public boolean accept(Path keyPath, String prefix) {
       return false;
     }
+
+    @Override
+    public boolean accept(FileStatus status) {
+      return (status != null) && status.isFile();
+    }
   }
 
   /**
@@ -534,6 +690,80 @@ public class Listing {
   }
 
   /**
+   * Wraps another iterator and filters out files that appear in the provided
+   * set of tombstones.  Will read ahead in the iterator when necessary to
+   * ensure that emptiness is detected early enough if only deleted objects
+   * remain in the source iterator.
+   */
+  static class TombstoneReconcilingIterator implements
+      RemoteIterator<LocatedFileStatus> {
+    private LocatedFileStatus next = null;
+    private final RemoteIterator<LocatedFileStatus> iterator;
+    private final Set<Path> tombstones;
+
+    /**
+     * @param iterator Source iterator to filter
+     * @param tombstones set of tombstone markers to filter out of results
+     */
+    TombstoneReconcilingIterator(RemoteIterator<LocatedFileStatus>
+        iterator, Set<Path> tombstones) {
+      this.iterator = iterator;
+      if (tombstones != null) {
+        this.tombstones = tombstones;
+      } else {
+        this.tombstones = Collections.EMPTY_SET;
+      }
+    }
+
+    private boolean fetch() throws IOException {
+      while (next == null && iterator.hasNext()) {
+        LocatedFileStatus candidate = iterator.next();
+        if (!tombstones.contains(candidate.getPath())) {
+          next = candidate;
+          return true;
+        }
+      }
+      return false;
+    }
+
+    public boolean hasNext() throws IOException {
+      if (next != null) {
+        return true;
+      }
+      return fetch();
+    }
+
+    public LocatedFileStatus next() throws IOException {
+      if (hasNext()) {
+        LocatedFileStatus result = next;
+        next = null;
+        fetch();
+        return result;
+      }
+      throw new NoSuchElementException();
+    }
+  }
+
+  /**
+   * Accept all entries except those which map to S3N pseudo directory markers.
+   */
+  static class AcceptAllButS3nDirs implements FileStatusAcceptor {
+
+    public boolean accept(Path keyPath, S3ObjectSummary summary) {
+      return !summary.getKey().endsWith(S3N_FOLDER_SUFFIX);
+    }
+
+    public boolean accept(Path keyPath, String prefix) {
+      return !keyPath.toString().endsWith(S3N_FOLDER_SUFFIX);
+    }
+
+    public boolean accept(FileStatus status) {
+      return !status.getPath().toString().endsWith(S3N_FOLDER_SUFFIX);
+    }
+
+  }
+
+  /**
    * Accept all entries except the base path and those which map to S3N
    * pseudo directory markers.
    */
@@ -575,6 +805,11 @@ public class Listing {
     public boolean accept(Path keyPath, String prefix) {
       return !keyPath.equals(qualifiedPath);
     }
+
+    @Override
+    public boolean accept(FileStatus status) {
+      return (status != null) && !status.getPath().equals(qualifiedPath);
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
index 3fbdcb0..f846689 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
@@ -79,6 +79,9 @@ class S3ABlockOutputStream extends OutputStream {
   /** Size of all blocks. */
   private final int blockSize;
 
+  /** Total bytes for uploads submitted so far. */
+  private long bytesSubmitted;
+
   /** Callback for progress. */
   private final ProgressListener progressListener;
   private final ListeningExecutorService executorService;
@@ -302,6 +305,7 @@ class S3ABlockOutputStream extends OutputStream {
     }
     try {
       multiPartUpload.uploadBlockAsync(getActiveBlock());
+      bytesSubmitted += getActiveBlock().dataSize();
     } finally {
       // set the block to null, so the next write will create a new block.
       clearActiveBlock();
@@ -330,13 +334,14 @@ class S3ABlockOutputStream extends OutputStream {
         this,
         blockCount,
         hasBlock ? block : "(none)");
+    long bytes = 0;
     try {
       if (multiPartUpload == null) {
         if (hasBlock) {
           // no uploads of data have taken place, put the single block up.
           // This must happen even if there is no data, so that 0 byte files
           // are created.
-          putObject();
+          bytes = putObject();
         }
       } else {
         // there has already been at least one block scheduled for upload;
@@ -350,6 +355,7 @@ class S3ABlockOutputStream extends OutputStream {
             multiPartUpload.waitForAllPartUploads();
         // then complete the operation
         multiPartUpload.complete(partETags);
+        bytes = bytesSubmitted;
       }
       LOG.debug("Upload complete for {}", writeOperationHelper);
     } catch (IOException ioe) {
@@ -362,7 +368,7 @@ class S3ABlockOutputStream extends OutputStream {
       clearActiveBlock();
     }
     // All end of write operations, including deleting fake parent directories
-    writeOperationHelper.writeSuccessful();
+    writeOperationHelper.writeSuccessful(bytes);
   }
 
   /**
@@ -370,8 +376,11 @@ class S3ABlockOutputStream extends OutputStream {
    * is empty a 0-byte PUT will be invoked, as it is needed to create an
    * entry at the far end.
    * @throws IOException any problem.
+   * @return number of bytes uploaded. If thread was interrupted while
+   * waiting for upload to complete, returns zero with interrupted flag set
+   * on this thread.
    */
-  private void putObject() throws IOException {
+  private int putObject() throws IOException {
     LOG.debug("Executing regular upload for {}", writeOperationHelper);
 
     final S3ADataBlocks.DataBlock block = getActiveBlock();
@@ -405,9 +414,11 @@ class S3ABlockOutputStream extends OutputStream {
     //wait for completion
     try {
       putObjectResult.get();
+      return size;
     } catch (InterruptedException ie) {
       LOG.warn("Interrupted object upload", ie);
       Thread.currentThread().interrupt();
+      return 0;
     } catch (ExecutionException ee) {
       throw extractException("regular upload", key, ee);
     }


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


[2/8] hadoop git commit: HADOOP-13345 S3Guard: Improved Consistency for S3A. Contributed by: Chris Nauroth, Aaron Fabbri, Mingliang Liu, Lei (Eddy) Xu, Sean Mackrory, Steve Loughran and others.

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreTestBase.java
new file mode 100644
index 0000000..c19ae91
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreTestBase.java
@@ -0,0 +1,887 @@
+/*
+ * 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.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+
+import com.google.common.collect.Sets;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.Test;
+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.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.s3a.S3ATestUtils;
+import org.apache.hadoop.fs.s3a.Tristate;
+import org.apache.hadoop.io.IOUtils;
+
+/**
+ * Main test class for MetadataStore implementations.
+ * Implementations should each create a test by subclassing this and
+ * overriding {@link #createContract()}.
+ * If your implementation may return missing results for recently set paths,
+ * override {@link MetadataStoreTestBase#allowMissing()}.
+ */
+public abstract class MetadataStoreTestBase extends Assert {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(MetadataStoreTestBase.class);
+
+  /** Some dummy values for sanity-checking FileStatus contents. */
+  static final long BLOCK_SIZE = 32 * 1024 * 1024;
+  static final int REPLICATION = 1;
+  static final FsPermission PERMISSION = new FsPermission((short)0755);
+  static final String OWNER = "bob";
+  static final String GROUP = "uncles";
+  private final long accessTime = System.currentTimeMillis();
+  private final long modTime = accessTime - 5000;
+
+  /**
+   * Each test should override this.  Will use a new Configuration instance.
+   * @return Contract which specifies the MetadataStore under test plus config.
+   */
+  public abstract AbstractMSContract createContract() throws IOException;
+
+  /**
+   * Each test should override this.
+   * @param conf Base configuration instance to use.
+   * @return Contract which specifies the MetadataStore under test plus config.
+   */
+  public abstract AbstractMSContract createContract(Configuration conf)
+      throws IOException;
+
+  /**
+   * Tests assume that implementations will return recently set results.  If
+   * your implementation does not always hold onto metadata (e.g. LRU or
+   * time-based expiry) you can override this to return false.
+   * @return true if the test should succeed when null results are returned
+   *  from the MetadataStore under test.
+   */
+  public boolean allowMissing() {
+    return false;
+  }
+
+  /**
+   * Pruning is an optional feature for metadata store implementations.
+   * Tests will only check that functionality if it is expected to work.
+   * @return true if the test should expect pruning to work.
+   */
+  public boolean supportsPruning() {
+    return true;
+  }
+
+  /** The MetadataStore contract used to test against. */
+  private AbstractMSContract contract;
+
+  private MetadataStore ms;
+
+  /**
+   * @return reference to the test contract.
+   */
+  protected AbstractMSContract getContract() {
+    return contract;
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    LOG.debug("== Setup. ==");
+    contract = createContract();
+    ms = contract.getMetadataStore();
+    assertNotNull("null MetadataStore", ms);
+    assertNotNull("null FileSystem", contract.getFileSystem());
+    ms.initialize(contract.getFileSystem());
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    LOG.debug("== Tear down. ==");
+    if (ms != null) {
+      try {
+        ms.destroy();
+      } catch (Exception e) {
+        LOG.warn("Failed to destroy tables in teardown", e);
+      }
+      IOUtils.closeStream(ms);
+      ms = null;
+    }
+  }
+
+  /**
+   * Helper function for verifying DescendantsIterator and
+   * MetadataStoreListFilesIterator behavior.
+   * @param createNodes List of paths to create
+   * @param checkNodes List of paths that the iterator should return
+   */
+  private void doTestDescendantsIterator(
+      Class implementation, String[] createNodes,
+      String[] checkNodes) throws Exception {
+    // we set up the example file system tree in metadata store
+    for (String pathStr : createNodes) {
+      final FileStatus status = pathStr.contains("file")
+          ? basicFileStatus(strToPath(pathStr), 100, false)
+          : basicFileStatus(strToPath(pathStr), 0, true);
+      ms.put(new PathMetadata(status));
+    }
+
+    final PathMetadata rootMeta = new PathMetadata(makeDirStatus("/"));
+    RemoteIterator<FileStatus> iterator;
+    if (implementation == DescendantsIterator.class) {
+      iterator = new DescendantsIterator(ms, rootMeta);
+    } else if (implementation == MetadataStoreListFilesIterator.class) {
+      iterator = new MetadataStoreListFilesIterator(ms, rootMeta, false);
+    } else {
+      throw new UnsupportedOperationException("Unrecognized class");
+    }
+
+    final Set<String> actual = new HashSet<>();
+    while (iterator.hasNext()) {
+      final Path p = iterator.next().getPath();
+      actual.add(Path.getPathWithoutSchemeAndAuthority(p).toString());
+    }
+    LOG.info("We got {} by iterating DescendantsIterator", actual);
+
+    if (!allowMissing()) {
+      assertEquals(Sets.newHashSet(checkNodes), actual);
+    }
+  }
+
+  /**
+   * Test that we can get the whole sub-tree by iterating DescendantsIterator.
+   *
+   * The tree is similar to or same as the example in code comment.
+   */
+  @Test
+  public void testDescendantsIterator() throws Exception {
+    final String[] tree = new String[] {
+        "/dir1",
+        "/dir1/dir2",
+        "/dir1/dir3",
+        "/dir1/dir2/file1",
+        "/dir1/dir2/file2",
+        "/dir1/dir3/dir4",
+        "/dir1/dir3/dir5",
+        "/dir1/dir3/dir4/file3",
+        "/dir1/dir3/dir5/file4",
+        "/dir1/dir3/dir6"
+    };
+    doTestDescendantsIterator(DescendantsIterator.class,
+        tree, tree);
+  }
+
+  /**
+   * Test that we can get the correct subset of the tree with
+   * MetadataStoreListFilesIterator.
+   *
+   * The tree is similar to or same as the example in code comment.
+   */
+  @Test
+  public void testMetadataStoreListFilesIterator() throws Exception {
+    final String[] wholeTree = new String[] {
+        "/dir1",
+        "/dir1/dir2",
+        "/dir1/dir3",
+        "/dir1/dir2/file1",
+        "/dir1/dir2/file2",
+        "/dir1/dir3/dir4",
+        "/dir1/dir3/dir5",
+        "/dir1/dir3/dir4/file3",
+        "/dir1/dir3/dir5/file4",
+        "/dir1/dir3/dir6"
+    };
+    final String[] leafNodes = new String[] {
+        "/dir1/dir2/file1",
+        "/dir1/dir2/file2",
+        "/dir1/dir3/dir4/file3",
+        "/dir1/dir3/dir5/file4"
+    };
+    doTestDescendantsIterator(MetadataStoreListFilesIterator.class, wholeTree,
+        leafNodes);
+  }
+
+  @Test
+  public void testPutNew() throws Exception {
+    /* create three dirs /da1, /da2, /da3 */
+    createNewDirs("/da1", "/da2", "/da3");
+
+    /* It is caller's responsibility to set up ancestor entries beyond the
+     * containing directory.  We only track direct children of the directory.
+     * Thus this will not affect entry for /da1.
+     */
+    ms.put(new PathMetadata(makeFileStatus("/da1/db1/fc1", 100)));
+
+    assertEmptyDirs("/da2", "/da3");
+    assertDirectorySize("/da1/db1", 1);
+
+    /* Check contents of dir status. */
+    PathMetadata dirMeta = ms.get(strToPath("/da1"));
+    if (!allowMissing() || dirMeta != null) {
+      verifyDirStatus(dirMeta.getFileStatus());
+    }
+
+    /* This already exists, and should silently replace it. */
+    ms.put(new PathMetadata(makeDirStatus("/da1/db1")));
+
+    /* If we had putNew(), and used it above, this would be empty again. */
+    assertDirectorySize("/da1", 1);
+
+    assertEmptyDirs("/da2", "/da3");
+
+    /* Ensure new files update correct parent dirs. */
+    ms.put(new PathMetadata(makeFileStatus("/da1/db1/fc1", 100)));
+    ms.put(new PathMetadata(makeFileStatus("/da1/db1/fc2", 200)));
+    assertDirectorySize("/da1", 1);
+    assertDirectorySize("/da1/db1", 2);
+    assertEmptyDirs("/da2", "/da3");
+    PathMetadata meta = ms.get(strToPath("/da1/db1/fc2"));
+    if (!allowMissing() || meta != null) {
+      assertNotNull("Get file after put new.", meta);
+      verifyFileStatus(meta.getFileStatus(), 200);
+    }
+  }
+
+  @Test
+  public void testPutOverwrite() throws Exception {
+    final String filePath = "/a1/b1/c1/some_file";
+    final String dirPath = "/a1/b1/c1/d1";
+    ms.put(new PathMetadata(makeFileStatus(filePath, 100)));
+    ms.put(new PathMetadata(makeDirStatus(dirPath)));
+    PathMetadata meta = ms.get(strToPath(filePath));
+    if (!allowMissing() || meta != null) {
+      verifyFileStatus(meta.getFileStatus(), 100);
+    }
+
+    ms.put(new PathMetadata(basicFileStatus(strToPath(filePath), 9999, false)));
+    meta = ms.get(strToPath(filePath));
+    if (!allowMissing() || meta != null) {
+      verifyFileStatus(meta.getFileStatus(), 9999);
+    }
+  }
+
+  @Test
+  public void testRootDirPutNew() throws Exception {
+    Path rootPath = strToPath("/");
+
+    ms.put(new PathMetadata(makeFileStatus("/file1", 100)));
+    DirListingMetadata dir = ms.listChildren(rootPath);
+    if (!allowMissing() || dir != null) {
+      assertNotNull("Root dir cached", dir);
+      assertFalse("Root not fully cached", dir.isAuthoritative());
+      assertNotNull("have root dir file listing", dir.getListing());
+      assertEquals("One file in root dir", 1, dir.getListing().size());
+      assertEquals("file1 in root dir", strToPath("/file1"),
+          dir.getListing().iterator().next().getFileStatus().getPath());
+    }
+  }
+
+  @Test
+  public void testDelete() throws Exception {
+    setUpDeleteTest();
+
+    ms.delete(strToPath("/ADirectory1/db1/file2"));
+
+    /* Ensure delete happened. */
+    assertDirectorySize("/ADirectory1/db1", 1);
+    PathMetadata meta = ms.get(strToPath("/ADirectory1/db1/file2"));
+    assertTrue("File deleted", meta == null || meta.isDeleted());
+  }
+
+  @Test
+  public void testDeleteSubtree() throws Exception {
+    deleteSubtreeHelper("");
+  }
+
+  @Test
+  public void testDeleteSubtreeHostPath() throws Exception {
+    deleteSubtreeHelper(contract.getFileSystem().getUri().toString());
+  }
+
+  private void deleteSubtreeHelper(String pathPrefix) throws Exception {
+
+    String p = pathPrefix;
+    setUpDeleteTest(p);
+    createNewDirs(p + "/ADirectory1/db1/dc1", p + "/ADirectory1/db1/dc1/dd1");
+    ms.put(new PathMetadata(
+        makeFileStatus(p + "/ADirectory1/db1/dc1/dd1/deepFile", 100)));
+    if (!allowMissing()) {
+      assertCached(p + "/ADirectory1/db1");
+    }
+    ms.deleteSubtree(strToPath(p + "/ADirectory1/db1/"));
+
+    assertEmptyDirectory(p + "/ADirectory1");
+    assertDeleted(p + "/ADirectory1/db1");
+    assertDeleted(p + "/ADirectory1/file1");
+    assertDeleted(p + "/ADirectory1/file2");
+    assertDeleted(p + "/ADirectory1/db1/dc1/dd1/deepFile");
+    assertEmptyDirectory(p + "/ADirectory2");
+  }
+
+
+  /*
+   * Some implementations might not support this.  It was useful to test
+   * correctness of the LocalMetadataStore implementation, but feel free to
+   * override this to be a no-op.
+   */
+  @Test
+  public void testDeleteRecursiveRoot() throws Exception {
+    setUpDeleteTest();
+
+    ms.deleteSubtree(strToPath("/"));
+    assertDeleted("/ADirectory1");
+    assertDeleted("/ADirectory2");
+    assertDeleted("/ADirectory2/db1");
+    assertDeleted("/ADirectory2/db1/file1");
+    assertDeleted("/ADirectory2/db1/file2");
+  }
+
+  @Test
+  public void testDeleteNonExisting() throws Exception {
+    // Path doesn't exist, but should silently succeed
+    ms.delete(strToPath("/bobs/your/uncle"));
+
+    // Ditto.
+    ms.deleteSubtree(strToPath("/internets"));
+  }
+
+
+  private void setUpDeleteTest() throws IOException {
+    setUpDeleteTest("");
+  }
+
+  private void setUpDeleteTest(String prefix) throws IOException {
+    createNewDirs(prefix + "/ADirectory1", prefix + "/ADirectory2",
+        prefix + "/ADirectory1/db1");
+    ms.put(new PathMetadata(makeFileStatus(prefix + "/ADirectory1/db1/file1",
+        100)));
+    ms.put(new PathMetadata(makeFileStatus(prefix + "/ADirectory1/db1/file2",
+        100)));
+
+    PathMetadata meta = ms.get(strToPath(prefix + "/ADirectory1/db1/file2"));
+    if (!allowMissing() || meta != null) {
+      assertNotNull("Found test file", meta);
+      assertDirectorySize(prefix + "/ADirectory1/db1", 2);
+    }
+  }
+
+  @Test
+  public void testGet() throws Exception {
+    final String filePath = "/a1/b1/c1/some_file";
+    final String dirPath = "/a1/b1/c1/d1";
+    ms.put(new PathMetadata(makeFileStatus(filePath, 100)));
+    ms.put(new PathMetadata(makeDirStatus(dirPath)));
+    PathMetadata meta = ms.get(strToPath(filePath));
+    if (!allowMissing() || meta != null) {
+      assertNotNull("Get found file", meta);
+      verifyFileStatus(meta.getFileStatus(), 100);
+    }
+
+    if (!(ms instanceof NullMetadataStore)) {
+      ms.delete(strToPath(filePath));
+      meta = ms.get(strToPath(filePath));
+      assertTrue("Tombstone not left for deleted file", meta.isDeleted());
+    }
+
+    meta = ms.get(strToPath(dirPath));
+    if (!allowMissing() || meta != null) {
+      assertNotNull("Get found file (dir)", meta);
+      assertTrue("Found dir", meta.getFileStatus().isDirectory());
+    }
+
+    meta = ms.get(strToPath("/bollocks"));
+    assertNull("Don't get non-existent file", meta);
+  }
+
+  @Test
+  public void testGetEmptyDir() throws Exception {
+    final String dirPath = "/a1/b1/c1/d1";
+    // Creates /a1/b1/c1/d1 as an empty dir
+    setupListStatus();
+
+    // 1. Tell MetadataStore (MS) that there are zero children
+    putListStatusFiles(dirPath, true /* authoritative */
+        /* zero children */);
+
+    // 2. Request a file status for dir, including whether or not the dir
+    // is empty.
+    PathMetadata meta = ms.get(strToPath(dirPath), true);
+
+    // 3. Check that either (a) the MS doesn't track whether or not it is
+    // empty (which is allowed), or (b) the MS knows the dir is empty.
+    if (!allowMissing() || meta != null) {
+      assertNotNull("Get should find meta for dir", meta);
+      assertNotEquals("Dir is empty or unknown", Tristate.FALSE,
+          meta.isEmptyDirectory());
+    }
+  }
+
+  @Test
+  public void testGetNonEmptyDir() throws Exception {
+    final String dirPath = "/a1/b1/c1";
+    // Creates /a1/b1/c1 as an non-empty dir
+    setupListStatus();
+
+    // Request a file status for dir, including whether or not the dir
+    // is empty.
+    PathMetadata meta = ms.get(strToPath(dirPath), true);
+
+    // MetadataStore knows /a1/b1/c1 has at least one child.  It is valid
+    // for it to answer either (a) UNKNOWN: the MS doesn't track whether
+    // or not the dir is empty, or (b) the MS knows the dir is non-empty.
+    if (!allowMissing() || meta != null) {
+      assertNotNull("Get should find meta for dir", meta);
+      assertNotEquals("Dir is non-empty or unknown", Tristate.TRUE,
+          meta.isEmptyDirectory());
+    }
+  }
+
+  @Test
+  public void testGetDirUnknownIfEmpty() throws Exception {
+    final String dirPath = "/a1/b1/c1/d1";
+    // 1. Create /a1/b1/c1/d1 as an empty dir, but do not tell MetadataStore
+    // (MS) whether or not it has any children.
+    setupListStatus();
+
+    // 2. Request a file status for dir, including whether or not the dir
+    // is empty.
+    PathMetadata meta = ms.get(strToPath(dirPath), true);
+
+    // 3. Assert MS reports isEmptyDir as UNKONWN: We haven't told MS
+    // whether or not the directory has any children.
+    if (!allowMissing() || meta != null) {
+      assertNotNull("Get should find meta for dir", meta);
+      assertEquals("Dir empty is unknown", Tristate.UNKNOWN,
+          meta.isEmptyDirectory());
+    }
+  }
+
+  @Test
+  public void testListChildren() throws Exception {
+    setupListStatus();
+
+    DirListingMetadata dirMeta;
+    dirMeta = ms.listChildren(strToPath("/"));
+    if (!allowMissing()) {
+      assertNotNull(dirMeta);
+        /* Cache has no way of knowing it has all entries for root unless we
+         * specifically tell it via put() with
+         * DirListingMetadata.isAuthoritative = true */
+      assertFalse("Root dir is not cached, or partially cached",
+          dirMeta.isAuthoritative());
+      assertListingsEqual(dirMeta.getListing(), "/a1", "/a2");
+    }
+
+    dirMeta = ms.listChildren(strToPath("/a1"));
+    if (!allowMissing() || dirMeta != null) {
+      dirMeta = dirMeta.withoutTombstones();
+      assertListingsEqual(dirMeta.getListing(), "/a1/b1", "/a1/b2");
+    }
+
+    // TODO HADOOP-14756 instrument MetadataStore for asserting & testing
+    dirMeta = ms.listChildren(strToPath("/a1/b1"));
+    if (!allowMissing() || dirMeta != null) {
+      assertListingsEqual(dirMeta.getListing(), "/a1/b1/file1", "/a1/b1/file2",
+          "/a1/b1/c1");
+    }
+  }
+
+  @Test
+  public void testDirListingRoot() throws Exception {
+    commonTestPutListStatus("/");
+  }
+
+  @Test
+  public void testPutDirListing() throws Exception {
+    commonTestPutListStatus("/a");
+  }
+
+  @Test
+  public void testInvalidListChildren() throws Exception {
+    setupListStatus();
+    assertNull("missing path returns null",
+        ms.listChildren(strToPath("/a1/b1x")));
+  }
+
+  @Test
+  public void testMove() throws Exception {
+    // Create test dir structure
+    createNewDirs("/a1", "/a2", "/a3");
+    createNewDirs("/a1/b1", "/a1/b2");
+    putListStatusFiles("/a1/b1", false, "/a1/b1/file1", "/a1/b1/file2");
+
+    // Assert root listing as expected
+    Collection<PathMetadata> entries;
+    DirListingMetadata dirMeta = ms.listChildren(strToPath("/"));
+    if (!allowMissing() || dirMeta != null) {
+      dirMeta = dirMeta.withoutTombstones();
+      assertNotNull("Listing root", dirMeta);
+      entries = dirMeta.getListing();
+      assertListingsEqual(entries, "/a1", "/a2", "/a3");
+    }
+
+    // Assert src listing as expected
+    dirMeta = ms.listChildren(strToPath("/a1/b1"));
+    if (!allowMissing() || dirMeta != null) {
+      assertNotNull("Listing /a1/b1", dirMeta);
+      entries = dirMeta.getListing();
+      assertListingsEqual(entries, "/a1/b1/file1", "/a1/b1/file2");
+    }
+
+    // Do the move(): rename(/a1/b1, /b1)
+    Collection<Path> srcPaths = Arrays.asList(strToPath("/a1/b1"),
+        strToPath("/a1/b1/file1"), strToPath("/a1/b1/file2"));
+
+    ArrayList<PathMetadata> destMetas = new ArrayList<>();
+    destMetas.add(new PathMetadata(makeDirStatus("/b1")));
+    destMetas.add(new PathMetadata(makeFileStatus("/b1/file1", 100)));
+    destMetas.add(new PathMetadata(makeFileStatus("/b1/file2", 100)));
+    ms.move(srcPaths, destMetas);
+
+    // Assert src is no longer there
+    dirMeta = ms.listChildren(strToPath("/a1"));
+    if (!allowMissing() || dirMeta != null) {
+      assertNotNull("Listing /a1", dirMeta);
+      entries = dirMeta.withoutTombstones().getListing();
+      assertListingsEqual(entries, "/a1/b2");
+    }
+
+    PathMetadata meta = ms.get(strToPath("/a1/b1/file1"));
+    assertTrue("Src path deleted", meta == null || meta.isDeleted());
+
+    // Assert dest looks right
+    meta = ms.get(strToPath("/b1/file1"));
+    if (!allowMissing() || meta != null) {
+      assertNotNull("dest file not null", meta);
+      verifyFileStatus(meta.getFileStatus(), 100);
+    }
+
+    dirMeta = ms.listChildren(strToPath("/b1"));
+    if (!allowMissing() || dirMeta != null) {
+      assertNotNull("dest listing not null", dirMeta);
+      entries = dirMeta.getListing();
+      assertListingsEqual(entries, "/b1/file1", "/b1/file2");
+    }
+  }
+
+  /**
+   * Test that the MetadataStore differentiates between the same path in two
+   * different buckets.
+   */
+  @Test
+  public void testMultiBucketPaths() throws Exception {
+    String p1 = "s3a://bucket-a/path1";
+    String p2 = "s3a://bucket-b/path2";
+
+    // Make sure we start out empty
+    PathMetadata meta = ms.get(new Path(p1));
+    assertNull("Path should not be present yet.", meta);
+    meta = ms.get(new Path(p2));
+    assertNull("Path2 should not be present yet.", meta);
+
+    // Put p1, assert p2 doesn't match
+    ms.put(new PathMetadata(makeFileStatus(p1, 100)));
+    meta = ms.get(new Path(p2));
+    assertNull("Path 2 should not match path 1.", meta);
+
+    // Make sure delete is correct as well
+    if (!allowMissing()) {
+      ms.delete(new Path(p2));
+      meta = ms.get(new Path(p1));
+      assertNotNull("Path should not have been deleted", meta);
+    }
+    ms.delete(new Path(p1));
+  }
+
+  @Test
+  public void testPruneFiles() throws Exception {
+    Assume.assumeTrue(supportsPruning());
+    createNewDirs("/pruneFiles");
+
+    long oldTime = getTime();
+    ms.put(new PathMetadata(makeFileStatus("/pruneFiles/old", 1, oldTime,
+        oldTime)));
+    DirListingMetadata ls2 = ms.listChildren(strToPath("/pruneFiles"));
+    if (!allowMissing()) {
+      assertListingsEqual(ls2.getListing(), "/pruneFiles/old");
+    }
+
+    // It's possible for the Local implementation to get from /pruneFiles/old's
+    // modification time to here in under 1ms, causing it to not get pruned
+    Thread.sleep(1);
+    long cutoff = System.currentTimeMillis();
+    long newTime = getTime();
+    ms.put(new PathMetadata(makeFileStatus("/pruneFiles/new", 1, newTime,
+        newTime)));
+
+    DirListingMetadata ls;
+    ls = ms.listChildren(strToPath("/pruneFiles"));
+    if (!allowMissing()) {
+      assertListingsEqual(ls.getListing(), "/pruneFiles/new",
+          "/pruneFiles/old");
+    }
+    ms.prune(cutoff);
+    ls = ms.listChildren(strToPath("/pruneFiles"));
+    if (allowMissing()) {
+      assertDeleted("/pruneFiles/old");
+    } else {
+      assertListingsEqual(ls.getListing(), "/pruneFiles/new");
+    }
+  }
+
+  @Test
+  public void testPruneDirs() throws Exception {
+    Assume.assumeTrue(supportsPruning());
+
+    // We only test that files, not dirs, are removed during prune.
+    // We specifically allow directories to remain, as it is more robust
+    // for DynamoDBMetadataStore's prune() implementation: If a
+    // file was created in a directory while it was being pruned, it would
+    // violate the invariant that all ancestors of a file exist in the table.
+
+    createNewDirs("/pruneDirs/dir");
+
+    long oldTime = getTime();
+    ms.put(new PathMetadata(makeFileStatus("/pruneDirs/dir/file",
+        1, oldTime, oldTime)));
+
+    // It's possible for the Local implementation to get from the old
+    // modification time to here in under 1ms, causing it to not get pruned
+    Thread.sleep(1);
+    long cutoff = getTime();
+
+    ms.prune(cutoff);
+
+    assertDeleted("/pruneDirs/dir/file");
+  }
+
+  @Test
+  public void testPruneUnsetsAuthoritative() throws Exception {
+    String rootDir = "/unpruned-root-dir";
+    String grandparentDir = rootDir + "/pruned-grandparent-dir";
+    String parentDir = grandparentDir + "/pruned-parent-dir";
+    String staleFile = parentDir + "/stale-file";
+    String freshFile = rootDir + "/fresh-file";
+    String[] directories = {rootDir, grandparentDir, parentDir};
+
+    createNewDirs(rootDir, grandparentDir, parentDir);
+    long time = System.currentTimeMillis();
+    ms.put(new PathMetadata(
+        new FileStatus(0, false, 0, 0, time - 1, strToPath(staleFile)),
+        Tristate.FALSE, false));
+    ms.put(new PathMetadata(
+        new FileStatus(0, false, 0, 0, time + 1, strToPath(freshFile)),
+        Tristate.FALSE, false));
+
+    ms.prune(time);
+    DirListingMetadata listing;
+    for (String directory : directories) {
+      Path path = strToPath(directory);
+      if (ms.get(path) != null) {
+        listing = ms.listChildren(path);
+        assertFalse(listing.isAuthoritative());
+      }
+    }
+  }
+
+  /*
+   * Helper functions.
+   */
+
+  /** Modifies paths input array and returns it. */
+  private String[] buildPathStrings(String parent, String... paths)
+      throws IOException {
+    for (int i = 0; i < paths.length; i++) {
+      Path p = new Path(strToPath(parent), paths[i]);
+      paths[i] = p.toString();
+    }
+    return paths;
+  }
+
+  private void commonTestPutListStatus(final String parent) throws IOException {
+    putListStatusFiles(parent, true, buildPathStrings(parent, "file1", "file2",
+        "file3"));
+    DirListingMetadata dirMeta = ms.listChildren(strToPath(parent));
+    if (!allowMissing() || dirMeta != null) {
+      dirMeta = dirMeta.withoutTombstones();
+      assertNotNull("list after putListStatus", dirMeta);
+      Collection<PathMetadata> entries = dirMeta.getListing();
+      assertNotNull("listStatus has entries", entries);
+      assertListingsEqual(entries,
+          buildPathStrings(parent, "file1", "file2", "file3"));
+    }
+  }
+
+  private void setupListStatus() throws IOException {
+    createNewDirs("/a1", "/a2", "/a1/b1", "/a1/b2", "/a1/b1/c1",
+        "/a1/b1/c1/d1");
+    ms.put(new PathMetadata(makeFileStatus("/a1/b1/file1", 100)));
+    ms.put(new PathMetadata(makeFileStatus("/a1/b1/file2", 100)));
+  }
+
+  private void assertListingsEqual(Collection<PathMetadata> listing,
+      String ...pathStrs) throws IOException {
+    Set<Path> a = new HashSet<>();
+    for (PathMetadata meta : listing) {
+      a.add(meta.getFileStatus().getPath());
+    }
+
+    Set<Path> b = new HashSet<>();
+    for (String ps : pathStrs) {
+      b.add(strToPath(ps));
+    }
+    assertEquals("Same set of files", b, a);
+  }
+
+  private void putListStatusFiles(String dirPath, boolean authoritative,
+      String... filenames) throws IOException {
+    ArrayList<PathMetadata> metas = new ArrayList<>(filenames .length);
+    for (String filename : filenames) {
+      metas.add(new PathMetadata(makeFileStatus(filename, 100)));
+    }
+    DirListingMetadata dirMeta =
+        new DirListingMetadata(strToPath(dirPath), metas, authoritative);
+    ms.put(dirMeta);
+  }
+
+  private void createNewDirs(String... dirs)
+      throws IOException {
+    for (String pathStr : dirs) {
+      ms.put(new PathMetadata(makeDirStatus(pathStr)));
+    }
+  }
+
+  private void assertDirectorySize(String pathStr, int size)
+      throws IOException {
+    DirListingMetadata dirMeta = ms.listChildren(strToPath(pathStr));
+    if (!allowMissing()) {
+      assertNotNull("Directory " + pathStr + " in cache", dirMeta);
+    }
+    if (!allowMissing() || dirMeta != null) {
+      dirMeta = dirMeta.withoutTombstones();
+      assertEquals("Number of entries in dir " + pathStr, size,
+          nonDeleted(dirMeta.getListing()).size());
+    }
+  }
+
+  /** @return only file statuses which are *not* marked deleted. */
+  private Collection<PathMetadata> nonDeleted(
+      Collection<PathMetadata> statuses) {
+    Collection<PathMetadata> currentStatuses = new ArrayList<>();
+    for (PathMetadata status : statuses) {
+      if (!status.isDeleted()) {
+        currentStatuses.add(status);
+      }
+    }
+    return currentStatuses;
+  }
+
+  private void assertDeleted(String pathStr) throws IOException {
+    Path path = strToPath(pathStr);
+    PathMetadata meta = ms.get(path);
+    boolean cached = meta != null && !meta.isDeleted();
+    assertFalse(pathStr + " should not be cached.", cached);
+  }
+
+  protected void assertCached(String pathStr) throws IOException {
+    Path path = strToPath(pathStr);
+    PathMetadata meta = ms.get(path);
+    boolean cached = meta != null && !meta.isDeleted();
+    assertTrue(pathStr + " should be cached.", cached);
+  }
+
+  /**
+   * Convenience to create a fully qualified Path from string.
+   */
+  Path strToPath(String p) throws IOException {
+    final Path path = new Path(p);
+    assert path.isAbsolute();
+    return path.makeQualified(contract.getFileSystem().getUri(), null);
+  }
+
+  private void assertEmptyDirectory(String pathStr) throws IOException {
+    assertDirectorySize(pathStr, 0);
+  }
+
+  private void assertEmptyDirs(String ...dirs) throws IOException {
+    for (String pathStr : dirs) {
+      assertEmptyDirectory(pathStr);
+    }
+  }
+
+  FileStatus basicFileStatus(Path path, int size, boolean isDir) throws
+      IOException {
+    return basicFileStatus(path, size, isDir, modTime, accessTime);
+  }
+
+  FileStatus basicFileStatus(Path path, int size, boolean isDir,
+      long newModTime, long newAccessTime) throws IOException {
+    return new FileStatus(size, isDir, REPLICATION, BLOCK_SIZE, newModTime,
+        newAccessTime, PERMISSION, OWNER, GROUP, path);
+  }
+
+  private FileStatus makeFileStatus(String pathStr, int size) throws
+      IOException {
+    return makeFileStatus(pathStr, size, modTime, accessTime);
+  }
+
+  private FileStatus makeFileStatus(String pathStr, int size, long newModTime,
+      long newAccessTime) throws IOException {
+    return basicFileStatus(strToPath(pathStr), size, false,
+        newModTime, newAccessTime);
+  }
+
+  void verifyFileStatus(FileStatus status, long size) {
+    S3ATestUtils.verifyFileStatus(status, size, BLOCK_SIZE, modTime);
+  }
+
+  private FileStatus makeDirStatus(String pathStr) throws IOException {
+    return basicFileStatus(strToPath(pathStr), 0, true, modTime, accessTime);
+  }
+
+  /**
+   * Verify the directory file status. Subclass may verify additional fields.
+   */
+  void verifyDirStatus(FileStatus status) {
+    assertTrue("Is a dir", status.isDirectory());
+    assertEquals("zero length", 0, status.getLen());
+  }
+
+  long getModTime() {
+    return modTime;
+  }
+
+  long getAccessTime() {
+    return accessTime;
+  }
+
+  protected static long getTime() {
+    return System.currentTimeMillis();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestDirListingMetadata.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestDirListingMetadata.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestDirListingMetadata.java
new file mode 100644
index 0000000..8458252
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestDirListingMetadata.java
@@ -0,0 +1,303 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.s3guard;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.S3AFileStatus;
+
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.junit.Assert.*;
+
+/**
+ * Unit tests of {@link DirListingMetadata}.
+ */
+public class TestDirListingMetadata {
+
+  private static final String TEST_OWNER = "hadoop";
+
+  @Rule
+  public ExpectedException exception = ExpectedException.none();
+
+  @Test
+  public void testNullPath() {
+    exception.expect(NullPointerException.class);
+    exception.expectMessage(notNullValue(String.class));
+    new DirListingMetadata(null, null, false);
+  }
+
+  @Test
+  public void testNullListing() {
+    Path path = new Path("/path");
+    DirListingMetadata meta = new DirListingMetadata(path, null, false);
+    assertEquals(path, meta.getPath());
+    assertNotNull(meta.getListing());
+    assertTrue(meta.getListing().isEmpty());
+    assertFalse(meta.isAuthoritative());
+  }
+
+  @Test
+  public void testEmptyListing() {
+    Path path = new Path("/path");
+    DirListingMetadata meta = new DirListingMetadata(path,
+        new ArrayList<PathMetadata>(0),
+        false);
+    assertEquals(path, meta.getPath());
+    assertNotNull(meta.getListing());
+    assertTrue(meta.getListing().isEmpty());
+    assertFalse(meta.isAuthoritative());
+  }
+
+  @Test
+  public void testListing() {
+    Path path = new Path("/path");
+    PathMetadata pathMeta1 = new PathMetadata(
+        new S3AFileStatus(true, new Path(path, "dir1"), TEST_OWNER));
+    PathMetadata pathMeta2 = new PathMetadata(
+        new S3AFileStatus(true, new Path(path, "dir2"), TEST_OWNER));
+    PathMetadata pathMeta3 = new PathMetadata(
+        new S3AFileStatus(123, 456, new Path(path, "file1"), 8192, TEST_OWNER));
+    List<PathMetadata> listing = Arrays.asList(pathMeta1, pathMeta2, pathMeta3);
+    DirListingMetadata meta = new DirListingMetadata(path, listing, false);
+    assertEquals(path, meta.getPath());
+    assertNotNull(meta.getListing());
+    assertFalse(meta.getListing().isEmpty());
+    assertTrue(meta.getListing().contains(pathMeta1));
+    assertTrue(meta.getListing().contains(pathMeta2));
+    assertTrue(meta.getListing().contains(pathMeta3));
+    assertFalse(meta.isAuthoritative());
+  }
+
+  @Test
+  public void testListingUnmodifiable() {
+    Path path = new Path("/path");
+    DirListingMetadata meta = makeTwoDirsOneFile(path);
+    assertNotNull(meta.getListing());
+    exception.expect(UnsupportedOperationException.class);
+    meta.getListing().clear();
+  }
+
+  @Test
+  public void testAuthoritative() {
+    Path path = new Path("/path");
+    DirListingMetadata meta = new DirListingMetadata(path, null, true);
+    assertEquals(path, meta.getPath());
+    assertNotNull(meta.getListing());
+    assertTrue(meta.getListing().isEmpty());
+    assertTrue(meta.isAuthoritative());
+  }
+
+  @Test
+  public void testSetAuthoritative() {
+    Path path = new Path("/path");
+    DirListingMetadata meta = new DirListingMetadata(path, null, false);
+    assertEquals(path, meta.getPath());
+    assertNotNull(meta.getListing());
+    assertTrue(meta.getListing().isEmpty());
+    assertFalse(meta.isAuthoritative());
+    meta.setAuthoritative(true);
+    assertTrue(meta.isAuthoritative());
+  }
+
+  @Test
+  public void testGet() {
+    Path path = new Path("/path");
+    PathMetadata pathMeta1 = new PathMetadata(
+        new S3AFileStatus(true, new Path(path, "dir1"), TEST_OWNER));
+    PathMetadata pathMeta2 = new PathMetadata(
+        new S3AFileStatus(true, new Path(path, "dir2"), TEST_OWNER));
+    PathMetadata pathMeta3 = new PathMetadata(
+        new S3AFileStatus(123, 456, new Path(path, "file1"), 8192, TEST_OWNER));
+    List<PathMetadata> listing = Arrays.asList(pathMeta1, pathMeta2, pathMeta3);
+    DirListingMetadata meta = new DirListingMetadata(path, listing, false);
+    assertEquals(path, meta.getPath());
+    assertNotNull(meta.getListing());
+    assertFalse(meta.getListing().isEmpty());
+    assertTrue(meta.getListing().contains(pathMeta1));
+    assertTrue(meta.getListing().contains(pathMeta2));
+    assertTrue(meta.getListing().contains(pathMeta3));
+    assertFalse(meta.isAuthoritative());
+    assertEquals(pathMeta1, meta.get(pathMeta1.getFileStatus().getPath()));
+    assertEquals(pathMeta2, meta.get(pathMeta2.getFileStatus().getPath()));
+    assertEquals(pathMeta3, meta.get(pathMeta3.getFileStatus().getPath()));
+    assertNull(meta.get(new Path(path, "notfound")));
+  }
+
+  @Test
+  public void testGetNull() {
+    Path path = new Path("/path");
+    DirListingMetadata meta = new DirListingMetadata(path, null, false);
+    exception.expect(NullPointerException.class);
+    exception.expectMessage(notNullValue(String.class));
+    meta.get(null);
+  }
+
+  @Test
+  public void testGetRoot() {
+    Path path = new Path("/path");
+    DirListingMetadata meta = new DirListingMetadata(path, null, false);
+    exception.expect(IllegalArgumentException.class);
+    exception.expectMessage(notNullValue(String.class));
+    meta.get(new Path("/"));
+  }
+
+  @Test
+  public void testGetNotChild() {
+    Path path = new Path("/path");
+    DirListingMetadata meta = new DirListingMetadata(path, null, false);
+    exception.expect(IllegalArgumentException.class);
+    exception.expectMessage(notNullValue(String.class));
+    meta.get(new Path("/different/ancestor"));
+  }
+
+  @Test
+  public void testPut() {
+    Path path = new Path("/path");
+    PathMetadata pathMeta1 = new PathMetadata(
+        new S3AFileStatus(true, new Path(path, "dir1"), TEST_OWNER));
+    PathMetadata pathMeta2 = new PathMetadata(
+        new S3AFileStatus(true, new Path(path, "dir2"), TEST_OWNER));
+    PathMetadata pathMeta3 = new PathMetadata(
+        new S3AFileStatus(123, 456, new Path(path, "file1"), 8192, TEST_OWNER));
+    List<PathMetadata> listing = Arrays.asList(pathMeta1, pathMeta2, pathMeta3);
+    DirListingMetadata meta = new DirListingMetadata(path, listing, false);
+    assertEquals(path, meta.getPath());
+    assertNotNull(meta.getListing());
+    assertFalse(meta.getListing().isEmpty());
+    assertTrue(meta.getListing().contains(pathMeta1));
+    assertTrue(meta.getListing().contains(pathMeta2));
+    assertTrue(meta.getListing().contains(pathMeta3));
+    assertFalse(meta.isAuthoritative());
+    PathMetadata pathMeta4 = new PathMetadata(
+        new S3AFileStatus(true, new Path(path, "dir3"), TEST_OWNER));
+    meta.put(pathMeta4.getFileStatus());
+    assertTrue(meta.getListing().contains(pathMeta4));
+    assertEquals(pathMeta4, meta.get(pathMeta4.getFileStatus().getPath()));
+  }
+
+  @Test
+  public void testPutNull() {
+    Path path = new Path("/path");
+    DirListingMetadata meta = new DirListingMetadata(path, null, false);
+    exception.expect(NullPointerException.class);
+    exception.expectMessage(notNullValue(String.class));
+    meta.put(null);
+  }
+
+  @Test
+  public void testPutNullPath() {
+    Path path = new Path("/path");
+    DirListingMetadata meta = new DirListingMetadata(path, null, false);
+    exception.expect(NullPointerException.class);
+    exception.expectMessage(notNullValue(String.class));
+    meta.put(new S3AFileStatus(true, null, TEST_OWNER));
+  }
+
+  @Test
+  public void testPutRoot() {
+    Path path = new Path("/path");
+    DirListingMetadata meta = new DirListingMetadata(path, null, false);
+    exception.expect(IllegalArgumentException.class);
+    exception.expectMessage(notNullValue(String.class));
+    meta.put(new S3AFileStatus(true, new Path("/"), TEST_OWNER));
+  }
+
+  @Test
+  public void testPutNotChild() {
+    Path path = new Path("/path");
+    DirListingMetadata meta = new DirListingMetadata(path, null, false);
+    exception.expect(IllegalArgumentException.class);
+    exception.expectMessage(notNullValue(String.class));
+    meta.put(new S3AFileStatus(true, new Path("/different/ancestor"),
+        TEST_OWNER));
+  }
+
+  @Test
+  public void testRemove() {
+    Path path = new Path("/path");
+    PathMetadata pathMeta1 = new PathMetadata(
+        new S3AFileStatus(true, new Path(path, "dir1"), TEST_OWNER));
+    PathMetadata pathMeta2 = new PathMetadata(
+        new S3AFileStatus(true, new Path(path, "dir2"), TEST_OWNER));
+    PathMetadata pathMeta3 = new PathMetadata(
+        new S3AFileStatus(123, 456, new Path(path, "file1"), 8192, TEST_OWNER));
+    List<PathMetadata> listing = Arrays.asList(pathMeta1, pathMeta2, pathMeta3);
+    DirListingMetadata meta = new DirListingMetadata(path, listing, false);
+    assertEquals(path, meta.getPath());
+    assertNotNull(meta.getListing());
+    assertFalse(meta.getListing().isEmpty());
+    assertTrue(meta.getListing().contains(pathMeta1));
+    assertTrue(meta.getListing().contains(pathMeta2));
+    assertTrue(meta.getListing().contains(pathMeta3));
+    assertFalse(meta.isAuthoritative());
+    meta.remove(pathMeta1.getFileStatus().getPath());
+    assertFalse(meta.getListing().contains(pathMeta1));
+    assertNull(meta.get(pathMeta1.getFileStatus().getPath()));
+  }
+
+  @Test
+  public void testRemoveNull() {
+    Path path = new Path("/path");
+    DirListingMetadata meta = new DirListingMetadata(path, null, false);
+    exception.expect(NullPointerException.class);
+    exception.expectMessage(notNullValue(String.class));
+    meta.remove(null);
+  }
+
+  @Test
+  public void testRemoveRoot() {
+    Path path = new Path("/path");
+    DirListingMetadata meta = new DirListingMetadata(path, null, false);
+    exception.expect(IllegalArgumentException.class);
+    exception.expectMessage(notNullValue(String.class));
+    meta.remove(new Path("/"));
+  }
+
+  @Test
+  public void testRemoveNotChild() {
+    Path path = new Path("/path");
+    DirListingMetadata meta = new DirListingMetadata(path, null, false);
+    exception.expect(IllegalArgumentException.class);
+    exception.expectMessage(notNullValue(String.class));
+    meta.remove(new Path("/different/ancestor"));
+  }
+
+  /*
+   * Create DirListingMetadata with two dirs and one file living in directory
+   * 'parent'
+   */
+  private static DirListingMetadata makeTwoDirsOneFile(Path parent) {
+    PathMetadata pathMeta1 = new PathMetadata(
+        new S3AFileStatus(true, new Path(parent, "dir1"), TEST_OWNER));
+    PathMetadata pathMeta2 = new PathMetadata(
+        new S3AFileStatus(true, new Path(parent, "dir2"), TEST_OWNER));
+    PathMetadata pathMeta3 = new PathMetadata(
+        new S3AFileStatus(123, 456, new Path(parent, "file1"), 8192,
+            TEST_OWNER));
+    List<PathMetadata> listing = Arrays.asList(pathMeta1, pathMeta2, pathMeta3);
+    return new DirListingMetadata(parent, listing, false);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestDynamoDBMetadataStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestDynamoDBMetadataStore.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestDynamoDBMetadataStore.java
new file mode 100644
index 0000000..02eb7b8
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestDynamoDBMetadataStore.java
@@ -0,0 +1,594 @@
+/*
+ * 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.net.URI;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import com.amazonaws.AmazonServiceException;
+import com.amazonaws.services.dynamodbv2.document.DynamoDB;
+import com.amazonaws.services.dynamodbv2.document.Item;
+import com.amazonaws.services.dynamodbv2.document.PrimaryKey;
+import com.amazonaws.services.dynamodbv2.document.Table;
+import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughputDescription;
+import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException;
+import com.amazonaws.services.dynamodbv2.model.TableDescription;
+
+import com.google.common.collect.Lists;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.hadoop.fs.s3a.Tristate;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.MockS3ClientFactory;
+import org.apache.hadoop.fs.s3a.S3AFileStatus;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.S3ClientFactory;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import static org.apache.hadoop.fs.s3a.Constants.*;
+import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.*;
+import static org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStore.*;
+import static org.apache.hadoop.test.LambdaTestUtils.*;
+
+/**
+ * Test that {@link DynamoDBMetadataStore} implements {@link MetadataStore}.
+ *
+ * In this unit test, we use an in-memory DynamoDBLocal server instead of real
+ * AWS DynamoDB. An {@link S3AFileSystem} object is created and shared for
+ * initializing {@link DynamoDBMetadataStore} objects.  There are no real S3
+ * request issued as the underlying AWS S3Client is mocked.  You won't be
+ * charged bills for AWS S3 or DynamoDB when you run this test.
+ *
+ * According to the base class, every test case will have independent contract
+ * to create a new {@link DynamoDBMetadataStore} instance and initializes it.
+ * A table will be created for each test by the test contract, and will be
+ * destroyed after the test case finishes.
+ */
+public class TestDynamoDBMetadataStore extends MetadataStoreTestBase {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestDynamoDBMetadataStore.class);
+  private static final String BUCKET = "TestDynamoDBMetadataStore";
+  private static final String S3URI =
+      URI.create(FS_S3A + "://" + BUCKET + "/").toString();
+  public static final PrimaryKey
+      VERSION_MARKER_PRIMARY_KEY = createVersionMarkerPrimaryKey(
+      DynamoDBMetadataStore.VERSION_MARKER);
+
+  /** The DynamoDB instance that can issue requests directly to server. */
+  private static DynamoDB dynamoDB;
+
+  @Rule
+  public final Timeout timeout = new Timeout(60 * 1000);
+
+  /**
+   * Start the in-memory DynamoDBLocal server and initializes s3 file system.
+   */
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    DynamoDBLocalClientFactory.startSingletonServer();
+    try {
+      dynamoDB = new DynamoDBMSContract().getMetadataStore().getDynamoDB();
+    } catch (AmazonServiceException e) {
+      final String msg = "Cannot initialize a DynamoDBMetadataStore instance "
+          + "against the local DynamoDB server. Perhaps the DynamoDBLocal "
+          + "server is not configured correctly. ";
+      LOG.error(msg, e);
+      // fail fast if the DynamoDBLocal server can not work
+      throw e;
+    }
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    if (dynamoDB != null) {
+      dynamoDB.shutdown();
+    }
+    DynamoDBLocalClientFactory.stopSingletonServer();
+  }
+
+  /**
+   * Each contract has its own S3AFileSystem and DynamoDBMetadataStore objects.
+   */
+  private static class DynamoDBMSContract extends AbstractMSContract {
+    private final S3AFileSystem s3afs;
+    private final DynamoDBMetadataStore ms = new DynamoDBMetadataStore();
+
+    DynamoDBMSContract() throws IOException {
+        this(new Configuration());
+    }
+
+    DynamoDBMSContract(Configuration conf) throws IOException {
+      // using mocked S3 clients
+      conf.setClass(S3_CLIENT_FACTORY_IMPL, MockS3ClientFactory.class,
+          S3ClientFactory.class);
+      conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, S3URI);
+      // setting config for creating a DynamoDBClient against local server
+      conf.set(ACCESS_KEY, "dummy-access-key");
+      conf.set(SECRET_KEY, "dummy-secret-key");
+      conf.setBoolean(S3GUARD_DDB_TABLE_CREATE_KEY, true);
+      conf.setClass(S3Guard.S3GUARD_DDB_CLIENT_FACTORY_IMPL,
+          DynamoDBLocalClientFactory.class, DynamoDBClientFactory.class);
+
+      // always create new file system object for a test contract
+      s3afs = (S3AFileSystem) FileSystem.newInstance(conf);
+      ms.initialize(s3afs);
+    }
+
+    @Override
+    public S3AFileSystem getFileSystem() {
+      return s3afs;
+    }
+
+    @Override
+    public DynamoDBMetadataStore getMetadataStore() {
+      return ms;
+    }
+  }
+
+  @Override
+  public DynamoDBMSContract createContract() throws IOException {
+    return new DynamoDBMSContract();
+  }
+
+  @Override
+  public DynamoDBMSContract createContract(Configuration conf) throws
+      IOException {
+    return new DynamoDBMSContract(conf);
+  }
+
+  @Override
+  FileStatus basicFileStatus(Path path, int size, boolean isDir)
+      throws IOException {
+    String owner = UserGroupInformation.getCurrentUser().getShortUserName();
+    return isDir
+        ? new S3AFileStatus(true, path, owner)
+        : new S3AFileStatus(size, getModTime(), path, BLOCK_SIZE, owner);
+  }
+
+  private DynamoDBMetadataStore getDynamoMetadataStore() throws IOException {
+    return (DynamoDBMetadataStore) getContract().getMetadataStore();
+  }
+
+  private S3AFileSystem getFileSystem() throws IOException {
+    return (S3AFileSystem) getContract().getFileSystem();
+  }
+
+  /**
+   * This tests that after initialize() using an S3AFileSystem object, the
+   * instance should have been initialized successfully, and tables are ACTIVE.
+   */
+  @Test
+  public void testInitialize() throws IOException {
+    final String tableName = "testInitializeWithFileSystem";
+    final S3AFileSystem s3afs = getFileSystem();
+    final Configuration conf = s3afs.getConf();
+    conf.set(S3GUARD_DDB_TABLE_NAME_KEY, tableName);
+    try (DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore()) {
+      ddbms.initialize(s3afs);
+      verifyTableInitialized(tableName);
+      assertNotNull(ddbms.getTable());
+      assertEquals(tableName, ddbms.getTable().getTableName());
+      String expectedRegion = conf.get(S3GUARD_DDB_REGION_KEY,
+          s3afs.getBucketLocation(tableName));
+      assertEquals("DynamoDB table should be in configured region or the same" +
+              " region as S3 bucket",
+          expectedRegion,
+          ddbms.getRegion());
+    }
+  }
+
+  /**
+   * This tests that after initialize() using a Configuration object, the
+   * instance should have been initialized successfully, and tables are ACTIVE.
+   */
+  @Test
+  public void testInitializeWithConfiguration() throws IOException {
+    final String tableName = "testInitializeWithConfiguration";
+    final Configuration conf = getFileSystem().getConf();
+    conf.unset(S3GUARD_DDB_TABLE_NAME_KEY);
+    String savedRegion = conf.get(S3GUARD_DDB_REGION_KEY,
+        getFileSystem().getBucketLocation());
+    conf.unset(S3GUARD_DDB_REGION_KEY);
+    try (DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore()) {
+      ddbms.initialize(conf);
+      fail("Should have failed because the table name is not set!");
+    } catch (IllegalArgumentException ignored) {
+    }
+    // config table name
+    conf.set(S3GUARD_DDB_TABLE_NAME_KEY, tableName);
+    try (DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore()) {
+      ddbms.initialize(conf);
+      fail("Should have failed because as the region is not set!");
+    } catch (IllegalArgumentException ignored) {
+    }
+    // config region
+    conf.set(S3GUARD_DDB_REGION_KEY, savedRegion);
+    try (DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore()) {
+      ddbms.initialize(conf);
+      verifyTableInitialized(tableName);
+      assertNotNull(ddbms.getTable());
+      assertEquals(tableName, ddbms.getTable().getTableName());
+      assertEquals("Unexpected key schema found!",
+          keySchema(),
+          ddbms.getTable().describe().getKeySchema());
+    }
+  }
+
+  /**
+   * Test that for a large batch write request, the limit is handled correctly.
+   */
+  @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
+    };
+    for (int numOldMetas : numMetasToDeleteOrPut) {
+      for (int numNewMetas : numMetasToDeleteOrPut) {
+        doTestBatchWrite(numOldMetas, numNewMetas);
+      }
+    }
+  }
+
+  private void doTestBatchWrite(int numDelete, int numPut) throws IOException {
+    final String root = S3URI + "/testBatchWrite_" + numDelete + '_' + numPut;
+    final Path oldDir = new Path(root, "oldDir");
+    final Path newDir = new Path(root, "newDir");
+    LOG.info("doTestBatchWrite: oldDir={}, newDir={}", oldDir, newDir);
+
+    DynamoDBMetadataStore ms = getDynamoMetadataStore();
+    ms.put(new PathMetadata(basicFileStatus(oldDir, 0, true)));
+    ms.put(new PathMetadata(basicFileStatus(newDir, 0, true)));
+
+    final List<PathMetadata> oldMetas =
+        numDelete < 0 ? null : new ArrayList<PathMetadata>(numDelete);
+    for (int i = 0; i < numDelete; i++) {
+      oldMetas.add(new PathMetadata(
+          basicFileStatus(new Path(oldDir, "child" + i), i, true)));
+    }
+    final List<PathMetadata> newMetas =
+        numPut < 0 ? null : new ArrayList<PathMetadata>(numPut);
+    for (int i = 0; i < numPut; i++) {
+      newMetas.add(new PathMetadata(
+          basicFileStatus(new Path(newDir, "child" + i), i, false)));
+    }
+
+    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());
+      assertTrue(CollectionUtils.isEqualCollection(oldMetas,
+          ms.listChildren(oldDir).getListing()));
+
+      pathsToDelete = new ArrayList<>(oldMetas.size());
+      for (PathMetadata meta : oldMetas) {
+        pathsToDelete.add(meta.getFileStatus().getPath());
+      }
+    }
+
+    // move the old paths to new paths and verify
+    ms.move(pathsToDelete, newMetas);
+    assertEquals(0, ms.listChildren(oldDir).withoutTombstones().numEntries());
+    if (newMetas != null) {
+      assertTrue(CollectionUtils.isEqualCollection(newMetas,
+          ms.listChildren(newDir).getListing()));
+    }
+  }
+
+  @Test
+  public void testInitExistingTable() throws IOException {
+    final DynamoDBMetadataStore ddbms = getDynamoMetadataStore();
+    final String tableName = ddbms.getTable().getTableName();
+    verifyTableInitialized(tableName);
+    // create existing table
+    ddbms.initTable();
+    verifyTableInitialized(tableName);
+  }
+
+  /**
+   * Test the low level version check code.
+   */
+  @Test
+  public void testItemVersionCompatibility() throws Throwable {
+    verifyVersionCompatibility("table",
+        createVersionMarker(VERSION_MARKER, VERSION, 0));
+  }
+
+  /**
+   * Test that a version marker entry without the version number field
+   * is rejected as incompatible with a meaningful error message.
+   */
+  @Test
+  public void testItemLacksVersion() throws Throwable {
+    intercept(IOException.class, E_NOT_VERSION_MARKER,
+        new VoidCallable() {
+          @Override
+          public void call() throws Exception {
+            verifyVersionCompatibility("table",
+                new Item().withPrimaryKey(
+                    createVersionMarkerPrimaryKey(VERSION_MARKER)));
+          }
+        });
+  }
+
+  /**
+   * Delete the version marker and verify that table init fails.
+   */
+  @Test
+  public void testTableVersionRequired() throws Exception {
+    Configuration conf = getFileSystem().getConf();
+    int maxRetries = conf.getInt(S3GUARD_DDB_MAX_RETRIES,
+        S3GUARD_DDB_MAX_RETRIES_DEFAULT);
+    conf.setInt(S3GUARD_DDB_MAX_RETRIES, 3);
+
+    final DynamoDBMetadataStore ddbms = createContract(conf).getMetadataStore();
+    String tableName = conf.get(S3GUARD_DDB_TABLE_NAME_KEY, BUCKET);
+    Table table = verifyTableInitialized(tableName);
+    table.deleteItem(VERSION_MARKER_PRIMARY_KEY);
+
+    // create existing table
+    intercept(IOException.class, E_NO_VERSION_MARKER,
+        new VoidCallable() {
+          @Override
+          public void call() throws Exception {
+            ddbms.initTable();
+          }
+        });
+
+    conf.setInt(S3GUARD_DDB_MAX_RETRIES, maxRetries);
+  }
+
+  /**
+   * Set the version value to a different number and verify that
+   * table init fails.
+   */
+  @Test
+  public void testTableVersionMismatch() throws Exception {
+    final DynamoDBMetadataStore ddbms = createContract().getMetadataStore();
+    String tableName = getFileSystem().getConf()
+        .get(S3GUARD_DDB_TABLE_NAME_KEY, BUCKET);
+    Table table = verifyTableInitialized(tableName);
+    table.deleteItem(VERSION_MARKER_PRIMARY_KEY);
+    Item v200 = createVersionMarker(VERSION_MARKER, 200, 0);
+    table.putItem(v200);
+
+    // create existing table
+    intercept(IOException.class, E_INCOMPATIBLE_VERSION,
+        new VoidCallable() {
+          @Override
+          public void call() throws Exception {
+            ddbms.initTable();
+          }
+        });
+  }
+
+  /**
+   * Test that initTable fails with IOException when table does not exist and
+   * table auto-creation is disabled.
+   */
+  @Test
+  public void testFailNonexistentTable() throws IOException {
+    final String tableName = "testFailNonexistentTable";
+    final S3AFileSystem s3afs = getFileSystem();
+    final Configuration conf = s3afs.getConf();
+    conf.set(S3GUARD_DDB_TABLE_NAME_KEY, tableName);
+    conf.unset(S3GUARD_DDB_TABLE_CREATE_KEY);
+    try (DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore()) {
+      ddbms.initialize(s3afs);
+      fail("Should have failed as table does not exist and table auto-creation"
+          + " is disabled");
+    } catch (IOException ignored) {
+    }
+  }
+
+  /**
+   * Test cases about root directory as it is not in the DynamoDB table.
+   */
+  @Test
+  public void testRootDirectory() throws IOException {
+    final DynamoDBMetadataStore ddbms = getDynamoMetadataStore();
+    Path rootPath = new Path(S3URI);
+    verifyRootDirectory(ddbms.get(rootPath), true);
+
+    ddbms.put(new PathMetadata(new S3AFileStatus(true,
+        new Path(rootPath, "foo"),
+        UserGroupInformation.getCurrentUser().getShortUserName())));
+    verifyRootDirectory(ddbms.get(new Path(S3URI)), false);
+  }
+
+  private void verifyRootDirectory(PathMetadata rootMeta, boolean isEmpty) {
+    assertNotNull(rootMeta);
+    final FileStatus status = rootMeta.getFileStatus();
+    assertNotNull(status);
+    assertTrue(status.isDirectory());
+    // UNKNOWN is always a valid option, but true / false should not contradict
+    if (isEmpty) {
+      assertNotSame("Should not be marked non-empty",
+          Tristate.FALSE,
+          rootMeta.isEmptyDirectory());
+    } else {
+      assertNotSame("Should not be marked empty",
+          Tristate.TRUE,
+          rootMeta.isEmptyDirectory());
+    }
+  }
+
+  /**
+   * Test that when moving nested paths, all its ancestors up to destination
+   * root will also be created.
+   * Here is the directory tree before move:
+   * <pre>
+   * testMovePopulateAncestors
+   * ├── a
+   * │   └── b
+   * │       └── src
+   * │           ├── dir1
+   * │           │   └── dir2
+   * │           └── file1.txt
+   * └── c
+   *     └── d
+   *         └── dest
+   *</pre>
+   * As part of rename(a/b/src, d/c/dest), S3A will enumerate the subtree at
+   * a/b/src.  This test verifies that after the move, the new subtree at
+   * 'dest' is reachable from the root (i.e. c/ and c/d exist in the table.
+   * DynamoDBMetadataStore depends on this property to do recursive delete
+   * without a full table scan.
+   */
+  @Test
+  public void testMovePopulatesAncestors() throws IOException {
+    final DynamoDBMetadataStore ddbms = getDynamoMetadataStore();
+    final String testRoot = "/testMovePopulatesAncestors";
+    final String srcRoot = testRoot + "/a/b/src";
+    final String destRoot = testRoot + "/c/d/e/dest";
+
+    final Path nestedPath1 = strToPath(srcRoot + "/file1.txt");
+    ddbms.put(new PathMetadata(basicFileStatus(nestedPath1, 1024, false)));
+    final Path nestedPath2 = strToPath(srcRoot + "/dir1/dir2");
+    ddbms.put(new PathMetadata(basicFileStatus(nestedPath2, 0, true)));
+
+    // We don't put the destRoot path here, since put() would create ancestor
+    // entries, and we want to ensure that move() does it, instead.
+
+    // Build enumeration of src / dest paths and do the move()
+    final Collection<Path> fullSourcePaths = Lists.newArrayList(
+        strToPath(srcRoot),
+        strToPath(srcRoot + "/dir1"),
+        strToPath(srcRoot + "/dir1/dir2"),
+        strToPath(srcRoot + "/file1.txt")
+    );
+    final Collection<PathMetadata> pathsToCreate = Lists.newArrayList(
+        new PathMetadata(basicFileStatus(strToPath(destRoot),
+            0, true)),
+        new PathMetadata(basicFileStatus(strToPath(destRoot + "/dir1"),
+            0, true)),
+        new PathMetadata(basicFileStatus(strToPath(destRoot + "/dir1/dir2"),
+            0, true)),
+        new PathMetadata(basicFileStatus(strToPath(destRoot + "/file1.txt"),
+            1024, false))
+    );
+
+    ddbms.move(fullSourcePaths, pathsToCreate);
+
+    // 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 */);
+
+    // Also check moved files while we're at it
+    assertCached(destRoot + "/dir1");
+    assertCached(destRoot + "/dir1/dir2");
+    assertCached(destRoot + "/file1.txt");
+  }
+
+  @Test
+  public void testProvisionTable() throws IOException {
+    final DynamoDBMetadataStore ddbms = getDynamoMetadataStore();
+    final String tableName = ddbms.getTable().getTableName();
+    final ProvisionedThroughputDescription oldProvision =
+        dynamoDB.getTable(tableName).describe().getProvisionedThroughput();
+    ddbms.provisionTable(oldProvision.getReadCapacityUnits() * 2,
+        oldProvision.getWriteCapacityUnits() * 2);
+    final ProvisionedThroughputDescription newProvision =
+        dynamoDB.getTable(tableName).describe().getProvisionedThroughput();
+    LOG.info("Old provision = {}, new provision = {}",
+        oldProvision, newProvision);
+    assertEquals(oldProvision.getReadCapacityUnits() * 2,
+        newProvision.getReadCapacityUnits().longValue());
+    assertEquals(oldProvision.getWriteCapacityUnits() * 2,
+        newProvision.getWriteCapacityUnits().longValue());
+  }
+
+  @Test
+  public void testDeleteTable() throws IOException {
+    final String tableName = "testDeleteTable";
+    final S3AFileSystem s3afs = getFileSystem();
+    final Configuration conf = s3afs.getConf();
+    conf.set(S3GUARD_DDB_TABLE_NAME_KEY, tableName);
+    try (DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore()) {
+      ddbms.initialize(s3afs);
+      // we can list the empty table
+      ddbms.listChildren(new Path(S3URI));
+
+      ddbms.destroy();
+      verifyTableNotExist(tableName);
+
+      // delete table once more; be ResourceNotFoundException swallowed silently
+      ddbms.destroy();
+      verifyTableNotExist(tableName);
+
+      try {
+        // we can no longer list the destroyed table
+        ddbms.listChildren(new Path(S3URI));
+        fail("Should have failed after the table is destroyed!");
+      } catch (IOException ignored) {
+      }
+    }
+  }
+
+  /**
+   * This validates the table is created and ACTIVE in DynamoDB.
+   *
+   * This should not rely on the {@link DynamoDBMetadataStore} implementation.
+   * Return the table
+   */
+  private static Table verifyTableInitialized(String tableName) {
+    final Table table = dynamoDB.getTable(tableName);
+    final TableDescription td = table.describe();
+    assertEquals(tableName, td.getTableName());
+    assertEquals("ACTIVE", td.getTableStatus());
+    return table;
+  }
+
+  /**
+   * This validates the table is not found in DynamoDB.
+   *
+   * This should not rely on the {@link DynamoDBMetadataStore} implementation.
+   */
+  private static void verifyTableNotExist(String tableName) {
+    final Table table = dynamoDB.getTable(tableName);
+    try {
+      table.describe();
+      fail("Expecting ResourceNotFoundException for table '" + tableName + "'");
+    } catch (ResourceNotFoundException ignored) {
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestLocalMetadataStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestLocalMetadataStore.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestLocalMetadataStore.java
new file mode 100644
index 0000000..1b765af
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestLocalMetadataStore.java
@@ -0,0 +1,140 @@
+/*
+ * 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.HashMap;
+import java.util.Map;
+
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.S3ATestUtils;
+
+/**
+ * MetadataStore unit test for {@link LocalMetadataStore}.
+ */
+public class TestLocalMetadataStore extends MetadataStoreTestBase {
+
+  private static final String MAX_ENTRIES_STR = "16";
+
+  private final static class LocalMSContract extends AbstractMSContract {
+
+    private FileSystem fs;
+
+    private LocalMSContract() throws IOException {
+      this(new Configuration());
+    }
+
+    private LocalMSContract(Configuration config) throws IOException {
+      config.set(LocalMetadataStore.CONF_MAX_RECORDS, MAX_ENTRIES_STR);
+      fs = FileSystem.getLocal(config);
+    }
+
+    @Override
+    public FileSystem getFileSystem() {
+      return fs;
+    }
+
+    @Override
+    public MetadataStore getMetadataStore() throws IOException {
+      LocalMetadataStore lms = new LocalMetadataStore();
+      return lms;
+    }
+  }
+
+  @Override
+  public AbstractMSContract createContract() throws IOException {
+    return new LocalMSContract();
+  }
+
+  @Override
+  public AbstractMSContract createContract(Configuration conf) throws
+      IOException {
+    return new LocalMSContract(conf);
+  }
+
+  @Test
+  public void testClearByAncestor() {
+    Map<Path, PathMetadata> map = new HashMap<>();
+
+    // 1. Test paths without scheme/host
+    assertClearResult(map, "", "/", 0);
+    assertClearResult(map, "", "/dirA/dirB", 2);
+    assertClearResult(map, "", "/invalid", 5);
+
+
+    // 2. Test paths w/ scheme/host
+    String p = "s3a://fake-bucket-name";
+    assertClearResult(map, p, "/", 0);
+    assertClearResult(map, p, "/dirA/dirB", 2);
+    assertClearResult(map, p, "/invalid", 5);
+  }
+
+  private static void populateMap(Map<Path, PathMetadata> map,
+      String prefix) {
+    populateEntry(map, new Path(prefix + "/dirA/dirB/"));
+    populateEntry(map, new Path(prefix + "/dirA/dirB/dirC"));
+    populateEntry(map, new Path(prefix + "/dirA/dirB/dirC/file1"));
+    populateEntry(map, new Path(prefix + "/dirA/dirB/dirC/file2"));
+    populateEntry(map, new Path(prefix + "/dirA/file1"));
+  }
+
+  private static void populateEntry(Map<Path, PathMetadata> map,
+      Path path) {
+    map.put(path, new PathMetadata(new FileStatus(0, true, 0, 0, 0, path)));
+  }
+
+  private static int sizeOfMap(Map<Path, PathMetadata> map) {
+    int count = 0;
+    for (PathMetadata meta : map.values()) {
+      if (!meta.isDeleted()) {
+        count++;
+      }
+    }
+    return count;
+  }
+
+  private static void assertClearResult(Map <Path, PathMetadata> map,
+      String prefixStr, String pathStr, int leftoverSize) {
+    populateMap(map, prefixStr);
+    LocalMetadataStore.deleteHashByAncestor(new Path(prefixStr + pathStr), map,
+        true);
+    assertEquals(String.format("Map should have %d entries", leftoverSize),
+        leftoverSize, sizeOfMap(map));
+    map.clear();
+  }
+
+  @Override
+  protected void verifyFileStatus(FileStatus status, long size) {
+    S3ATestUtils.verifyFileStatus(status, size, REPLICATION, getModTime(),
+        getAccessTime(),
+        BLOCK_SIZE, OWNER, GROUP, PERMISSION);
+  }
+
+  @Override
+  protected void verifyDirStatus(FileStatus status) {
+    S3ATestUtils.verifyDirStatus(status, REPLICATION, getModTime(),
+        getAccessTime(), OWNER, GROUP, PERMISSION);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestNullMetadataStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestNullMetadataStore.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestNullMetadataStore.java
new file mode 100644
index 0000000..c0541ea
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestNullMetadataStore.java
@@ -0,0 +1,58 @@
+/*
+ * 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.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+
+import java.io.IOException;
+
+/**
+ * Run MetadataStore unit tests on the NullMetadataStore implementation.
+ */
+public class TestNullMetadataStore extends MetadataStoreTestBase {
+  private static class NullMSContract extends AbstractMSContract {
+    @Override
+    public FileSystem getFileSystem() throws IOException {
+      Configuration config = new Configuration();
+      return FileSystem.getLocal(config);
+    }
+
+    @Override
+    public MetadataStore getMetadataStore() throws IOException {
+      return new NullMetadataStore();
+    }
+  }
+
+  /** This MetadataStore always says "I don't know, ask the backing store". */
+  @Override
+  public boolean allowMissing() {
+    return true;
+  }
+
+  @Override
+  public AbstractMSContract createContract() {
+    return new NullMSContract();
+  }
+
+  @Override
+  public AbstractMSContract createContract(Configuration conf) {
+    return createContract();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestPathMetadataDynamoDBTranslation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestPathMetadataDynamoDBTranslation.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestPathMetadataDynamoDBTranslation.java
new file mode 100644
index 0000000..1678746
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestPathMetadataDynamoDBTranslation.java
@@ -0,0 +1,238 @@
+/*
+ * 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.net.URI;
+import java.util.Collection;
+import java.util.concurrent.Callable;
+
+import com.amazonaws.services.dynamodbv2.document.Item;
+import com.amazonaws.services.dynamodbv2.document.KeyAttribute;
+import com.amazonaws.services.dynamodbv2.document.PrimaryKey;
+import com.amazonaws.services.dynamodbv2.model.AttributeDefinition;
+import com.amazonaws.services.dynamodbv2.model.KeySchemaElement;
+import com.google.common.base.Preconditions;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.S3AFileStatus;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.test.LambdaTestUtils;
+
+import static com.amazonaws.services.dynamodbv2.model.KeyType.HASH;
+import static com.amazonaws.services.dynamodbv2.model.KeyType.RANGE;
+import static com.amazonaws.services.dynamodbv2.model.ScalarAttributeType.S;
+import static org.hamcrest.CoreMatchers.anyOf;
+import static org.hamcrest.CoreMatchers.is;
+
+import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.*;
+import static org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStore.VERSION_MARKER;
+import static org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStore.VERSION;
+
+/**
+ * Test the PathMetadataDynamoDBTranslation is able to translate between domain
+ * model objects and DynamoDB items.
+ */
+public class TestPathMetadataDynamoDBTranslation extends Assert {
+
+  private static final Path TEST_DIR_PATH = new Path("s3a://test-bucket/myDir");
+  private static final Item TEST_DIR_ITEM = new Item();
+  private static PathMetadata testDirPathMetadata;
+
+  private static final long TEST_FILE_LENGTH = 100;
+  private static final long TEST_MOD_TIME = 9999;
+  private static final long TEST_BLOCK_SIZE = 128;
+  private static final Path TEST_FILE_PATH = new Path(TEST_DIR_PATH, "myFile");
+  private static final Item TEST_FILE_ITEM = new Item();
+  private static PathMetadata testFilePathMetadata;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws IOException {
+    String username = UserGroupInformation.getCurrentUser().getShortUserName();
+
+    testDirPathMetadata =
+        new PathMetadata(new S3AFileStatus(false, TEST_DIR_PATH, username));
+    TEST_DIR_ITEM
+        .withPrimaryKey(PARENT, "/test-bucket", CHILD, TEST_DIR_PATH.getName())
+        .withBoolean(IS_DIR, true);
+
+    testFilePathMetadata = new PathMetadata(
+        new S3AFileStatus(TEST_FILE_LENGTH, TEST_MOD_TIME, TEST_FILE_PATH,
+            TEST_BLOCK_SIZE, username));
+    TEST_FILE_ITEM
+        .withPrimaryKey(PARENT, pathToParentKey(TEST_FILE_PATH.getParent()),
+            CHILD, TEST_FILE_PATH.getName())
+        .withBoolean(IS_DIR, false)
+        .withLong(FILE_LENGTH, TEST_FILE_LENGTH)
+        .withLong(MOD_TIME, TEST_MOD_TIME)
+        .withLong(BLOCK_SIZE, TEST_BLOCK_SIZE);
+  }
+
+  /**
+   * It should not take long time as it doesn't involve remote server operation.
+   */
+  @Rule
+  public final Timeout timeout = new Timeout(30 * 1000);
+
+  @Test
+  public void testKeySchema() {
+    final Collection<KeySchemaElement> keySchema =
+        PathMetadataDynamoDBTranslation.keySchema();
+    assertNotNull(keySchema);
+    assertEquals("There should be HASH and RANGE key in key schema",
+        2, keySchema.size());
+    for (KeySchemaElement element : keySchema) {
+      assertThat(element.getAttributeName(), anyOf(is(PARENT), is(CHILD)));
+      assertThat(element.getKeyType(),
+          anyOf(is(HASH.toString()), is(RANGE.toString())));
+    }
+  }
+
+  @Test
+  public void testAttributeDefinitions() {
+    final Collection<AttributeDefinition> attrs =
+        PathMetadataDynamoDBTranslation.attributeDefinitions();
+    assertNotNull(attrs);
+    assertEquals("There should be HASH and RANGE attributes", 2, attrs.size());
+    for (AttributeDefinition definition : attrs) {
+      assertThat(definition.getAttributeName(), anyOf(is(PARENT), is(CHILD)));
+      assertEquals(S.toString(), definition.getAttributeType());
+    }
+  }
+
+  @Test
+  public void testItemToPathMetadata() throws IOException {
+    final String user =
+        UserGroupInformation.getCurrentUser().getShortUserName();
+    assertNull(itemToPathMetadata(null, user));
+
+    verify(TEST_DIR_ITEM, itemToPathMetadata(TEST_DIR_ITEM, user));
+    verify(TEST_FILE_ITEM, itemToPathMetadata(TEST_FILE_ITEM, user));
+  }
+
+  /**
+   * Verify that the Item and PathMetadata objects hold the same information.
+   */
+  private static void verify(Item item, PathMetadata meta) {
+    assertNotNull(meta);
+    final FileStatus status = meta.getFileStatus();
+    final Path path = status.getPath();
+    assertEquals(item.get(PARENT), pathToParentKey(path.getParent()));
+    assertEquals(item.get(CHILD), path.getName());
+    boolean isDir = item.hasAttribute(IS_DIR) && item.getBoolean(IS_DIR);
+    assertEquals(isDir, status.isDirectory());
+    long len = item.hasAttribute(FILE_LENGTH) ? item.getLong(FILE_LENGTH) : 0;
+    assertEquals(len, status.getLen());
+    long bSize = item.hasAttribute(BLOCK_SIZE) ? item.getLong(BLOCK_SIZE) : 0;
+    assertEquals(bSize, status.getBlockSize());
+
+    /*
+     * S3AFileStatue#getModificationTime() reports the current time, so the
+     * following assertion is failing.
+     *
+     * long modTime = item.hasAttribute(MOD_TIME) ? item.getLong(MOD_TIME) : 0;
+     * assertEquals(modTime, status.getModificationTime());
+     */
+  }
+
+  @Test
+  public void testPathMetadataToItem() {
+    verify(pathMetadataToItem(testDirPathMetadata), testDirPathMetadata);
+    verify(pathMetadataToItem(testFilePathMetadata),
+        testFilePathMetadata);
+  }
+
+  @Test
+  public void testPathToParentKeyAttribute() {
+    doTestPathToParentKeyAttribute(TEST_DIR_PATH);
+    doTestPathToParentKeyAttribute(TEST_FILE_PATH);
+  }
+
+  private static void doTestPathToParentKeyAttribute(Path path) {
+    final KeyAttribute attr = pathToParentKeyAttribute(path);
+    assertNotNull(attr);
+    assertEquals(PARENT, attr.getName());
+    // this path is expected as parent filed
+    assertEquals(pathToParentKey(path), attr.getValue());
+  }
+
+  private static String pathToParentKey(Path p) {
+    Preconditions.checkArgument(p.isUriPathAbsolute());
+    URI parentUri = p.toUri();
+    String bucket = parentUri.getHost();
+    Preconditions.checkNotNull(bucket);
+    String s =  "/" + bucket + parentUri.getPath();
+    // strip trailing slash
+    if (s.endsWith("/")) {
+      s = s.substring(0, s.length()-1);
+    }
+    return s;
+  }
+
+  @Test
+  public void testPathToKey() throws Exception {
+    LambdaTestUtils.intercept(IllegalArgumentException.class,
+        new Callable<PrimaryKey>() {
+          @Override
+          public PrimaryKey call() throws Exception {
+            return pathToKey(new Path("/"));
+          }
+        });
+    doTestPathToKey(TEST_DIR_PATH);
+    doTestPathToKey(TEST_FILE_PATH);
+  }
+
+  private static void doTestPathToKey(Path path) {
+    final PrimaryKey key = pathToKey(path);
+    assertNotNull(key);
+    assertEquals("There should be both HASH and RANGE keys",
+        2, key.getComponents().size());
+
+    for (KeyAttribute keyAttribute : key.getComponents()) {
+      assertThat(keyAttribute.getName(), anyOf(is(PARENT), is(CHILD)));
+      if (PARENT.equals(keyAttribute.getName())) {
+        assertEquals(pathToParentKey(path.getParent()),
+            keyAttribute.getValue());
+      } else {
+        assertEquals(path.getName(), keyAttribute.getValue());
+      }
+    }
+  }
+
+  @Test
+  public void testVersionRoundTrip() throws Throwable {
+    final Item marker = createVersionMarker(VERSION_MARKER, VERSION, 0);
+    assertEquals("Extracted version from " + marker,
+        VERSION, extractVersionFromMarker(marker));
+  }
+
+  @Test
+  public void testVersionMarkerNotStatusIllegalPath() throws Throwable {
+    final Item marker = createVersionMarker(VERSION_MARKER, VERSION, 0);
+    assertNull("Path metadata fromfrom " + marker,
+        itemToPathMetadata(marker, "alice"));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestS3Guard.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestS3Guard.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestS3Guard.java
new file mode 100644
index 0000000..745e7aa
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestS3Guard.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.s3guard;
+
+import java.util.Arrays;
+import java.util.List;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * Tests for the {@link S3Guard} utility class.
+ */
+public class TestS3Guard extends Assert {
+
+  /**
+   * Basic test to ensure results from S3 and MetadataStore are merged
+   * correctly.
+   */
+  @Test
+  public void testDirListingUnion() throws Exception {
+    MetadataStore ms = new LocalMetadataStore();
+
+    Path dirPath = new Path("s3a://bucket/dir");
+
+    // Two files in metadata store listing
+    PathMetadata m1 = makePathMeta("s3a://bucket/dir/ms-file1", false);
+    PathMetadata m2 = makePathMeta("s3a://bucket/dir/ms-file2", false);
+    DirListingMetadata dirMeta = new DirListingMetadata(dirPath,
+        Arrays.asList(m1, m2), false);
+
+    // Two other files in s3
+    List<FileStatus> s3Listing = Arrays.asList(
+        makeFileStatus("s3a://bucket/dir/s3-file3", false),
+        makeFileStatus("s3a://bucket/dir/s3-file4", false)
+    );
+
+    FileStatus[] result = S3Guard.dirListingUnion(ms, dirPath, s3Listing,
+        dirMeta, false);
+
+    assertEquals("listing length", 4, result.length);
+    assertContainsPath(result, "s3a://bucket/dir/ms-file1");
+    assertContainsPath(result, "s3a://bucket/dir/ms-file2");
+    assertContainsPath(result, "s3a://bucket/dir/s3-file3");
+    assertContainsPath(result, "s3a://bucket/dir/s3-file4");
+  }
+
+  void assertContainsPath(FileStatus[] statuses, String pathStr) {
+    assertTrue("listing doesn't contain " + pathStr,
+        containsPath(statuses, pathStr));
+  }
+
+  boolean containsPath(FileStatus[] statuses, String pathStr) {
+    for (FileStatus s : statuses) {
+      if (s.getPath().toString().equals(pathStr)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  private PathMetadata makePathMeta(String pathStr, boolean isDir) {
+    return new PathMetadata(makeFileStatus(pathStr, isDir));
+  }
+
+  private FileStatus makeFileStatus(String pathStr, boolean isDir) {
+    Path p = new Path(pathStr);
+    if (isDir) {
+      return new FileStatus(0, true, 1, 1, System.currentTimeMillis(), p);
+    } else {
+      return new FileStatus(100, false, 1, 1, System.currentTimeMillis(), p);
+    }
+  }
+}


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


[7/8] hadoop git commit: HADOOP-13345 S3Guard: Improved Consistency for S3A. Contributed by: Chris Nauroth, Aaron Fabbri, Mingliang Liu, Lei (Eddy) Xu, Sean Mackrory, Steve Loughran and others.

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileStatus.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileStatus.java
index b0f08e3..be08afe 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileStatus.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileStatus.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.fs.Path;
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public class S3AFileStatus extends FileStatus {
-  private boolean isEmptyDirectory;
+  private Tristate isEmptyDirectory;
 
   /**
    * Create a directory status.
@@ -42,6 +42,18 @@ public class S3AFileStatus extends FileStatus {
   public S3AFileStatus(boolean isemptydir,
       Path path,
       String owner) {
+    this(Tristate.fromBool(isemptydir), path, owner);
+  }
+
+  /**
+   * Create a directory status.
+   * @param isemptydir is this an empty directory?
+   * @param path the path
+   * @param owner the owner
+   */
+  public S3AFileStatus(Tristate isemptydir,
+      Path path,
+      String owner) {
     super(0, true, 1, 0, 0, path);
     isEmptyDirectory = isemptydir;
     setOwner(owner);
@@ -59,12 +71,37 @@ public class S3AFileStatus extends FileStatus {
   public S3AFileStatus(long length, long modification_time, Path path,
       long blockSize, String owner) {
     super(length, false, 1, blockSize, modification_time, path);
-    isEmptyDirectory = false;
+    isEmptyDirectory = Tristate.FALSE;
     setOwner(owner);
     setGroup(owner);
   }
 
-  public boolean isEmptyDirectory() {
+  /**
+   * Convenience constructor for creating from a vanilla FileStatus plus
+   * an isEmptyDirectory flag.
+   * @param source FileStatus to convert to S3AFileStatus
+   * @param isEmptyDirectory TRUE/FALSE if known to be / not be an empty
+   *     directory, UNKNOWN if that information was not computed.
+   * @return a new S3AFileStatus
+   */
+  public static S3AFileStatus fromFileStatus(FileStatus source,
+      Tristate isEmptyDirectory) {
+    if (source.isDirectory()) {
+      return new S3AFileStatus(isEmptyDirectory, source.getPath(),
+          source.getOwner());
+    } else {
+      return new S3AFileStatus(source.getLen(), source.getModificationTime(),
+          source.getPath(), source.getBlockSize(), source.getOwner());
+    }
+  }
+
+
+  /**
+   * @return FALSE if status is not a directory, or its a dir, but known to
+   * not be empty.  TRUE if it is an empty directory.  UNKNOWN if it is a
+   * directory, but we have not computed whether or not it is empty.
+   */
+  public Tristate isEmptyDirectory() {
     return isEmptyDirectory;
   }
 
@@ -110,7 +147,7 @@ public class S3AFileStatus extends FileStatus {
   @Override
   public String toString() {
     return super.toString() +
-        String.format(" isEmptyDirectory=%s", isEmptyDirectory());
+        String.format(" isEmptyDirectory=%s", isEmptyDirectory().name());
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
index 9733179..1cce86a 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
@@ -25,12 +25,16 @@ import java.io.InputStream;
 import java.io.InterruptedIOException;
 import java.net.URI;
 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.Map;
-import java.util.concurrent.ExecutorService;
+import java.util.Set;
 import java.util.Objects;
+import java.util.concurrent.ExecutorService;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
@@ -92,6 +96,11 @@ import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.StorageStatistics;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.s3a.s3guard.DirListingMetadata;
+import org.apache.hadoop.fs.s3a.s3guard.MetadataStoreListFilesIterator;
+import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
+import org.apache.hadoop.fs.s3a.s3guard.PathMetadata;
+import org.apache.hadoop.fs.s3a.s3guard.S3Guard;
 import org.apache.hadoop.fs.s3native.S3xLoginHelper;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Progressable;
@@ -149,6 +158,8 @@ public class S3AFileSystem extends FileSystem {
   private long readAhead;
   private S3AInputPolicy inputPolicy;
   private final AtomicBoolean closed = new AtomicBoolean(false);
+  private MetadataStore metadataStore;
+  private boolean allowAuthoritative;
 
   // The maximum number of entries that can be deleted in any call to s3
   private static final int MAX_ENTRIES_TO_DELETE = 1000;
@@ -277,6 +288,10 @@ public class S3AFileSystem extends FileSystem {
       } else {
         LOG.debug("Using S3AOutputStream");
       }
+
+      metadataStore = S3Guard.getMetadataStore(this);
+      allowAuthoritative = conf.getBoolean(METADATASTORE_AUTHORITATIVE,
+          DEFAULT_METADATASTORE_AUTHORITATIVE);
     } catch (AmazonClientException e) {
       throw translateException("initializing ", new Path(name), e);
     }
@@ -388,12 +403,35 @@ public class S3AFileSystem extends FileSystem {
    * Returns the S3 client used by this filesystem.
    * @return AmazonS3Client
    */
-  @VisibleForTesting
   AmazonS3 getAmazonS3Client() {
     return s3;
   }
 
   /**
+   * Get the region of a bucket.
+   * @return the region in which a bucket is located
+   * @throws IOException on any failure.
+   */
+  public String getBucketLocation() throws IOException {
+    return getBucketLocation(bucket);
+  }
+
+  /**
+   * Get the region of a bucket.
+   * @param bucketName the name of the bucket
+   * @return the region in which a bucket is located
+   * @throws IOException on any failure.
+   */
+  public String getBucketLocation(String bucketName) throws IOException {
+    try {
+      return s3.getBucketLocation(bucketName);
+    } catch (AmazonClientException e) {
+      throw translateException("getBucketLocation()",
+          bucketName, e);
+    }
+  }
+
+  /**
    * Returns the read ahead range value used by this filesystem
    * @return
    */
@@ -457,7 +495,7 @@ public class S3AFileSystem extends FileSystem {
    * @return a key excluding the leading "/", or, if it is the root path, ""
    */
   @VisibleForTesting
-  String pathToKey(Path path) {
+  public String pathToKey(Path path) {
     if (!path.isAbsolute()) {
       path = new Path(workingDir, path);
     }
@@ -508,7 +546,7 @@ public class S3AFileSystem extends FileSystem {
    * @param path path to qualify
    * @return a qualified path.
    */
-  Path qualify(Path path) {
+  public Path qualify(Path path) {
     return path.makeQualified(uri, workingDir);
   }
 
@@ -578,7 +616,7 @@ public class S3AFileSystem extends FileSystem {
       boolean overwrite, int bufferSize, short replication, long blockSize,
       Progressable progress) throws IOException {
     String key = pathToKey(f);
-    S3AFileStatus status = null;
+    FileStatus status = null;
     try {
       // get the status or throw an FNFE
       status = getFileStatus(f);
@@ -705,8 +743,8 @@ public class S3AFileSystem extends FileSystem {
    * the description of the operation.
    * This operation throws an exception on any failure which needs to be
    * reported and downgraded to a failure. That is: if a rename
-   * @param src path to be renamed
-   * @param dst new path after rename
+   * @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
@@ -715,9 +753,12 @@ public class S3AFileSystem extends FileSystem {
    * @throws IOException on IO failure.
    * @throws AmazonClientException on failures inside the AWS SDK
    */
-  private boolean innerRename(Path src, Path dst)
+  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);
     incrementStatistic(INVOCATION_RENAME);
 
@@ -733,7 +774,7 @@ public class S3AFileSystem extends FileSystem {
 
     // get the source file status; this raises a FNFE if there is no source
     // file.
-    S3AFileStatus srcStatus = getFileStatus(src);
+    S3AFileStatus srcStatus = innerGetFileStatus(src, true);
 
     if (srcKey.equals(dstKey)) {
       LOG.debug("rename: src and dest refer to the same file or directory: {}",
@@ -745,7 +786,7 @@ public class S3AFileSystem extends FileSystem {
 
     S3AFileStatus dstStatus = null;
     try {
-      dstStatus = getFileStatus(dst);
+      dstStatus = innerGetFileStatus(dst, true);
       // if there is no destination entry, an exception is raised.
       // hence this code sequence can assume that there is something
       // at the end of the path; the only detail being what it is and
@@ -755,7 +796,7 @@ public class S3AFileSystem extends FileSystem {
           throw new RenameFailedException(src, dst,
               "source is a directory and dest is a file")
               .withExitCode(srcStatus.isFile());
-        } else if (!dstStatus.isEmptyDirectory()) {
+        } else if (dstStatus.isEmptyDirectory() != Tristate.TRUE) {
           throw new RenameFailedException(src, dst,
               "Destination is a non-empty directory")
               .withExitCode(false);
@@ -777,7 +818,8 @@ public class S3AFileSystem extends FileSystem {
       Path parent = dst.getParent();
       if (!pathToKey(parent).isEmpty()) {
         try {
-          S3AFileStatus dstParentStatus = getFileStatus(dst.getParent());
+          S3AFileStatus dstParentStatus = innerGetFileStatus(dst.getParent(),
+              false);
           if (!dstParentStatus.isDirectory()) {
             throw new RenameFailedException(src, dst,
                 "destination parent is not a directory");
@@ -789,9 +831,20 @@ public class S3AFileSystem extends FileSystem {
       }
     }
 
+    // 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();
       if (dstStatus != null && dstStatus.isDirectory()) {
         String newDstKey = dstKey;
         if (!newDstKey.endsWith("/")) {
@@ -800,9 +853,14 @@ public class S3AFileSystem extends FileSystem {
         String filename =
             srcKey.substring(pathToKey(src.getParent()).length()+1);
         newDstKey = newDstKey + filename;
-        copyFile(srcKey, newDstKey, srcStatus.getLen());
+        copyFile(srcKey, newDstKey, length);
+        S3Guard.addMoveFile(metadataStore, srcPaths, dstMetas, src,
+            keyToQualifiedPath(newDstKey), length, getDefaultBlockSize(dst),
+            username);
       } else {
         copyFile(srcKey, dstKey, srcStatus.getLen());
+        S3Guard.addMoveFile(metadataStore, srcPaths, dstMetas, src, dst,
+            length, getDefaultBlockSize(dst), username);
       }
       innerDelete(srcStatus, false);
     } else {
@@ -824,42 +882,66 @@ public class S3AFileSystem extends FileSystem {
       }
 
       List<DeleteObjectsRequest.KeyVersion> keysToDelete = new ArrayList<>();
-      if (dstStatus != null && dstStatus.isEmptyDirectory()) {
+      if (dstStatus != null && dstStatus.isEmptyDirectory() == Tristate.TRUE) {
         // delete unnecessary fake directory.
         keysToDelete.add(new DeleteObjectsRequest.KeyVersion(dstKey));
       }
 
-      ListObjectsRequest request = new ListObjectsRequest();
-      request.setBucketName(bucket);
-      request.setPrefix(srcKey);
-      request.setMaxKeys(maxKeys);
-
-      ObjectListing objects = listObjects(request);
-
-      while (true) {
-        for (S3ObjectSummary summary : objects.getObjectSummaries()) {
-          keysToDelete.add(
-              new DeleteObjectsRequest.KeyVersion(summary.getKey()));
-          String newDstKey =
-              dstKey + summary.getKey().substring(srcKey.length());
-          copyFile(summary.getKey(), newDstKey, summary.getSize());
-
-          if (keysToDelete.size() == MAX_ENTRIES_TO_DELETE) {
-            removeKeys(keysToDelete, true, false);
+      Path parentPath = keyToPath(srcKey);
+      RemoteIterator<LocatedFileStatus> iterator = listFilesAndEmptyDirectories(
+          parentPath, true);
+      while (iterator.hasNext()) {
+        LocatedFileStatus 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());
+        copyFile(key, newDstKey, length);
+
+        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);
           }
+          // Ancestor directories may not be listed, so we explicitly add them
+          S3Guard.addMoveAncestors(metadataStore, srcPaths, dstMetas,
+              keyToQualifiedPath(srcKey), childSrc, childDst, username);
         }
 
-        if (objects.isTruncated()) {
-          objects = continueListObjects(objects);
-        } else {
-          if (!keysToDelete.isEmpty()) {
-            removeKeys(keysToDelete, false, false);
-          }
-          break;
+        if (keysToDelete.size() == MAX_ENTRIES_TO_DELETE) {
+          removeKeys(keysToDelete, true, false);
         }
       }
+      if (!keysToDelete.isEmpty()) {
+        removeKeys(keysToDelete, false, false);
+      }
+
+      // We moved all the children, now move the top-level dir
+      // Empty directory should have been added as the object summary
+      if (hasMetadataStore()
+          && srcPaths != null
+          && !srcPaths.contains(src)) {
+        LOG.debug("To move the non-empty top-level dir src={} and dst={}",
+            src, dst);
+        S3Guard.addMoveDir(metadataStore, srcPaths, dstMetas, src, dst,
+            username);
+      }
     }
 
+    metadataStore.move(srcPaths, dstMetas);
+
     if (src.getParent() != dst.getParent()) {
       deleteUnnecessaryFakeDirectories(dst.getParent());
       createFakeDirectoryIfNecessary(src.getParent());
@@ -879,6 +961,31 @@ public class S3AFileSystem extends FileSystem {
   }
 
   /**
+   * Does this Filesystem have a metadata store?
+   * @return true iff the FS has been instantiated with a metadata store
+   */
+  public boolean hasMetadataStore() {
+    return !S3Guard.isNullMetadataStore(metadataStore);
+  }
+
+  /**
+   * Get the metadata store.
+   * This will always be non-null, but may be bound to the
+   * {@code NullMetadataStore}.
+   * @return the metadata store of this FS instance
+   */
+  @VisibleForTesting
+  MetadataStore getMetadataStore() {
+    return metadataStore;
+  }
+
+  /** For testing only.  See ITestS3GuardEmptyDirs. */
+  @VisibleForTesting
+  void setMetadataStore(MetadataStore ms) {
+    metadataStore = ms;
+  }
+
+  /**
    * Increment a statistic by 1.
    * @param statistic The operation to increment
    */
@@ -1062,8 +1169,9 @@ public class S3AFileSystem extends FileSystem {
    * @param inputStream source data.
    * @return the request
    */
-  private PutObjectRequest newPutObjectRequest(String key,
-      ObjectMetadata metadata, InputStream inputStream) {
+  PutObjectRequest newPutObjectRequest(String key,
+      ObjectMetadata metadata,
+      InputStream inputStream) {
     Preconditions.checkNotNull(inputStream);
     PutObjectRequest putObjectRequest = new PutObjectRequest(bucket, key,
         inputStream, metadata);
@@ -1114,7 +1222,7 @@ public class S3AFileSystem extends FileSystem {
    * @param putObjectRequest the request
    * @return the upload initiated
    */
-  public Upload putObject(PutObjectRequest putObjectRequest) {
+  public UploadInfo putObject(PutObjectRequest putObjectRequest) {
     long len;
     if (putObjectRequest.getFile() != null) {
       len = putObjectRequest.getFile().length();
@@ -1125,7 +1233,7 @@ public class S3AFileSystem extends FileSystem {
     try {
       Upload upload = transfers.upload(putObjectRequest);
       incrementPutCompletedStatistics(true, len);
-      return upload;
+      return new UploadInfo(upload, len);
     } catch (AmazonClientException e) {
       incrementPutCompletedStatistics(false, len);
       throw e;
@@ -1141,14 +1249,10 @@ public class S3AFileSystem extends FileSystem {
    * @return the upload initiated
    * @throws AmazonClientException on problems
    */
-  public PutObjectResult putObjectDirect(PutObjectRequest putObjectRequest)
+  PutObjectResult putObjectDirect(PutObjectRequest putObjectRequest)
       throws AmazonClientException {
-    long len;
-    if (putObjectRequest.getFile() != null) {
-      len = putObjectRequest.getFile().length();
-    } else {
-      len = putObjectRequest.getMetadata().getContentLength();
-    }
+    long len = getPutRequestLength(putObjectRequest);
+    LOG.debug("PUT {} bytes to {}", len, putObjectRequest.getKey());
     incrementPutStartStatistics(len);
     try {
       PutObjectResult result = s3.putObject(putObjectRequest);
@@ -1161,6 +1265,23 @@ public class S3AFileSystem extends FileSystem {
   }
 
   /**
+   * Get the length of the PUT, verifying that the length is known.
+   * @param putObjectRequest a request bound to a file or a stream.
+   * @return the request length
+   * @throws IllegalArgumentException if the length is negative
+   */
+  private long getPutRequestLength(PutObjectRequest putObjectRequest) {
+    long len;
+    if (putObjectRequest.getFile() != null) {
+      len = putObjectRequest.getFile().length();
+    } else {
+      len = putObjectRequest.getMetadata().getContentLength();
+    }
+    Preconditions.checkState(len >= 0, "Cannot PUT object of unknown length");
+    return len;
+  }
+
+  /**
    * Upload part of a multi-partition file.
    * Increments the write and put counters.
    * <i>Important: this call does not close any input stream in the request.</i>
@@ -1287,7 +1408,7 @@ public class S3AFileSystem extends FileSystem {
    */
   public boolean delete(Path f, boolean recursive) throws IOException {
     try {
-      return innerDelete(getFileStatus(f), recursive);
+      return innerDelete(innerGetFileStatus(f, true), recursive);
     } catch (FileNotFoundException e) {
       LOG.debug("Couldn't delete {} - does not exist", f);
       instrumentation.errorIgnored();
@@ -1317,6 +1438,9 @@ public class S3AFileSystem extends FileSystem {
 
     if (status.isDirectory()) {
       LOG.debug("delete: Path is a directory: {}", f);
+      Preconditions.checkArgument(
+          status.isEmptyDirectory() != Tristate.UNKNOWN,
+          "File status must have directory emptiness computed");
 
       if (!key.endsWith("/")) {
         key = key + "/";
@@ -1326,13 +1450,15 @@ public class S3AFileSystem extends FileSystem {
         return rejectRootDirectoryDelete(status, recursive);
       }
 
-      if (!recursive && !status.isEmptyDirectory()) {
+      if (!recursive && status.isEmptyDirectory() == Tristate.FALSE) {
         throw new PathIsNotEmptyDirectoryException(f.toString());
       }
 
-      if (status.isEmptyDirectory()) {
+      if (status.isEmptyDirectory() == Tristate.TRUE) {
         LOG.debug("Deleting fake empty directory {}", key);
+        // HADOOP-13761 S3Guard: retries here
         deleteObject(key);
+        metadataStore.delete(f);
         instrumentation.directoryDeleted();
       } else {
         LOG.debug("Getting objects for directory prefix {} to delete", key);
@@ -1348,6 +1474,7 @@ public class S3AFileSystem extends FileSystem {
             LOG.debug("Got object to delete {}", summary.getKey());
 
             if (keys.size() == MAX_ENTRIES_TO_DELETE) {
+              // TODO: HADOOP-13761 S3Guard: retries
               removeKeys(keys, true, false);
             }
           }
@@ -1356,16 +1483,19 @@ public class S3AFileSystem extends FileSystem {
             objects = continueListObjects(objects);
           } else {
             if (!keys.isEmpty()) {
+              // TODO: HADOOP-13761 S3Guard: retries
               removeKeys(keys, false, false);
             }
             break;
           }
         }
       }
+      metadataStore.deleteSubtree(f);
     } else {
       LOG.debug("delete: Path is a file");
       instrumentation.fileDeleted(1);
       deleteObject(key);
+      metadataStore.delete(f);
     }
 
     Path parent = f.getParent();
@@ -1389,7 +1519,7 @@ public class S3AFileSystem extends FileSystem {
   private boolean rejectRootDirectoryDelete(S3AFileStatus status,
       boolean recursive) throws IOException {
     LOG.info("s3a delete the {} root directory of {}", bucket, recursive);
-    boolean emptyRoot = status.isEmptyDirectory();
+    boolean emptyRoot = status.isEmptyDirectory() == Tristate.TRUE;
     if (emptyRoot) {
       return true;
     }
@@ -1404,7 +1534,7 @@ public class S3AFileSystem extends FileSystem {
   private void createFakeDirectoryIfNecessary(Path f)
       throws IOException, AmazonClientException {
     String key = pathToKey(f);
-    if (!key.isEmpty() && !exists(f)) {
+    if (!key.isEmpty() && !s3Exists(f)) {
       LOG.debug("Creating new fake directory at {}", f);
       createFakeDirectory(key);
     }
@@ -1453,6 +1583,11 @@ public class S3AFileSystem extends FileSystem {
         key = key + '/';
       }
 
+      DirListingMetadata dirMeta = metadataStore.listChildren(path);
+      if (allowAuthoritative && dirMeta != null && dirMeta.isAuthoritative()) {
+        return S3Guard.dirMetaToStatuses(dirMeta);
+      }
+
       ListObjectsRequest request = createListObjectsRequest(key, "/");
       LOG.debug("listStatus: doing listObjects for directory {}", key);
 
@@ -1465,7 +1600,8 @@ public class S3AFileSystem extends FileSystem {
       while (files.hasNext()) {
         result.add(files.next());
       }
-      return result.toArray(new FileStatus[result.size()]);
+      return S3Guard.dirListingUnion(metadataStore, path, result, dirMeta,
+          allowAuthoritative);
     } else {
       LOG.debug("Adding: rd (not a dir): {}", path);
       FileStatus[] stats = new FileStatus[1];
@@ -1481,7 +1617,8 @@ public class S3AFileSystem extends FileSystem {
    * @param delimiter any delimiter
    * @return the request
    */
-  private ListObjectsRequest createListObjectsRequest(String key,
+  @VisibleForTesting
+  ListObjectsRequest createListObjectsRequest(String key,
       String delimiter) {
     ListObjectsRequest request = new ListObjectsRequest();
     request.setBucketName(bucket);
@@ -1540,23 +1677,30 @@ public class S3AFileSystem extends FileSystem {
       throw translateException("innerMkdirs", path, e);
     }
   }
+
   /**
    *
    * Make the given path and all non-existent parents into
    * directories.
    * See {@link #mkdirs(Path, FsPermission)}
-   * @param f path to create
+   * @param p path to create
    * @param permission to apply to f
-   * @return true if a directory was created
+   * @return true if a directory was created or already existed
    * @throws FileAlreadyExistsException there is a file at the path specified
    * @throws IOException other IO problems
    * @throws AmazonClientException on failures inside the AWS SDK
    */
-  private boolean innerMkdirs(Path f, FsPermission permission)
+  private boolean innerMkdirs(Path p, FsPermission permission)
       throws IOException, FileAlreadyExistsException, AmazonClientException {
+    Path f = qualify(p);
     LOG.debug("Making directory: {}", f);
     incrementStatistic(INVOCATION_MKDIRS);
     FileStatus fileStatus;
+    List<Path> metadataStoreDirs = null;
+    if (hasMetadataStore()) {
+      metadataStoreDirs = new ArrayList<>();
+    }
+
     try {
       fileStatus = getFileStatus(f);
 
@@ -1566,8 +1710,12 @@ public class S3AFileSystem extends FileSystem {
         throw new FileAlreadyExistsException("Path is a file: " + f);
       }
     } catch (FileNotFoundException e) {
+      // Walk path to root, ensuring closest ancestor is a directory, not file
       Path fPart = f.getParent();
-      do {
+      if (metadataStoreDirs != null) {
+        metadataStoreDirs.add(f);
+      }
+      while (fPart != null) {
         try {
           fileStatus = getFileStatus(fPart);
           if (fileStatus.isDirectory()) {
@@ -1580,12 +1728,17 @@ public class S3AFileSystem extends FileSystem {
           }
         } catch (FileNotFoundException fnfe) {
           instrumentation.errorIgnored();
+          // We create all missing directories in MetadataStore; it does not
+          // infer directories exist by prefix like S3.
+          if (metadataStoreDirs != null) {
+            metadataStoreDirs.add(fPart);
+          }
         }
         fPart = fPart.getParent();
-      } while (fPart != null);
-
+      }
       String key = pathToKey(f);
       createFakeDirectory(key);
+      S3Guard.makeDirsOrdered(metadataStore, metadataStoreDirs, username, true);
       // this is complicated because getParent(a/b/c/) returns a/b/c, but
       // we want a/b. See HADOOP-14428 for more details.
       deleteUnnecessaryFakeDirectories(new Path(f.toString()).getParent());
@@ -1597,21 +1750,93 @@ public class S3AFileSystem extends FileSystem {
    * Return a file status object that represents the path.
    * @param f The path we want information from
    * @return a FileStatus object
-   * @throws java.io.FileNotFoundException when the path does not exist;
+   * @throws FileNotFoundException when the path does not exist
    * @throws IOException on other problems.
    */
-  public S3AFileStatus getFileStatus(final Path f) throws IOException {
+  public FileStatus getFileStatus(final Path f) throws IOException {
+    return innerGetFileStatus(f, false);
+  }
+
+  /**
+   * Internal version of {@link #getFileStatus(Path)}.
+   * @param f The path we want information from
+   * @param needEmptyDirectoryFlag if true, implementation will calculate
+   *        a TRUE or FALSE value for {@link S3AFileStatus#isEmptyDirectory()}
+   * @return a S3AFileStatus object
+   * @throws FileNotFoundException when the path does not exist
+   * @throws IOException on other problems.
+   */
+  @VisibleForTesting
+  S3AFileStatus innerGetFileStatus(final Path f,
+      boolean needEmptyDirectoryFlag) throws IOException {
     incrementStatistic(INVOCATION_GET_FILE_STATUS);
     final Path path = qualify(f);
     String key = pathToKey(path);
-    LOG.debug("Getting path status for {}  ({})", path , key);
+    LOG.debug("Getting path status for {}  ({})", path, key);
+
+    // Check MetadataStore, if any.
+    PathMetadata pm = metadataStore.get(path, needEmptyDirectoryFlag);
+    Set<Path> tombstones = Collections.EMPTY_SET;
+    if (pm != null) {
+      if (pm.isDeleted()) {
+        throw new FileNotFoundException("Path " + f + " is recorded as " +
+            "deleted by S3Guard");
+      }
+
+      FileStatus msStatus = pm.getFileStatus();
+      if (needEmptyDirectoryFlag && msStatus.isDirectory()) {
+        if (pm.isEmptyDirectory() != Tristate.UNKNOWN) {
+          // We have a definitive true / false from MetadataStore, we are done.
+          return S3AFileStatus.fromFileStatus(msStatus, pm.isEmptyDirectory());
+        } else {
+          DirListingMetadata children = metadataStore.listChildren(path);
+          if (children != null) {
+            tombstones = children.listTombstones();
+          }
+          LOG.debug("MetadataStore doesn't know if dir is empty, using S3.");
+        }
+      } else {
+        // Either this is not a directory, or we don't care if it is empty
+        return S3AFileStatus.fromFileStatus(msStatus, pm.isEmptyDirectory());
+      }
+
+      // If the metadata store has no children for it and it's not listed in
+      // S3 yet, we'll assume the empty directory is true;
+      S3AFileStatus s3FileStatus;
+      try {
+        s3FileStatus = s3GetFileStatus(path, key, tombstones);
+      } catch (FileNotFoundException e) {
+        return S3AFileStatus.fromFileStatus(msStatus, Tristate.TRUE);
+      }
+      // entry was found, save in S3Guard
+      return S3Guard.putAndReturn(metadataStore, s3FileStatus, instrumentation);
+    } else {
+      // there was no entry in S3Guard
+      // retrieve the data and update the metadata store in the process.
+      return S3Guard.putAndReturn(metadataStore,
+          s3GetFileStatus(path, key, tombstones), instrumentation);
+    }
+  }
+
+  /**
+   * Raw {@code getFileStatus} that talks direct to S3.
+   * Used to implement {@link #innerGetFileStatus(Path, boolean)},
+   * and for direct management of empty directory blobs.
+   * @param path Qualified path
+   * @param key  Key string for the path
+   * @return Status
+   * @throws FileNotFoundException when the path does not exist
+   * @throws IOException on other problems.
+   */
+  private S3AFileStatus s3GetFileStatus(final Path path, String key,
+      Set<Path> tombstones) throws IOException {
     if (!key.isEmpty()) {
       try {
         ObjectMetadata meta = getObjectMetadata(key);
 
         if (objectRepresentsDirectory(key, meta.getContentLength())) {
           LOG.debug("Found exact file: fake directory");
-          return new S3AFileStatus(true, path, username);
+          return new S3AFileStatus(Tristate.TRUE, path, username);
         } else {
           LOG.debug("Found exact file: normal file");
           return new S3AFileStatus(meta.getContentLength(),
@@ -1636,16 +1861,16 @@ public class S3AFileSystem extends FileSystem {
 
           if (objectRepresentsDirectory(newKey, meta.getContentLength())) {
             LOG.debug("Found file (with /): fake directory");
-            return new S3AFileStatus(true, path, username);
+            return new S3AFileStatus(Tristate.TRUE, path, username);
           } else {
             LOG.warn("Found file (with /): real file? should not happen: {}",
                 key);
 
             return new S3AFileStatus(meta.getContentLength(),
-                dateToLong(meta.getLastModified()),
-                path,
-                getDefaultBlockSize(path),
-                username);
+                    dateToLong(meta.getLastModified()),
+                    path,
+                    getDefaultBlockSize(path),
+                    username);
           }
         } catch (AmazonServiceException e) {
           if (e.getStatusCode() != 404) {
@@ -1667,25 +1892,26 @@ public class S3AFileSystem extends FileSystem {
 
       ObjectListing objects = listObjects(request);
 
-      if (!objects.getCommonPrefixes().isEmpty()
-          || !objects.getObjectSummaries().isEmpty()) {
+      Collection<String> prefixes = objects.getCommonPrefixes();
+      Collection<S3ObjectSummary> summaries = objects.getObjectSummaries();
+      if (!isEmptyOfKeys(prefixes, tombstones) ||
+          !isEmptyOfObjects(summaries, tombstones)) {
         if (LOG.isDebugEnabled()) {
           LOG.debug("Found path as directory (with /): {}/{}",
-              objects.getCommonPrefixes().size() ,
-              objects.getObjectSummaries().size());
+              prefixes.size(), summaries.size());
 
-          for (S3ObjectSummary summary : objects.getObjectSummaries()) {
+          for (S3ObjectSummary summary : summaries) {
             LOG.debug("Summary: {} {}", summary.getKey(), summary.getSize());
           }
-          for (String prefix : objects.getCommonPrefixes()) {
+          for (String prefix : prefixes) {
             LOG.debug("Prefix: {}", prefix);
           }
         }
 
-        return new S3AFileStatus(false, path, username);
+        return new S3AFileStatus(Tristate.FALSE, path, username);
       } else if (key.isEmpty()) {
         LOG.debug("Found root directory");
-        return new S3AFileStatus(true, path, username);
+        return new S3AFileStatus(Tristate.TRUE, path, username);
       }
     } catch (AmazonServiceException e) {
       if (e.getStatusCode() != 404) {
@@ -1700,6 +1926,64 @@ public class S3AFileSystem extends FileSystem {
   }
 
   /**
+   * Helper function to determine if a collection of paths is empty
+   * after accounting for tombstone markers (if provided).
+   * @param keys Collection of path (prefixes / directories or keys).
+   * @param tombstones Set of tombstone markers, or null if not applicable.
+   * @return false if summaries contains objects not accounted for by
+   * tombstones.
+   */
+  private boolean isEmptyOfKeys(Collection<String> keys, Set<Path>
+      tombstones) {
+    if (tombstones == null) {
+      return keys.isEmpty();
+    }
+    for (String key : keys) {
+      Path qualified = keyToQualifiedPath(key);
+      if (!tombstones.contains(qualified)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /**
+   * Helper function to determine if a collection of object summaries is empty
+   * after accounting for tombstone markers (if provided).
+   * @param summaries Collection of objects as returned by listObjects.
+   * @param tombstones Set of tombstone markers, or null if not applicable.
+   * @return false if summaries contains objects not accounted for by
+   * tombstones.
+   */
+  private boolean isEmptyOfObjects(Collection<S3ObjectSummary> summaries,
+      Set<Path> tombstones) {
+    if (tombstones == null) {
+      return summaries.isEmpty();
+    }
+    Collection<String> stringCollection = new ArrayList<>(summaries.size());
+    for (S3ObjectSummary summary : summaries) {
+      stringCollection.add(summary.getKey());
+    }
+    return isEmptyOfKeys(stringCollection, tombstones);
+  }
+
+  /**
+   * Raw version of {@link FileSystem#exists(Path)} which uses S3 only:
+   * S3Guard MetadataStore, if any, will be skipped.
+   * @return true if path exists in S3
+   */
+  private boolean s3Exists(final Path f) throws IOException {
+    Path path = qualify(f);
+    String key = pathToKey(path);
+    try {
+      s3GetFileStatus(path, key, null);
+      return true;
+    } catch (FileNotFoundException e) {
+      return false;
+    }
+  }
+
+  /**
    * The src file is on the local disk.  Add it to FS at
    * the given dst name.
    *
@@ -1761,12 +2045,13 @@ public class S3AFileSystem extends FileSystem {
 
     final ObjectMetadata om = newObjectMetadata(srcfile.length());
     PutObjectRequest putObjectRequest = newPutObjectRequest(key, om, srcfile);
-    Upload up = putObject(putObjectRequest);
+    UploadInfo info = putObject(putObjectRequest);
+    Upload upload = info.getUpload();
     ProgressableProgressListener listener = new ProgressableProgressListener(
-        this, key, up, null);
-    up.addProgressListener(listener);
+        this, key, upload, null);
+    upload.addProgressListener(listener);
     try {
-      up.waitForUploadResult();
+      upload.waitForUploadResult();
     } catch (InterruptedException e) {
       throw new InterruptedIOException("Interrupted copying " + src
           + " to "  + dst + ", cancelling");
@@ -1774,7 +2059,7 @@ public class S3AFileSystem extends FileSystem {
     listener.uploadCompleted();
 
     // This will delete unnecessary fake parent directories
-    finishedWrite(key);
+    finishedWrite(key, info.getLength());
 
     if (delSrc) {
       local.delete(src, false);
@@ -1798,6 +2083,10 @@ public class S3AFileSystem extends FileSystem {
         transfers.shutdownNow(true);
         transfers = null;
       }
+      if (metadataStore != null) {
+        metadataStore.close();
+        metadataStore = null;
+      }
     }
   }
 
@@ -1940,11 +2229,38 @@ public class S3AFileSystem extends FileSystem {
 
   /**
    * Perform post-write actions.
+   * This operation MUST be called after any PUT/multipart PUT completes
+   * successfully.
+   * This includes
+   * <ol>
+   *   <li>Calling {@link #deleteUnnecessaryFakeDirectories(Path)}</li>
+   *   <li>Updating any metadata store with details on the newly created
+   *   object.</li>
+   * </ol>
    * @param key key written to
+   * @param length  total length of file written
    */
-  public void finishedWrite(String key) {
-    LOG.debug("Finished write to {}", key);
-    deleteUnnecessaryFakeDirectories(keyToPath(key).getParent());
+  @InterfaceAudience.Private
+  void finishedWrite(String key, long length) {
+    LOG.debug("Finished write to {}, len {}", key, length);
+    Path p = keyToQualifiedPath(key);
+    deleteUnnecessaryFakeDirectories(p.getParent());
+    Preconditions.checkArgument(length >= 0, "content length is negative");
+
+    // See note about failure semantics in S3Guard documentation
+    try {
+      if (hasMetadataStore()) {
+        S3Guard.addAncestors(metadataStore, p, username);
+        S3AFileStatus status = createUploadFileStatus(p,
+            S3AUtils.objectRepresentsDirectory(key, length), length,
+            getDefaultBlockSize(p), username);
+        S3Guard.putAndReturn(metadataStore, status, instrumentation);
+      }
+    } catch (IOException e) {
+      LOG.error("S3Guard: Error updating MetadataStore for write to {}:",
+          key, e);
+      instrumentation.errorIgnored();
+    }
   }
 
   /**
@@ -1999,9 +2315,9 @@ public class S3AFileSystem extends FileSystem {
     PutObjectRequest putObjectRequest = newPutObjectRequest(objectName,
         newObjectMetadata(0L),
         im);
-    Upload upload = putObject(putObjectRequest);
+    UploadInfo info = putObject(putObjectRequest);
     try {
-      upload.waitForUploadResult();
+      info.getUpload().waitForUploadResult();
     } catch (InterruptedException e) {
       throw new InterruptedIOException("Interrupted creating " + objectName);
     }
@@ -2107,6 +2423,8 @@ public class S3AFileSystem extends FileSystem {
     if (blockFactory != null) {
       sb.append(", blockFactory=").append(blockFactory);
     }
+    sb.append(", metastore=").append(metadataStore);
+    sb.append(", authoritative=").append(allowAuthoritative);
     sb.append(", boundedExecutor=").append(boundedThreadPool);
     sb.append(", unboundedExecutor=").append(unboundedThreadPool);
     sb.append(", statistics {")
@@ -2223,6 +2541,18 @@ public class S3AFileSystem extends FileSystem {
   @Override
   public RemoteIterator<LocatedFileStatus> listFiles(Path f,
       boolean recursive) throws FileNotFoundException, IOException {
+    return innerListFiles(f, recursive,
+        new Listing.AcceptFilesOnly(qualify(f)));
+  }
+
+  public RemoteIterator<LocatedFileStatus> listFilesAndEmptyDirectories(Path f,
+      boolean recursive) throws IOException {
+    return innerListFiles(f, recursive,
+        new Listing.AcceptAllButS3nDirs());
+  }
+
+  private RemoteIterator<LocatedFileStatus> innerListFiles(Path f, boolean
+      recursive, Listing.FileStatusAcceptor acceptor) throws IOException {
     incrementStatistic(INVOCATION_LIST_FILES);
     Path path = qualify(f);
     LOG.debug("listFiles({}, {})", path, recursive);
@@ -2240,13 +2570,42 @@ public class S3AFileSystem extends FileSystem {
         String delimiter = recursive ? null : "/";
         LOG.debug("Requesting all entries under {} with delimiter '{}'",
             key, delimiter);
-        return listing.createLocatedFileStatusIterator(
-            listing.createFileStatusListingIterator(path,
-                createListObjectsRequest(key, delimiter),
-                ACCEPT_ALL,
-                new Listing.AcceptFilesOnly(path)));
+        final RemoteIterator<FileStatus> cachedFilesIterator;
+        final Set<Path> tombstones;
+        if (recursive) {
+          final PathMetadata pm = metadataStore.get(path, true);
+          // shouldn't need to check pm.isDeleted() because that will have
+          // been caught by getFileStatus above.
+          MetadataStoreListFilesIterator metadataStoreListFilesIterator =
+              new MetadataStoreListFilesIterator(metadataStore, pm,
+                  allowAuthoritative);
+          tombstones = metadataStoreListFilesIterator.listTombstones();
+          cachedFilesIterator = metadataStoreListFilesIterator;
+        } else {
+          DirListingMetadata meta = metadataStore.listChildren(path);
+          if (meta != null) {
+            tombstones = meta.listTombstones();
+          } else {
+            tombstones = null;
+          }
+          cachedFilesIterator = listing.createProvidedFileStatusIterator(
+              S3Guard.dirMetaToStatuses(meta), ACCEPT_ALL, acceptor);
+          if (allowAuthoritative && meta != null && meta.isAuthoritative()) {
+            // metadata listing is authoritative, so return it directly
+            return listing.createLocatedFileStatusIterator(cachedFilesIterator);
+          }
+        }
+        return listing.createTombstoneReconcilingIterator(
+            listing.createLocatedFileStatusIterator(
+                listing.createFileStatusListingIterator(path,
+                    createListObjectsRequest(key, delimiter),
+                    ACCEPT_ALL,
+                    acceptor,
+                    cachedFilesIterator)),
+            tombstones);
       }
     } catch (AmazonClientException e) {
+      // TODO S3Guard: retry on file not found exception
       throw translateException("listFiles", path, e);
     }
   }
@@ -2291,12 +2650,21 @@ public class S3AFileSystem extends FileSystem {
             filter.accept(path) ? toLocatedFileStatus(fileStatus) : null);
       } else {
         // directory: trigger a lookup
-        String key = maybeAddTrailingSlash(pathToKey(path));
-        return listing.createLocatedFileStatusIterator(
-            listing.createFileStatusListingIterator(path,
-                createListObjectsRequest(key, "/"),
-                filter,
-                new Listing.AcceptAllButSelfAndS3nDirs(path)));
+        final String key = maybeAddTrailingSlash(pathToKey(path));
+        final Listing.FileStatusAcceptor acceptor =
+            new Listing.AcceptAllButSelfAndS3nDirs(path);
+        DirListingMetadata meta = metadataStore.listChildren(path);
+        final RemoteIterator<FileStatus> cachedFileStatusIterator =
+            listing.createProvidedFileStatusIterator(
+                S3Guard.dirMetaToStatuses(meta), filter, acceptor);
+        return (allowAuthoritative && meta != null && meta.isAuthoritative())
+            ? listing.createLocatedFileStatusIterator(cachedFileStatusIterator)
+            : listing.createLocatedFileStatusIterator(
+                listing.createFileStatusListingIterator(path,
+                    createListObjectsRequest(key, "/"),
+                    filter,
+                    acceptor,
+                    cachedFileStatusIterator));
       }
     } catch (AmazonClientException e) {
       throw translateException("listLocatedStatus", path, e);
@@ -2371,8 +2739,8 @@ public class S3AFileSystem extends FileSystem {
     /**
      * Callback on a successful write.
      */
-    void writeSuccessful() {
-      finishedWrite(key);
+    void writeSuccessful(long length) {
+      finishedWrite(key, length);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
----------------------------------------------------------------------
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 d2e7a88..da1fc5a 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
@@ -23,6 +23,7 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileSystem.Statistics;
 import org.apache.hadoop.metrics2.MetricStringBuilder;
 import org.apache.hadoop.metrics2.annotation.Metrics;
 import org.apache.hadoop.metrics2.lib.Interns;
@@ -30,6 +31,7 @@ import org.apache.hadoop.metrics2.lib.MetricsRegistry;
 import org.apache.hadoop.metrics2.lib.MutableCounterLong;
 import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
 import org.apache.hadoop.metrics2.lib.MutableMetric;
+import org.apache.hadoop.metrics2.lib.MutableQuantiles;
 
 import java.io.Closeable;
 import java.net.URI;
@@ -38,7 +40,6 @@ import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
-import org.apache.hadoop.fs.FileSystem.Statistics;
 
 import static org.apache.hadoop.fs.s3a.Statistic.*;
 
@@ -90,6 +91,10 @@ public class S3AInstrumentation {
   private final Map<String, MutableCounterLong> streamMetrics =
       new HashMap<>(30);
 
+  /** Instantiate this without caring whether or not S3Guard is enabled. */
+  private final S3GuardInstrumentation s3GuardInstrumentation
+      = new S3GuardInstrumentation();
+
   private static final Statistic[] COUNTERS_TO_CREATE = {
       INVOCATION_COPY_FROM_LOCAL_FILE,
       INVOCATION_EXISTS,
@@ -117,6 +122,8 @@ public class S3AInstrumentation {
       STREAM_WRITE_BLOCK_UPLOADS_ABORTED,
       STREAM_WRITE_TOTAL_TIME,
       STREAM_WRITE_TOTAL_DATA,
+      S3GUARD_METADATASTORE_PUT_PATH_REQUEST,
+      S3GUARD_METADATASTORE_INITIALIZATION
   };
 
 
@@ -171,6 +178,9 @@ public class S3AInstrumentation {
     for (Statistic statistic : GAUGES_TO_CREATE) {
       gauge(statistic.getSymbol(), statistic.getDescription());
     }
+    //todo need a config for the quantiles interval?
+    quantiles(S3GUARD_METADATASTORE_PUT_PATH_LATENCY,
+        "ops", "latency", 1);
   }
 
   /**
@@ -227,6 +237,22 @@ public class S3AInstrumentation {
   }
 
   /**
+   * Create a quantiles in the registry.
+   * @param op  statistic to collect
+   * @param sampleName sample name of the quantiles
+   * @param valueName value name of the quantiles
+   * @param interval interval of the quantiles in seconds
+   * @return the created quantiles metric
+   */
+  protected final MutableQuantiles quantiles(Statistic op,
+      String sampleName,
+      String valueName,
+      int interval) {
+    return registry.newQuantiles(op.getSymbol(), op.getDescription(),
+        sampleName, valueName, interval);
+  }
+
+  /**
    * Get the metrics registry.
    * @return the registry
    */
@@ -311,6 +337,20 @@ public class S3AInstrumentation {
   }
 
   /**
+   * Look up a quantiles.
+   * @param name quantiles name
+   * @return the quantiles or null
+   * @throws ClassCastException if the metric is not a Quantiles.
+   */
+  public MutableQuantiles lookupQuantiles(String name) {
+    MutableMetric metric = lookupMetric(name);
+    if (metric == null) {
+      LOG.debug("No quantiles {}", name);
+    }
+    return (MutableQuantiles) metric;
+  }
+
+  /**
    * Look up a metric from both the registered set and the lighter weight
    * stream entries.
    * @param name metric name
@@ -391,6 +431,21 @@ public class S3AInstrumentation {
       counter.incr(count);
     }
   }
+
+  /**
+   * Add a value to a quantiles statistic. No-op if the quantile
+   * isn't found.
+   * @param op operation to look up.
+   * @param value value to add.
+   * @throws ClassCastException if the metric is not a Quantiles.
+   */
+  public void addValueToQuantiles(Statistic op, long value) {
+    MutableQuantiles quantiles = lookupQuantiles(op.getSymbol());
+    if (quantiles != null) {
+      quantiles.add(value);
+    }
+  }
+
   /**
    * Increment a specific counter.
    * No-op if not defined.
@@ -442,6 +497,15 @@ public class S3AInstrumentation {
   }
 
   /**
+   * Create a S3Guard instrumentation instance.
+   * There's likely to be at most one instance of this per FS instance.
+   * @return the S3Guard instrumentation point.
+   */
+  public S3GuardInstrumentation getS3GuardInstrumentation() {
+    return s3GuardInstrumentation;
+  }
+
+  /**
    * Merge in the statistics of a single input stream into
    * the filesystem-wide statistics.
    * @param statistics stream statistics
@@ -840,4 +904,19 @@ public class S3AInstrumentation {
       return sb.toString();
     }
   }
+
+  /**
+   * Instrumentation exported to S3Guard.
+   */
+  public final class S3GuardInstrumentation {
+
+    /** Initialized event. */
+    public void initialized() {
+      incrementCounter(S3GUARD_METADATASTORE_INITIALIZATION, 1);
+    }
+
+    public void storeClosed() {
+
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java
index 6ebc9e4..e723b75 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.fs.s3a;
 
 import com.amazonaws.AmazonClientException;
 import com.amazonaws.services.s3.model.ObjectMetadata;
-import com.amazonaws.services.s3.transfer.Upload;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
@@ -101,19 +100,20 @@ public class S3AOutputStream extends OutputStream {
 
     try {
       final ObjectMetadata om = fs.newObjectMetadata(backupFile.length());
-      Upload upload = fs.putObject(
+      UploadInfo info = fs.putObject(
           fs.newPutObjectRequest(
               key,
               om,
               backupFile));
       ProgressableProgressListener listener =
-          new ProgressableProgressListener(fs, key, upload, progress);
-      upload.addProgressListener(listener);
+          new ProgressableProgressListener(fs, key, info.getUpload(), progress);
+      info.getUpload().addProgressListener(listener);
 
-      upload.waitForUploadResult();
+      info.getUpload().waitForUploadResult();
       listener.uploadCompleted();
-      // This will delete unnecessary fake parent directories
-      fs.finishedWrite(key);
+      // This will delete unnecessary fake parent directories, update any
+      // MetadataStore
+      fs.finishedWrite(key, info.getLength());
     } catch (InterruptedException e) {
       throw (InterruptedIOException) new InterruptedIOException(e.toString())
           .initCause(e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
----------------------------------------------------------------------
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 d57a0c6..ae9dd79 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
@@ -294,12 +294,38 @@ public final class S3AUtils {
       S3ObjectSummary summary,
       long blockSize,
       String owner) {
-    if (objectRepresentsDirectory(summary.getKey(), summary.getSize())) {
-      return new S3AFileStatus(true, keyPath, owner);
+    long size = summary.getSize();
+    return createFileStatus(keyPath,
+        objectRepresentsDirectory(summary.getKey(), size),
+        size, summary.getLastModified(), blockSize, owner);
+  }
+
+  /**
+   * Create a file status for object we just uploaded.  For files, we use
+   * current time as modification time, since s3a uses S3's service-based
+   * modification time, which will not be available until we do a
+   * getFileStatus() later on.
+   * @param keyPath path for created object
+   * @param isDir true iff directory
+   * @param size file length
+   * @param blockSize block size for file status
+   * @param owner Hadoop username
+   * @return a status entry
+   */
+  public static S3AFileStatus createUploadFileStatus(Path keyPath,
+      boolean isDir, long size, long blockSize, String owner) {
+    Date date = isDir ? null : new Date();
+    return createFileStatus(keyPath, isDir, size, date, blockSize, owner);
+  }
+
+  /* Date 'modified' is ignored when isDir is true. */
+  private static S3AFileStatus createFileStatus(Path keyPath, boolean isDir,
+      long size, Date modified, long blockSize, String owner) {
+    if (isDir) {
+      return new S3AFileStatus(Tristate.UNKNOWN, keyPath, owner);
     } else {
-      return new S3AFileStatus(summary.getSize(),
-          dateToLong(summary.getLastModified()), keyPath,
-          blockSize, owner);
+      return new S3AFileStatus(size, dateToLong(modified), keyPath, blockSize,
+          owner);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java
index 7ccdc06..e7603d9 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java
@@ -18,33 +18,20 @@
 
 package org.apache.hadoop.fs.s3a;
 
-import static org.apache.hadoop.fs.s3a.Constants.*;
-import static org.apache.hadoop.fs.s3a.S3AUtils.*;
-
 import java.io.IOException;
 import java.net.URI;
 
-import com.amazonaws.ClientConfiguration;
-import com.amazonaws.Protocol;
-import com.amazonaws.auth.AWSCredentialsProvider;
 import com.amazonaws.services.s3.AmazonS3;
-import com.amazonaws.services.s3.AmazonS3Client;
-import com.amazonaws.services.s3.S3ClientOptions;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.util.VersionInfo;
-
-import org.slf4j.Logger;
 
 /**
- * Factory for creation of S3 client instances to be used by {@link S3Store}.
+ * Factory for creation of {@link AmazonS3} client instances.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
-interface S3ClientFactory {
+public interface S3ClientFactory {
 
   /**
    * Creates a new {@link AmazonS3} client.  This method accepts the S3A file
@@ -57,182 +44,4 @@ interface S3ClientFactory {
    */
   AmazonS3 createS3Client(URI name) throws IOException;
 
-  /**
-   * The default factory implementation, which calls the AWS SDK to configure
-   * and create an {@link AmazonS3Client} that communicates with the S3 service.
-   */
-  static class DefaultS3ClientFactory extends Configured
-      implements S3ClientFactory {
-
-    private static final Logger LOG = S3AFileSystem.LOG;
-
-    @Override
-    public AmazonS3 createS3Client(URI name) throws IOException {
-      Configuration conf = getConf();
-      AWSCredentialsProvider credentials =
-          createAWSCredentialProviderSet(name, conf);
-      ClientConfiguration awsConf = new ClientConfiguration();
-      initConnectionSettings(conf, awsConf);
-      initProxySupport(conf, awsConf);
-      initUserAgent(conf, awsConf);
-      return createAmazonS3Client(conf, credentials, awsConf);
-    }
-
-    /**
-     * Initializes all AWS SDK settings related to connection management.
-     *
-     * @param conf Hadoop configuration
-     * @param awsConf AWS SDK configuration
-     */
-    private static void initConnectionSettings(Configuration conf,
-        ClientConfiguration awsConf) {
-      awsConf.setMaxConnections(intOption(conf, MAXIMUM_CONNECTIONS,
-          DEFAULT_MAXIMUM_CONNECTIONS, 1));
-      boolean secureConnections = conf.getBoolean(SECURE_CONNECTIONS,
-          DEFAULT_SECURE_CONNECTIONS);
-      awsConf.setProtocol(secureConnections ?  Protocol.HTTPS : Protocol.HTTP);
-      awsConf.setMaxErrorRetry(intOption(conf, MAX_ERROR_RETRIES,
-          DEFAULT_MAX_ERROR_RETRIES, 0));
-      awsConf.setConnectionTimeout(intOption(conf, ESTABLISH_TIMEOUT,
-          DEFAULT_ESTABLISH_TIMEOUT, 0));
-      awsConf.setSocketTimeout(intOption(conf, SOCKET_TIMEOUT,
-          DEFAULT_SOCKET_TIMEOUT, 0));
-      int sockSendBuffer = intOption(conf, SOCKET_SEND_BUFFER,
-          DEFAULT_SOCKET_SEND_BUFFER, 2048);
-      int sockRecvBuffer = intOption(conf, SOCKET_RECV_BUFFER,
-          DEFAULT_SOCKET_RECV_BUFFER, 2048);
-      awsConf.setSocketBufferSizeHints(sockSendBuffer, sockRecvBuffer);
-      String signerOverride = conf.getTrimmed(SIGNING_ALGORITHM, "");
-      if (!signerOverride.isEmpty()) {
-        LOG.debug("Signer override = {}", signerOverride);
-        awsConf.setSignerOverride(signerOverride);
-      }
-    }
-
-    /**
-     * Initializes AWS SDK proxy support if configured.
-     *
-     * @param conf Hadoop configuration
-     * @param awsConf AWS SDK configuration
-     * @throws IllegalArgumentException if misconfigured
-     */
-    private static void initProxySupport(Configuration conf,
-        ClientConfiguration awsConf)
-            throws IllegalArgumentException, IOException {
-      String proxyHost = conf.getTrimmed(PROXY_HOST, "");
-      int proxyPort = conf.getInt(PROXY_PORT, -1);
-      if (!proxyHost.isEmpty()) {
-        awsConf.setProxyHost(proxyHost);
-        if (proxyPort >= 0) {
-          awsConf.setProxyPort(proxyPort);
-        } else {
-          if (conf.getBoolean(SECURE_CONNECTIONS, DEFAULT_SECURE_CONNECTIONS)) {
-            LOG.warn("Proxy host set without port. Using HTTPS default 443");
-            awsConf.setProxyPort(443);
-          } else {
-            LOG.warn("Proxy host set without port. Using HTTP default 80");
-            awsConf.setProxyPort(80);
-          }
-        }
-        String proxyUsername = conf.getTrimmed(PROXY_USERNAME);
-        String proxyPassword = null;
-        char[] proxyPass = conf.getPassword(PROXY_PASSWORD);
-        if (proxyPass != null) {
-          proxyPassword = new String(proxyPass).trim();
-        }
-        if ((proxyUsername == null) != (proxyPassword == null)) {
-          String msg = "Proxy error: " + PROXY_USERNAME + " or " +
-              PROXY_PASSWORD + " set without the other.";
-          LOG.error(msg);
-          throw new IllegalArgumentException(msg);
-        }
-        awsConf.setProxyUsername(proxyUsername);
-        awsConf.setProxyPassword(proxyPassword);
-        awsConf.setProxyDomain(conf.getTrimmed(PROXY_DOMAIN));
-        awsConf.setProxyWorkstation(conf.getTrimmed(PROXY_WORKSTATION));
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Using proxy server {}:{} as user {} on " +
-                "domain {} as workstation {}", awsConf.getProxyHost(),
-              awsConf.getProxyPort(),
-              String.valueOf(awsConf.getProxyUsername()),
-              awsConf.getProxyDomain(),
-              awsConf.getProxyWorkstation());
-        }
-      } else if (proxyPort >= 0) {
-        String msg =
-            "Proxy error: " + PROXY_PORT + " set without " + PROXY_HOST;
-        LOG.error(msg);
-        throw new IllegalArgumentException(msg);
-      }
-    }
-
-    /**
-     * Initializes the User-Agent header to send in HTTP requests to the S3
-     * back-end.  We always include the Hadoop version number.  The user also
-     * may set an optional custom prefix to put in front of the Hadoop version
-     * number.  The AWS SDK interally appends its own information, which seems
-     * to include the AWS SDK version, OS and JVM version.
-     *
-     * @param conf Hadoop configuration
-     * @param awsConf AWS SDK configuration
-     */
-    private static void initUserAgent(Configuration conf,
-        ClientConfiguration awsConf) {
-      String userAgent = "Hadoop " + VersionInfo.getVersion();
-      String userAgentPrefix = conf.getTrimmed(USER_AGENT_PREFIX, "");
-      if (!userAgentPrefix.isEmpty()) {
-        userAgent = userAgentPrefix + ", " + userAgent;
-      }
-      LOG.debug("Using User-Agent: {}", userAgent);
-      awsConf.setUserAgentPrefix(userAgent);
-    }
-
-    /**
-     * Creates an {@link AmazonS3Client} from the established configuration.
-     *
-     * @param conf Hadoop configuration
-     * @param credentials AWS credentials
-     * @param awsConf AWS SDK configuration
-     * @return S3 client
-     * @throws IllegalArgumentException if misconfigured
-     */
-    private static AmazonS3 createAmazonS3Client(Configuration conf,
-        AWSCredentialsProvider credentials, ClientConfiguration awsConf)
-        throws IllegalArgumentException {
-      AmazonS3 s3 = new AmazonS3Client(credentials, awsConf);
-      String endPoint = conf.getTrimmed(ENDPOINT, "");
-      if (!endPoint.isEmpty()) {
-        try {
-          s3.setEndpoint(endPoint);
-        } catch (IllegalArgumentException e) {
-          String msg = "Incorrect endpoint: "  + e.getMessage();
-          LOG.error(msg);
-          throw new IllegalArgumentException(msg, e);
-        }
-      }
-      enablePathStyleAccessIfRequired(s3, conf);
-      return s3;
-    }
-
-    /**
-     * Enables path-style access to S3 buckets if configured.  By default, the
-     * behavior is to use virtual hosted-style access with URIs of the form
-     * http://bucketname.s3.amazonaws.com.  Enabling path-style access and a
-     * region-specific endpoint switches the behavior to use URIs of the form
-     * http://s3-eu-west-1.amazonaws.com/bucketname.
-     *
-     * @param s3 S3 client
-     * @param conf Hadoop configuration
-     */
-    private static void enablePathStyleAccessIfRequired(AmazonS3 s3,
-        Configuration conf) {
-      final boolean pathStyleAccess = conf.getBoolean(PATH_STYLE_ACCESS, false);
-      if (pathStyleAccess) {
-        LOG.debug("Enabling path style access!");
-        s3.setS3ClientOptions(S3ClientOptions.builder()
-            .setPathStyleAccess(true)
-            .build());
-      }
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
----------------------------------------------------------------------
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 789c6d7..777c161 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
@@ -140,7 +140,18 @@ public enum Statistic {
   STREAM_WRITE_TOTAL_DATA("stream_write_total_data",
       "Count of total data uploaded in block output"),
   STREAM_WRITE_QUEUE_DURATION("stream_write_queue_duration",
-      "Total queue duration of all block uploads");
+      "Total queue duration of all block uploads"),
+
+  // S3Guard stats
+  S3GUARD_METADATASTORE_PUT_PATH_REQUEST(
+      "s3guard_metadatastore_put_path_request",
+      "s3guard metadata store put one metadata path request"),
+  S3GUARD_METADATASTORE_PUT_PATH_LATENCY(
+      "s3guard_metadatastore_put_path_latency",
+      "s3guard metadata store put one metadata path lantency"),
+  S3GUARD_METADATASTORE_INITIALIZATION("s3guard_metadatastore_initialization",
+      "s3guard metadata store initialization times");
+
 
   private static final Map<String, Statistic> SYMBOL_MAP =
       new HashMap<>(Statistic.values().length);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Tristate.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Tristate.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Tristate.java
new file mode 100644
index 0000000..0462ccf
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Tristate.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+/**
+ * Simple enum to express {true, false, don't know}.
+ */
+public enum Tristate {
+  // Do not add additional values here.  Logic will assume there are exactly
+  // three possibilities.
+  TRUE, FALSE, UNKNOWN;
+
+  public static Tristate fromBool(boolean v) {
+    return v ? TRUE : FALSE;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/UploadInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/UploadInfo.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/UploadInfo.java
new file mode 100644
index 0000000..238cd97
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/UploadInfo.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import com.amazonaws.services.s3.transfer.Upload;
+
+/**
+ * Simple struct that contains information about a S3 upload.
+ */
+public class UploadInfo {
+  private final Upload upload;
+  private final long length;
+
+  public UploadInfo(Upload upload, long length) {
+    this.upload = upload;
+    this.length = length;
+  }
+
+  public Upload getUpload() {
+    return upload;
+  }
+
+  public long getLength() {
+    return length;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DescendantsIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DescendantsIterator.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DescendantsIterator.java
new file mode 100644
index 0000000..dcee358
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DescendantsIterator.java
@@ -0,0 +1,142 @@
+/*
+ * 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.LinkedList;
+import java.util.NoSuchElementException;
+import java.util.Queue;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+
+/**
+ * {@code DescendantsIterator} is a {@link RemoteIterator} that implements
+ * pre-ordering breadth-first traversal (BFS) of a path and all of its
+ * descendants recursively.  After visiting each path, that path's direct
+ * children are discovered by calling {@link MetadataStore#listChildren(Path)}.
+ * Each iteration returns the next direct child, and if that child is a
+ * directory, also pushes it onto a queue to discover its children later.
+ *
+ * For example, assume the consistent store contains metadata representing this
+ * file system structure:
+ *
+ * <pre>
+ * /dir1
+ * |-- dir2
+ * |   |-- file1
+ * |   `-- file2
+ * `-- dir3
+ *     |-- dir4
+ *     |   `-- file3
+ *     |-- dir5
+ *     |   `-- file4
+ *     `-- dir6
+ * </pre>
+ *
+ * Consider this code sample:
+ * <pre>
+ * final PathMetadata dir1 = get(new Path("/dir1"));
+ * for (DescendantsIterator descendants = new DescendantsIterator(dir1);
+ *     descendants.hasNext(); ) {
+ *   final FileStatus status = descendants.next().getFileStatus();
+ *   System.out.printf("%s %s%n", status.isDirectory() ? 'D' : 'F',
+ *       status.getPath());
+ * }
+ * </pre>
+ *
+ * The output is:
+ * <pre>
+ * D /dir1
+ * D /dir1/dir2
+ * D /dir1/dir3
+ * F /dir1/dir2/file1
+ * F /dir1/dir2/file2
+ * D /dir1/dir3/dir4
+ * D /dir1/dir3/dir5
+ * F /dir1/dir3/dir4/file3
+ * F /dir1/dir3/dir5/file4
+ * D /dir1/dir3/dir6
+ * </pre>
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class DescendantsIterator implements RemoteIterator<FileStatus> {
+
+  private final MetadataStore metadataStore;
+  private final Queue<PathMetadata> queue = new LinkedList<>();
+
+  /**
+   * Creates a new {@code DescendantsIterator}.
+   *
+   * @param ms the associated {@link MetadataStore}
+   * @param meta base path for descendants iteration, which will be the first
+   *     returned during iteration (except root). Null makes empty iterator.
+   * @throws IOException if errors happen during metadata store listing
+   */
+  public DescendantsIterator(MetadataStore ms, PathMetadata meta)
+      throws IOException {
+    Preconditions.checkNotNull(ms);
+    this.metadataStore = ms;
+
+    if (meta != null) {
+      final Path path = meta.getFileStatus().getPath();
+      if (path.isRoot()) {
+        DirListingMetadata rootListing = ms.listChildren(path);
+        if (rootListing != null) {
+          rootListing = rootListing.withoutTombstones();
+          queue.addAll(rootListing.getListing());
+        }
+      } else {
+        queue.add(meta);
+      }
+    }
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
+    return !queue.isEmpty();
+  }
+
+  @Override
+  public FileStatus next() throws IOException {
+    if (!hasNext()) {
+      throw new NoSuchElementException("No more descendants.");
+    }
+    PathMetadata next;
+    next = queue.poll();
+    if (next.getFileStatus().isDirectory()) {
+      final Path path = next.getFileStatus().getPath();
+      DirListingMetadata meta = metadataStore.listChildren(path);
+      if (meta != null) {
+        Collection<PathMetadata> more = meta.withoutTombstones().getListing();
+        if (!more.isEmpty()) {
+          queue.addAll(more);
+        }
+      }
+    }
+    return next.getFileStatus();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DirListingMetadata.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DirListingMetadata.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DirListingMetadata.java
new file mode 100644
index 0000000..e5b4fb5
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DirListingMetadata.java
@@ -0,0 +1,322 @@
+/*
+ * 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.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.Tristate;
+
+/**
+ * {@code DirListingMetadata} models a directory listing stored in a
+ * {@link MetadataStore}.  Instances of this class are mutable and thread-safe.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class DirListingMetadata {
+
+  /**
+   * Convenience parameter for passing into constructor.
+   */
+  public static final Collection<PathMetadata> EMPTY_DIR =
+      Collections.emptyList();
+
+  private final Path path;
+
+  /** Using a map for fast find / remove with large directories. */
+  private Map<Path, PathMetadata> listMap = new ConcurrentHashMap<>();
+
+  private boolean isAuthoritative;
+
+  /**
+   * Create a directory listing metadata container.
+   *
+   * @param path Path of the directory. If this path has a host component, then
+   *     all paths added later via {@link #put(FileStatus)} must also have
+   *     the same host.
+   * @param listing Entries in the directory.
+   * @param isAuthoritative true iff listing is the full contents of the
+   *     directory, and the calling client reports that this may be cached as
+   *     the full and authoritative listing of all files in the directory.
+   */
+  public DirListingMetadata(Path path, Collection<PathMetadata> listing,
+      boolean isAuthoritative) {
+
+    checkPathAbsolute(path);
+    this.path = path;
+
+    if (listing != null) {
+      for (PathMetadata entry : listing) {
+        Path childPath = entry.getFileStatus().getPath();
+        checkChildPath(childPath);
+        listMap.put(childPath, entry);
+      }
+    }
+    this.isAuthoritative  = isAuthoritative;
+  }
+
+  /**
+   * Copy constructor.
+   * @param d the existing {@link DirListingMetadata} object.
+   */
+  public DirListingMetadata(DirListingMetadata d) {
+    path = d.path;
+    isAuthoritative = d.isAuthoritative;
+    listMap = new ConcurrentHashMap<>(d.listMap);
+  }
+
+  /**
+   * @return {@code Path} of the directory that contains this listing.
+   */
+  public Path getPath() {
+    return path;
+  }
+
+  /**
+   * @return entries in the directory
+   */
+  public Collection<PathMetadata> getListing() {
+    return Collections.unmodifiableCollection(listMap.values());
+  }
+
+  public Set<Path> listTombstones() {
+    Set<Path> tombstones = new HashSet<>();
+    for (PathMetadata meta : listMap.values()) {
+      if (meta.isDeleted()) {
+        tombstones.add(meta.getFileStatus().getPath());
+      }
+    }
+    return tombstones;
+  }
+
+  public DirListingMetadata withoutTombstones() {
+    Collection<PathMetadata> filteredList = new ArrayList<>();
+    for (PathMetadata meta : listMap.values()) {
+      if (!meta.isDeleted()) {
+        filteredList.add(meta);
+      }
+    }
+    return new DirListingMetadata(path, filteredList, isAuthoritative);
+  }
+
+  /**
+   * @return number of entries tracked.  This is not the same as the number
+   * of entries in the actual directory unless {@link #isAuthoritative()} is
+   * true.
+   */
+  public int numEntries() {
+    return listMap.size();
+  }
+
+  /**
+   * @return true iff this directory listing is full and authoritative within
+   * the scope of the {@code MetadataStore} that returned it.
+   */
+  public boolean isAuthoritative() {
+    return isAuthoritative;
+  }
+
+
+  /**
+   * Is the underlying directory known to be empty?
+   * @return FALSE if directory is known to have a child entry, TRUE if
+   * directory is known to be empty, UNKNOWN otherwise.
+   */
+  public Tristate isEmpty() {
+    if (getListing().isEmpty()) {
+      if (isAuthoritative()) {
+        return Tristate.TRUE;
+      } else {
+        // This listing is empty, but may not be full list of underlying dir.
+        return Tristate.UNKNOWN;
+      }
+    } else { // not empty listing
+      // There exists at least one child, dir not empty.
+      return Tristate.FALSE;
+    }
+  }
+
+  /**
+   * Marks this directory listing as full and authoritative.
+   * @param authoritative see {@link #isAuthoritative()}.
+   */
+  public void setAuthoritative(boolean authoritative) {
+    this.isAuthoritative = authoritative;
+  }
+
+  /**
+   * Lookup entry within this directory listing.  This may return null if the
+   * {@code MetadataStore} only tracks a partial set of the directory entries.
+   * In the case where {@link #isAuthoritative()} is true, however, this
+   * function returns null iff the directory is known not to contain the listing
+   * at given path (within the scope of the {@code MetadataStore} that returned
+   * it).
+   *
+   * @param childPath path of entry to look for.
+   * @return entry, or null if it is not present or not being tracked.
+   */
+  public PathMetadata get(Path childPath) {
+    checkChildPath(childPath);
+    return listMap.get(childPath);
+  }
+
+  /**
+   * Replace an entry with a tombstone.
+   * @param childPath path of entry to replace.
+   */
+  public void markDeleted(Path childPath) {
+    checkChildPath(childPath);
+    listMap.put(childPath, PathMetadata.tombstone(childPath));
+  }
+
+  /**
+   * Remove entry from this directory.
+   *
+   * @param childPath path of entry to remove.
+   */
+  public void remove(Path childPath) {
+    checkChildPath(childPath);
+    listMap.remove(childPath);
+  }
+
+  /**
+   * Add an entry to the directory listing.  If this listing already contains a
+   * {@code FileStatus} with the same path, it will be replaced.
+   *
+   * @param childFileStatus entry to add to this directory listing.
+   * @return true if the status was added or replaced with a new value. False
+   * if the same FileStatus value was already present.
+   */
+  public boolean put(FileStatus childFileStatus) {
+    Preconditions.checkNotNull(childFileStatus,
+        "childFileStatus must be non-null");
+    Path childPath = childStatusToPathKey(childFileStatus);
+    PathMetadata newValue = new PathMetadata(childFileStatus);
+    PathMetadata oldValue = listMap.put(childPath, newValue);
+    return oldValue == null || !oldValue.equals(newValue);
+  }
+
+  @Override
+  public String toString() {
+    return "DirListingMetadata{" +
+        "path=" + path +
+        ", listMap=" + listMap +
+        ", isAuthoritative=" + isAuthoritative +
+        '}';
+  }
+
+  /**
+   * Log contents to supplied StringBuilder in a pretty fashion.
+   * @param sb target StringBuilder
+   */
+  public void prettyPrint(StringBuilder sb) {
+    sb.append(String.format("DirMeta %-20s %-18s",
+        path.toString(),
+        isAuthoritative ? "Authoritative" : "Not Authoritative"));
+    for (Map.Entry<Path, PathMetadata> entry : listMap.entrySet()) {
+      sb.append("\n   key: ").append(entry.getKey()).append(": ");
+      entry.getValue().prettyPrint(sb);
+    }
+    sb.append("\n");
+  }
+
+  public String prettyPrint() {
+    StringBuilder sb = new StringBuilder();
+    prettyPrint(sb);
+    return sb.toString();
+  }
+
+  /**
+   * Checks that child path is valid.
+   * @param childPath path to check.
+   */
+  private void checkChildPath(Path childPath) {
+    checkPathAbsolute(childPath);
+
+    // If this dir's path has host (and thus scheme), so must its children
+    URI parentUri = path.toUri();
+    if (parentUri.getHost() != null) {
+      URI childUri = childPath.toUri();
+      Preconditions.checkNotNull(childUri.getHost(), "Expected non-null URI " +
+          "host: %s", childUri);
+      Preconditions.checkArgument(
+          childUri.getHost().equals(parentUri.getHost()),
+          "childUri %s and parentUri %s must have the same host",
+          childUri, parentUri);
+      Preconditions.checkNotNull(childUri.getScheme(), "No scheme in path %s",
+          childUri);
+    }
+    Preconditions.checkArgument(!childPath.isRoot(),
+        "childPath cannot be the root path: %s", childPath);
+    Preconditions.checkArgument(childPath.getParent().equals(path),
+        "childPath %s must be a child of %s", childPath, path);
+  }
+
+  /**
+   * For Paths that are handed in directly, we assert they are in consistent
+   * format with checkPath().  For paths that are supplied embedded in
+   * FileStatus, we attempt to fill in missing scheme and host, when this
+   * DirListingMetadata is associated with one.
+   *
+   * @return Path suitable for consistent hashtable lookups
+   * @throws NullPointerException null status argument
+   * @throws IllegalArgumentException bad status values or failure to
+   *                                  create a URI.
+   */
+  private Path childStatusToPathKey(FileStatus status) {
+    Path p = status.getPath();
+    Preconditions.checkNotNull(p, "Child status' path cannot be null");
+    Preconditions.checkArgument(!p.isRoot(),
+        "childPath cannot be the root path: %s", p);
+    Preconditions.checkArgument(p.getParent().equals(path),
+        "childPath %s must be a child of %s", p, path);
+    URI uri = p.toUri();
+    URI parentUri = path.toUri();
+    // If FileStatus' path is missing host, but should have one, add it.
+    if (uri.getHost() == null && parentUri.getHost() != null) {
+      try {
+        return new Path(new URI(parentUri.getScheme(), parentUri.getHost(),
+            uri.getPath(), uri.getFragment()));
+      } catch (URISyntaxException e) {
+        throw new IllegalArgumentException("FileStatus path invalid with" +
+            " added " + parentUri.getScheme() + "://" + parentUri.getHost() +
+            " added", e);
+      }
+    }
+    return p;
+  }
+
+  private void checkPathAbsolute(Path p) {
+    Preconditions.checkNotNull(p, "path must be non-null");
+    Preconditions.checkArgument(p.isAbsolute(), "path must be absolute: %s", p);
+  }
+}


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


[3/8] hadoop git commit: HADOOP-13345 S3Guard: Improved Consistency for S3A. Contributed by: Chris Nauroth, Aaron Fabbri, Mingliang Liu, Lei (Eddy) Xu, Sean Mackrory, Steve Loughran and others.

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..6cff533
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java
@@ -0,0 +1,544 @@
+/*
+ * 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;
+
+import com.amazonaws.services.s3.model.ObjectListing;
+import com.amazonaws.services.s3.AmazonS3;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.contract.s3a.S3AContract;
+import org.junit.Assume;
+import org.junit.Test;
+
+import java.io.FileNotFoundException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.writeTextFile;
+import static org.apache.hadoop.fs.s3a.Constants.*;
+import static org.apache.hadoop.fs.s3a.InconsistentAmazonS3Client.*;
+
+/**
+ * Test S3Guard list consistency feature by injecting delayed listObjects()
+ * visibility via {@link InconsistentAmazonS3Client}.
+ *
+ * Tests here generally:
+ * 1. Use the inconsistency injection mentioned above.
+ * 2. Only run when S3Guard is enabled.
+ */
+public class ITestS3GuardListConsistency extends AbstractS3ATestBase {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    conf.setClass(S3_CLIENT_FACTORY_IMPL, InconsistentS3ClientFactory.class,
+        S3ClientFactory.class);
+    // Other configs would break test assumptions
+    conf.set(FAIL_INJECT_INCONSISTENCY_KEY, DEFAULT_DELAY_KEY_SUBSTRING);
+    conf.setFloat(FAIL_INJECT_INCONSISTENCY_PROBABILITY, 1.0f);
+    conf.setLong(FAIL_INJECT_INCONSISTENCY_MSEC, DEFAULT_DELAY_KEY_MSEC);
+    return new S3AContract(conf);
+  }
+
+  /**
+   * Helper function for other test cases: does a single rename operation and
+   * validates the aftermath.
+   * @param mkdirs Directories to create
+   * @param srcdirs Source paths for rename operation
+   * @param dstdirs Destination paths for rename operation
+   * @param yesdirs Files that must exist post-rename (e.g. srcdirs children)
+   * @param nodirs Files that must not exist post-rename (e.g. dstdirs children)
+   * @throws Exception
+   */
+  private void doTestRenameSequence(Path[] mkdirs, Path[] srcdirs,
+      Path[] dstdirs, Path[] yesdirs, Path[] nodirs) throws Exception {
+    S3AFileSystem fs = getFileSystem();
+    Assume.assumeTrue(fs.hasMetadataStore());
+
+    if (mkdirs != null) {
+      for (Path mkdir : mkdirs) {
+        assertTrue(fs.mkdirs(mkdir));
+      }
+      clearInconsistency(fs);
+    }
+
+    assertTrue("srcdirs and dstdirs must have equal length",
+        srcdirs.length == dstdirs.length);
+    for (int i = 0; i < srcdirs.length; i++) {
+      assertTrue("Rename returned false: " + srcdirs[i] + " -> " + dstdirs[i],
+          fs.rename(srcdirs[i], dstdirs[i]));
+    }
+
+    for (Path yesdir : yesdirs) {
+      assertTrue("Path was supposed to exist: " + yesdir, fs.exists(yesdir));
+    }
+    for (Path nodir : nodirs) {
+      assertFalse("Path is not supposed to exist: " + nodir, fs.exists(nodir));
+    }
+  }
+
+  /**
+   * Tests that after renaming a directory, the original directory and its
+   * contents are indeed missing and the corresponding new paths are visible.
+   * @throws Exception
+   */
+  @Test
+  public void testConsistentListAfterRename() throws Exception {
+    Path[] mkdirs = {
+      path("d1/f"),
+      path("d1/f" + DEFAULT_DELAY_KEY_SUBSTRING)
+    };
+    Path[] srcdirs = {path("d1")};
+    Path[] dstdirs = {path("d2")};
+    Path[] yesdirs = {path("d2"), path("d2/f"),
+        path("d2/f" + DEFAULT_DELAY_KEY_SUBSTRING)};
+    Path[] nodirs = {path("d1"), path("d1/f"),
+        path("d1/f" + DEFAULT_DELAY_KEY_SUBSTRING)};
+    doTestRenameSequence(mkdirs, srcdirs, dstdirs, yesdirs, nodirs);
+    getFileSystem().delete(path("d1"), true);
+    getFileSystem().delete(path("d2"), true);
+  }
+
+  /**
+   * Tests a circular sequence of renames to verify that overwriting recently
+   * deleted files and reading recently created files from rename operations
+   * works as expected.
+   * @throws Exception
+   */
+  @Test
+  public void testRollingRenames() throws Exception {
+    Path[] dir0 = {path("rolling/1")};
+    Path[] dir1 = {path("rolling/2")};
+    Path[] dir2 = {path("rolling/3")};
+    // These sets have to be in reverse order compared to the movement
+    Path[] setA = {dir1[0], dir0[0]};
+    Path[] setB = {dir2[0], dir1[0]};
+    Path[] setC = {dir0[0], dir2[0]};
+
+    for(int i = 0; i < 2; i++) {
+      Path[] firstSet = i == 0 ? setA : null;
+      doTestRenameSequence(firstSet, setA, setB, setB, dir0);
+      doTestRenameSequence(null, setB, setC, setC, dir1);
+      doTestRenameSequence(null, setC, setA, setA, dir2);
+    }
+
+    S3AFileSystem fs = getFileSystem();
+    assertFalse("Renaming deleted file should have failed",
+        fs.rename(dir2[0], dir1[0]));
+    assertTrue("Renaming over existing file should have succeeded",
+        fs.rename(dir1[0], dir0[0]));
+  }
+
+  /**
+   * Tests that deleted files immediately stop manifesting in list operations
+   * even when the effect in S3 is delayed.
+   * @throws Exception
+   */
+  @Test
+  public void testConsistentListAfterDelete() throws Exception {
+    S3AFileSystem fs = getFileSystem();
+    // test will fail if NullMetadataStore (the default) is configured: skip it.
+    Assume.assumeTrue(fs.hasMetadataStore());
+
+    // Any S3 keys that contain DELAY_KEY_SUBSTRING will be delayed
+    // in listObjects() results via InconsistentS3Client
+    Path inconsistentPath =
+        path("a/b/dir3-" + DEFAULT_DELAY_KEY_SUBSTRING);
+
+    Path[] testDirs = {path("a/b/dir1"),
+        path("a/b/dir2"),
+        inconsistentPath};
+
+    for (Path path : testDirs) {
+      assertTrue(fs.mkdirs(path));
+    }
+    clearInconsistency(fs);
+    for (Path path : testDirs) {
+      assertTrue(fs.delete(path, false));
+    }
+
+    FileStatus[] paths = fs.listStatus(path("a/b/"));
+    List<Path> list = new ArrayList<>();
+    for (FileStatus fileState : paths) {
+      list.add(fileState.getPath());
+    }
+    assertFalse(list.contains(path("a/b/dir1")));
+    assertFalse(list.contains(path("a/b/dir2")));
+    // This should fail without S3Guard, and succeed with it.
+    assertFalse(list.contains(inconsistentPath));
+  }
+
+  /**
+   * Tests that rename immediately after files in the source directory are
+   * deleted results in exactly the correct set of destination files and none
+   * of the source files.
+   * @throws Exception
+   */
+  @Test
+  public void testConsistentRenameAfterDelete() throws Exception {
+    S3AFileSystem fs = getFileSystem();
+    // test will fail if NullMetadataStore (the default) is configured: skip it.
+    Assume.assumeTrue(fs.hasMetadataStore());
+
+    // Any S3 keys that contain DELAY_KEY_SUBSTRING will be delayed
+    // in listObjects() results via InconsistentS3Client
+    Path inconsistentPath =
+        path("a/b/dir3-" + DEFAULT_DELAY_KEY_SUBSTRING);
+
+    Path[] testDirs = {path("a/b/dir1"),
+        path("a/b/dir2"),
+        inconsistentPath};
+
+    for (Path path : testDirs) {
+      assertTrue(fs.mkdirs(path));
+    }
+    clearInconsistency(fs);
+    assertTrue(fs.delete(testDirs[1], false));
+    assertTrue(fs.delete(testDirs[2], false));
+
+    fs.rename(path("a"), path("a3"));
+    FileStatus[] paths = fs.listStatus(path("a3/b"));
+    List<Path> list = new ArrayList<>();
+    for (FileStatus fileState : paths) {
+      list.add(fileState.getPath());
+    }
+    assertTrue(list.contains(path("a3/b/dir1")));
+    assertFalse(list.contains(path("a3/b/dir2")));
+    // This should fail without S3Guard, and succeed with it.
+    assertFalse(list.contains(path("a3/b/dir3-" +
+        DEFAULT_DELAY_KEY_SUBSTRING)));
+
+    try {
+      RemoteIterator<LocatedFileStatus> old = fs.listFilesAndEmptyDirectories(
+          path("a"), true);
+      fail("Recently renamed dir should not be visible");
+    } catch(FileNotFoundException e) {
+      // expected
+    }
+  }
+
+  @Test
+  public void testConsistentListStatusAfterPut() throws Exception {
+
+    S3AFileSystem fs = getFileSystem();
+
+    // This test will fail if NullMetadataStore (the default) is configured:
+    // skip it.
+    Assume.assumeTrue(fs.hasMetadataStore());
+
+    // Any S3 keys that contain DELAY_KEY_SUBSTRING will be delayed
+    // in listObjects() results via InconsistentS3Client
+    Path inconsistentPath =
+        path("a/b/dir3-" + DEFAULT_DELAY_KEY_SUBSTRING);
+
+    Path[] testDirs = {path("a/b/dir1"),
+        path("a/b/dir2"),
+        inconsistentPath};
+
+    for (Path path : testDirs) {
+      assertTrue(fs.mkdirs(path));
+    }
+
+    FileStatus[] paths = fs.listStatus(path("a/b/"));
+    List<Path> list = new ArrayList<>();
+    for (FileStatus fileState : paths) {
+      list.add(fileState.getPath());
+    }
+    assertTrue(list.contains(path("a/b/dir1")));
+    assertTrue(list.contains(path("a/b/dir2")));
+    // This should fail without S3Guard, and succeed with it.
+    assertTrue(list.contains(inconsistentPath));
+  }
+
+  /**
+   * Similar to {@link #testConsistentListStatusAfterPut()}, this tests that the
+   * FS listLocatedStatus() call will return consistent list.
+   */
+  @Test
+  public void testConsistentListLocatedStatusAfterPut() throws Exception {
+    final S3AFileSystem fs = getFileSystem();
+    // This test will fail if NullMetadataStore (the default) is configured:
+    // skip it.
+    Assume.assumeTrue(fs.hasMetadataStore());
+    String rootDir = "doTestConsistentListLocatedStatusAfterPut";
+    fs.mkdirs(path(rootDir));
+
+    final int[] numOfPaths = {0, 1, 5};
+    for (int normalPathNum : numOfPaths) {
+      for (int delayedPathNum : new int[] {0, 2}) {
+        LOG.info("Testing with normalPathNum={}, delayedPathNum={}",
+            normalPathNum, delayedPathNum);
+        doTestConsistentListLocatedStatusAfterPut(fs, rootDir, normalPathNum,
+            delayedPathNum);
+      }
+    }
+  }
+
+  /**
+   * Helper method to implement the tests of consistent listLocatedStatus().
+   * @param fs The S3 file system from contract
+   * @param normalPathNum number paths listed directly from S3 without delaying
+   * @param delayedPathNum number paths listed with delaying
+   * @throws Exception
+   */
+  private void doTestConsistentListLocatedStatusAfterPut(S3AFileSystem fs,
+      String rootDir, int normalPathNum, int delayedPathNum) throws Exception {
+    final List<Path> testDirs = new ArrayList<>(normalPathNum + delayedPathNum);
+    int index = 0;
+    for (; index < normalPathNum; index++) {
+      testDirs.add(path(rootDir + "/dir-" +
+          index));
+    }
+    for (; index < normalPathNum + delayedPathNum; index++) {
+      // Any S3 keys that contain DELAY_KEY_SUBSTRING will be delayed
+      // in listObjects() results via InconsistentS3Client
+      testDirs.add(path(rootDir + "/dir-" + index +
+          DEFAULT_DELAY_KEY_SUBSTRING));
+    }
+
+    for (Path path : testDirs) {
+      // delete the old test path (if any) so that when we call mkdirs() later,
+      // the to delay directories will be tracked via putObject() request.
+      fs.delete(path, true);
+      assertTrue(fs.mkdirs(path));
+    }
+
+    // this should return the union data from S3 and MetadataStore
+    final RemoteIterator<LocatedFileStatus> statusIterator =
+        fs.listLocatedStatus(path(rootDir + "/"));
+    List<Path> list = new ArrayList<>();
+    for (; statusIterator.hasNext();) {
+      list.add(statusIterator.next().getPath());
+    }
+
+    // This should fail without S3Guard, and succeed with it because part of the
+    // children under test path are delaying visibility
+    for (Path path : testDirs) {
+      assertTrue("listLocatedStatus should list " + path, list.contains(path));
+    }
+  }
+
+  /**
+   * Tests that the S3AFS listFiles() call will return consistent file list.
+   */
+  @Test
+  public void testConsistentListFiles() throws Exception {
+    final S3AFileSystem fs = getFileSystem();
+    // This test will fail if NullMetadataStore (the default) is configured:
+    // skip it.
+    Assume.assumeTrue(fs.hasMetadataStore());
+
+    final int[] numOfPaths = {0, 2};
+    for (int dirNum : numOfPaths) {
+      for (int normalFile : numOfPaths) {
+        for (int delayedFile : new int[] {0, 1}) {
+          for (boolean recursive : new boolean[] {true, false}) {
+            doTestListFiles(fs, dirNum, normalFile, delayedFile, recursive);
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * Helper method to implement the tests of consistent listFiles().
+   *
+   * The file structure has dirNum subdirectories, and each directory (including
+   * the test base directory itself) has normalFileNum normal files and
+   * delayedFileNum delayed files.
+   *
+   * @param fs The S3 file system from contract
+   * @param dirNum number of subdirectories
+   * @param normalFileNum number files in each directory without delay to list
+   * @param delayedFileNum number files in each directory with delay to list
+   * @param recursive listFiles recursively if true
+   * @throws Exception if any unexpected error
+   */
+  private void doTestListFiles(S3AFileSystem fs, int dirNum, int normalFileNum,
+      int delayedFileNum, boolean recursive) throws Exception {
+    describe("Testing dirNum=%d, normalFile=%d, delayedFile=%d, "
+        + "recursive=%s", dirNum, normalFileNum, delayedFileNum, recursive);
+    final Path baseTestDir = path("doTestListFiles-" + dirNum + "-"
+        + normalFileNum + "-" + delayedFileNum + "-" + recursive);
+    // delete the old test path (if any) so that when we call mkdirs() later,
+    // the to delay sub directories will be tracked via putObject() request.
+    fs.delete(baseTestDir, true);
+
+    // make subdirectories (if any)
+    final List<Path> testDirs = new ArrayList<>(dirNum + 1);
+    assertTrue(fs.mkdirs(baseTestDir));
+    testDirs.add(baseTestDir);
+    for (int i = 0; i < dirNum; i++) {
+      final Path subdir = path(baseTestDir + "/dir-" + i);
+      assertTrue(fs.mkdirs(subdir));
+      testDirs.add(subdir);
+    }
+
+    final Collection<String> fileNames
+        = new ArrayList<>(normalFileNum + delayedFileNum);
+    int index = 0;
+    for (; index < normalFileNum; index++) {
+      fileNames.add("file-" + index);
+    }
+    for (; index < normalFileNum + delayedFileNum; index++) {
+      // Any S3 keys that contain DELAY_KEY_SUBSTRING will be delayed
+      // in listObjects() results via InconsistentS3Client
+      fileNames.add("file-" + index + "-" + DEFAULT_DELAY_KEY_SUBSTRING);
+    }
+
+    int filesAndEmptyDirectories = 0;
+
+    // create files under each test directory
+    for (Path dir : testDirs) {
+      for (String fileName : fileNames) {
+        writeTextFile(fs, new Path(dir, fileName), "I, " + fileName, false);
+        filesAndEmptyDirectories++;
+      }
+    }
+
+    // this should return the union data from S3 and MetadataStore
+    final RemoteIterator<LocatedFileStatus> statusIterator
+        = fs.listFiles(baseTestDir, recursive);
+    final Collection<Path> listedFiles = new HashSet<>();
+    for (; statusIterator.hasNext();) {
+      final FileStatus status = statusIterator.next();
+      assertTrue("FileStatus " + status + " is not a file!", status.isFile());
+      listedFiles.add(status.getPath());
+    }
+    LOG.info("S3AFileSystem::listFiles('{}', {}) -> {}",
+        baseTestDir, recursive, listedFiles);
+
+    // This should fail without S3Guard, and succeed with it because part of the
+    // files to list are delaying visibility
+    if (!recursive) {
+      // in this case only the top level files are listed
+      assertEquals("Unexpected number of files returned by listFiles() call",
+          normalFileNum + delayedFileNum, listedFiles.size());
+      verifyFileIsListed(listedFiles, baseTestDir, fileNames);
+    } else {
+      assertEquals("Unexpected number of files returned by listFiles() call",
+          filesAndEmptyDirectories,
+          listedFiles.size());
+      for (Path dir : testDirs) {
+        verifyFileIsListed(listedFiles, dir, fileNames);
+      }
+    }
+  }
+
+  private static void verifyFileIsListed(Collection<Path> listedFiles,
+      Path currentDir, Collection<String> fileNames) {
+    for (String fileName : fileNames) {
+      final Path file = new Path(currentDir, fileName);
+      assertTrue(file + " should have been listed", listedFiles.contains(file));
+    }
+  }
+
+  @Test
+  public void testCommitByRenameOperations() throws Throwable {
+    S3AFileSystem fs = getFileSystem();
+    Assume.assumeTrue(fs.hasMetadataStore());
+    Path work = path("test-commit-by-rename-" + DEFAULT_DELAY_KEY_SUBSTRING);
+    Path task00 = new Path(work, "task00");
+    fs.mkdirs(task00);
+    String name = "part-00";
+    try (FSDataOutputStream out =
+             fs.create(new Path(task00, name), false)) {
+      out.writeChars("hello");
+    }
+    for (FileStatus stat : fs.listStatus(task00)) {
+      fs.rename(stat.getPath(), work);
+    }
+    List<FileStatus> files = new ArrayList<>(2);
+    for (FileStatus stat : fs.listStatus(work)) {
+      if (stat.isFile()) {
+        files.add(stat);
+      }
+    }
+    assertFalse("renamed file " + name + " not found in " + work,
+        files.isEmpty());
+    assertEquals("more files found than expected in " + work
+        + " " + ls(work), 1, files.size());
+    FileStatus status = files.get(0);
+    assertEquals("Wrong filename in " + status,
+        name, status.getPath().getName());
+  }
+
+  @Test
+  public void testInconsistentS3ClientDeletes() throws Throwable {
+    S3AFileSystem fs = getFileSystem();
+    Path root = path("testInconsistentClient" + DEFAULT_DELAY_KEY_SUBSTRING);
+    for (int i = 0; i < 3; i++) {
+      fs.mkdirs(new Path(root, "dir" + i));
+      touch(fs, new Path(root, "file" + i));
+      for (int j = 0; j < 3; j++) {
+        touch(fs, new Path(new Path(root, "dir" + i), "file" + i + "-" + j));
+      }
+    }
+    clearInconsistency(fs);
+
+    AmazonS3 client = fs.getAmazonS3Client();
+    String key = fs.pathToKey(root) + "/";
+
+    ObjectListing preDeleteDelimited = client.listObjects(
+        fs.createListObjectsRequest(key, "/"));
+    ObjectListing preDeleteUndelimited = client.listObjects(
+        fs.createListObjectsRequest(key, null));
+
+    fs.delete(root, true);
+
+    ObjectListing postDeleteDelimited = client.listObjects(
+        fs.createListObjectsRequest(key, "/"));
+    ObjectListing postDeleteUndelimited = client.listObjects(
+        fs.createListObjectsRequest(key, null));
+
+    assertEquals("InconsistentAmazonS3Client added back objects incorrectly " +
+            "in a non-recursive listing",
+        preDeleteDelimited.getObjectSummaries().size(),
+        postDeleteDelimited.getObjectSummaries().size()
+    );
+    assertEquals("InconsistentAmazonS3Client added back prefixes incorrectly " +
+            "in a non-recursive listing",
+        preDeleteDelimited.getCommonPrefixes().size(),
+        postDeleteDelimited.getCommonPrefixes().size()
+    );
+    assertEquals("InconsistentAmazonS3Client added back objects incorrectly " +
+            "in a recursive listing",
+        preDeleteUndelimited.getObjectSummaries().size(),
+        postDeleteUndelimited.getObjectSummaries().size()
+    );
+    assertEquals("InconsistentAmazonS3Client added back prefixes incorrectly " +
+            "in a recursive listing",
+        preDeleteUndelimited.getCommonPrefixes().size(),
+        postDeleteUndelimited.getCommonPrefixes().size()
+    );
+  }
+
+  private static void clearInconsistency(S3AFileSystem fs) throws Exception {
+    AmazonS3 s3 = fs.getAmazonS3Client();
+    InconsistentAmazonS3Client ic = InconsistentAmazonS3Client.castFrom(s3);
+    ic.clearInconsistency();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardWriteBack.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardWriteBack.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardWriteBack.java
new file mode 100644
index 0000000..a63b696
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardWriteBack.java
@@ -0,0 +1,141 @@
+/*
+ * 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;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.s3guard.DirListingMetadata;
+import org.junit.Assume;
+import org.junit.Test;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
+import java.util.Arrays;
+
+/**
+ * Test cases that validate S3Guard's behavior for writing things like
+ * directory listings back to the MetadataStore.
+ */
+public class ITestS3GuardWriteBack extends AbstractS3ATestBase {
+
+  /**
+   * In listStatus(), when S3Guard is enabled, the full listing for a
+   * directory is "written back" to the MetadataStore before the listing is
+   * returned.  Currently this "write back" behavior occurs when
+   * fs.s3a.metadatastore.authoritative is true.  This test validates this
+   * behavior.
+   * @throws Exception on failure
+   */
+  @Test
+  public void testListStatusWriteBack() throws Exception {
+    Assume.assumeTrue(getFileSystem().hasMetadataStore());
+
+    Path directory = path("ListStatusWriteBack");
+
+    // "raw" S3AFileSystem without S3Guard
+    S3AFileSystem noS3Guard = createTestFS(directory.toUri(), true, false);
+
+    // Another with S3Guard and write-back disabled
+    S3AFileSystem noWriteBack = createTestFS(directory.toUri(), false, false);
+
+    // Another S3Guard and write-back enabled
+    S3AFileSystem yesWriteBack = createTestFS(directory.toUri(), false, true);
+
+    // delete the existing directory (in case of last test failure)
+    noS3Guard.delete(directory, true);
+    // Create a directory on S3 only
+    noS3Guard.mkdirs(new Path(directory, "OnS3"));
+    // Create a directory on both S3 and metadata store
+    Path p = new Path(directory, "OnS3AndMS");
+    assertPathDoesntExist(noWriteBack, p);
+    noWriteBack.mkdirs(p);
+
+    FileStatus[] fsResults;
+    DirListingMetadata mdResults;
+
+    // FS should return both even though S3Guard is not writing back to MS
+    fsResults = noWriteBack.listStatus(directory);
+    assertEquals("Filesystem enabled S3Guard without write back should have "
+            + "both /OnS3 and /OnS3AndMS: " + Arrays.toString(fsResults),
+        2, fsResults.length);
+
+    // Metadata store without write-back should still only contain /OnS3AndMS,
+    // because newly discovered /OnS3 is not written back to metadata store
+    mdResults = noWriteBack.getMetadataStore().listChildren(directory);
+    assertEquals("Metadata store without write back should still only know "
+            + "about /OnS3AndMS, but it has: " + mdResults,
+        1, mdResults.numEntries());
+
+    // FS should return both (and will write it back)
+    fsResults = yesWriteBack.listStatus(directory);
+    assertEquals("Filesystem enabled S3Guard with write back should have "
+            + " both /OnS3 and /OnS3AndMS: " + Arrays.toString(fsResults),
+        2, fsResults.length);
+
+    // Metadata store with write-back should contain both because the newly
+    // discovered /OnS3 should have been written back to metadata store
+    mdResults = yesWriteBack.getMetadataStore().listChildren(directory);
+    assertEquals("Unexpected number of results from metadata store. "
+            + "Should have /OnS3 and /OnS3AndMS: " + mdResults,
+        2, mdResults.numEntries());
+
+    // If we don't clean this up, the next test run will fail because it will
+    // have recorded /OnS3 being deleted even after it's written to noS3Guard.
+    getFileSystem().getMetadataStore().forgetMetadata(
+        new Path(directory, "OnS3"));
+  }
+
+  /** Create a separate S3AFileSystem instance for testing. */
+  private S3AFileSystem createTestFS(URI fsURI, boolean disableS3Guard,
+      boolean authoritativeMeta) throws IOException {
+    Configuration conf;
+
+    // Create a FileSystem that is S3-backed only
+    conf = createConfiguration();
+    S3ATestUtils.disableFilesystemCaching(conf);
+    if (disableS3Guard) {
+      conf.set(Constants.S3_METADATA_STORE_IMPL,
+          Constants.S3GUARD_METASTORE_NULL);
+    } else {
+      S3ATestUtils.maybeEnableS3Guard(conf);
+      conf.setBoolean(Constants.METADATASTORE_AUTHORITATIVE, authoritativeMeta);
+    }
+    FileSystem fs = FileSystem.get(fsURI, conf);
+    return asS3AFS(fs);
+  }
+
+  private static S3AFileSystem asS3AFS(FileSystem fs) {
+    assertTrue("Not a S3AFileSystem: " + fs, fs instanceof S3AFileSystem);
+    return (S3AFileSystem)fs;
+  }
+
+  private static void assertPathDoesntExist(FileSystem fs, Path p)
+      throws IOException {
+    try {
+      FileStatus s = fs.getFileStatus(p);
+    } catch (FileNotFoundException e) {
+      return;
+    }
+    fail("Path should not exist: " + p);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java
index 9e0a5e4..4e25380 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java
@@ -23,6 +23,7 @@ import static org.mockito.Mockito.*;
 import java.net.URI;
 
 import com.amazonaws.services.s3.AmazonS3;
+import com.amazonaws.services.s3.model.Region;
 
 /**
  * An {@link S3ClientFactory} that returns Mockito mocks of the {@link AmazonS3}
@@ -35,6 +36,8 @@ public class MockS3ClientFactory implements S3ClientFactory {
     String bucket = name.getHost();
     AmazonS3 s3 = mock(AmazonS3.class);
     when(s3.doesBucketExist(bucket)).thenReturn(true);
+    when(s3.getBucketLocation(anyString()))
+        .thenReturn(Region.US_West.toString());
     return s3;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java
index acbe610..2c4f009 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java
@@ -135,6 +135,18 @@ public interface S3ATestConstants {
   String TEST_STS_ENDPOINT = "test.fs.s3a.sts.endpoint";
 
   /**
+   * Various S3Guard tests.
+   */
+  String TEST_S3GUARD_PREFIX = "fs.s3a.s3guard.test";
+  String TEST_S3GUARD_ENABLED = TEST_S3GUARD_PREFIX + ".enabled";
+  String TEST_S3GUARD_AUTHORITATIVE = TEST_S3GUARD_PREFIX + ".authoritative";
+  String TEST_S3GUARD_IMPLEMENTATION = TEST_S3GUARD_PREFIX + ".implementation";
+  String TEST_S3GUARD_IMPLEMENTATION_LOCAL = "local";
+  String TEST_S3GUARD_IMPLEMENTATION_DYNAMO = "dynamo";
+  String TEST_S3GUARD_IMPLEMENTATION_DYNAMODBLOCAL = "dynamodblocal";
+  String TEST_S3GUARD_IMPLEMENTATION_NONE = "none";
+
+  /**
    * Timeout in Milliseconds for standard tests: {@value}.
    */
   int S3A_TEST_TIMEOUT = 10 * 60 * 1000;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
index 9528967..8dbf90a 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
@@ -22,7 +22,14 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.s3a.s3guard.DynamoDBClientFactory;
+import org.apache.hadoop.fs.s3a.s3guard.DynamoDBLocalClientFactory;
+import org.apache.hadoop.fs.s3a.s3guard.S3Guard;
+
+import org.hamcrest.core.Is;
 import org.junit.Assert;
 import org.junit.Assume;
 import org.junit.internal.AssumptionViolatedException;
@@ -31,11 +38,13 @@ import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.net.URI;
+import java.net.URISyntaxException;
 import java.util.List;
 
 import static org.apache.hadoop.fs.contract.ContractTestUtils.skip;
 import static org.apache.hadoop.fs.s3a.S3ATestConstants.*;
 import static org.apache.hadoop.fs.s3a.Constants.*;
+import static org.apache.hadoop.fs.s3a.S3AUtils.propagateBucketOptions;
 import static org.junit.Assert.*;
 
 /**
@@ -52,6 +61,15 @@ public final class S3ATestUtils {
   public static final String UNSET_PROPERTY = "unset";
 
   /**
+   * Get S3A FS name.
+   * @param conf configuration.
+   * @return S3A fs name.
+   */
+  public static String getFsName(Configuration conf) {
+    return conf.getTrimmed(TEST_FS_S3A_NAME, "");
+  }
+
+  /**
    * Create the test filesystem.
    *
    * If the test.fs.s3a.name property is not set, this will
@@ -97,6 +115,8 @@ public final class S3ATestUtils {
       throw new AssumptionViolatedException(
           "No test filesystem in " + TEST_FS_S3A_NAME);
     }
+    // patch in S3Guard options
+    maybeEnableS3Guard(conf);
     S3AFileSystem fs1 = new S3AFileSystem();
     //enable purging in tests
     if (purge) {
@@ -137,6 +157,8 @@ public final class S3ATestUtils {
       throw new AssumptionViolatedException("No test filesystem in "
           + TEST_FS_S3A_NAME);
     }
+    // patch in S3Guard options
+    maybeEnableS3Guard(conf);
     FileContext fc = FileContext.getFileContext(testURI, conf);
     return fc;
   }
@@ -301,13 +323,96 @@ public final class S3ATestUtils {
    * @return a path
    */
   public static Path createTestPath(Path defVal) {
-    String testUniqueForkId = System.getProperty(
-        S3ATestConstants.TEST_UNIQUE_FORK_ID);
+    String testUniqueForkId =
+        System.getProperty(S3ATestConstants.TEST_UNIQUE_FORK_ID);
     return testUniqueForkId == null ? defVal :
         new Path("/" + testUniqueForkId, "test");
   }
 
   /**
+   * Test assumption that S3Guard is/is not enabled.
+   * @param shouldBeEnabled should S3Guard be enabled?
+   * @param originalConf configuration to check
+   * @throws URISyntaxException
+   */
+  public static void assumeS3GuardState(boolean shouldBeEnabled,
+      Configuration originalConf) throws URISyntaxException {
+    boolean isEnabled = getTestPropertyBool(originalConf, TEST_S3GUARD_ENABLED,
+        originalConf.getBoolean(TEST_S3GUARD_ENABLED, false));
+    Assume.assumeThat("Unexpected S3Guard test state:"
+            + " shouldBeEnabled=" + shouldBeEnabled
+            + " and isEnabled=" + isEnabled,
+        shouldBeEnabled, Is.is(isEnabled));
+
+    final String fsname = originalConf.getTrimmed(TEST_FS_S3A_NAME);
+    Assume.assumeNotNull(fsname);
+    final String bucket = new URI(fsname).getHost();
+    final Configuration conf = propagateBucketOptions(originalConf, bucket);
+    boolean usingNullImpl = S3GUARD_METASTORE_NULL.equals(
+        conf.getTrimmed(S3_METADATA_STORE_IMPL, S3GUARD_METASTORE_NULL));
+    Assume.assumeThat("Unexpected S3Guard test state:"
+            + " shouldBeEnabled=" + shouldBeEnabled
+            + " but usingNullImpl=" + usingNullImpl,
+        shouldBeEnabled, Is.is(!usingNullImpl));
+  }
+
+  /**
+   * Conditionally set the S3Guard options from test properties.
+   * @param conf configuration
+   */
+  public static void maybeEnableS3Guard(Configuration conf) {
+    if (getTestPropertyBool(conf, TEST_S3GUARD_ENABLED,
+        conf.getBoolean(TEST_S3GUARD_ENABLED, false))) {
+      // S3Guard is enabled.
+      boolean authoritative = getTestPropertyBool(conf,
+          TEST_S3GUARD_AUTHORITATIVE,
+          conf.getBoolean(TEST_S3GUARD_AUTHORITATIVE, true));
+      String impl = getTestProperty(conf, TEST_S3GUARD_IMPLEMENTATION,
+          conf.get(TEST_S3GUARD_IMPLEMENTATION,
+              TEST_S3GUARD_IMPLEMENTATION_LOCAL));
+      String implClass = "";
+      switch (impl) {
+      case TEST_S3GUARD_IMPLEMENTATION_LOCAL:
+        implClass = S3GUARD_METASTORE_LOCAL;
+        break;
+      case TEST_S3GUARD_IMPLEMENTATION_DYNAMODBLOCAL:
+        conf.setClass(S3Guard.S3GUARD_DDB_CLIENT_FACTORY_IMPL,
+            DynamoDBLocalClientFactory.class, DynamoDBClientFactory.class);
+      case TEST_S3GUARD_IMPLEMENTATION_DYNAMO:
+        implClass = S3GUARD_METASTORE_DYNAMO;
+        break;
+      case TEST_S3GUARD_IMPLEMENTATION_NONE:
+        implClass = S3GUARD_METASTORE_NULL;
+        break;
+      default:
+        fail("Unknown s3guard back end: \"" + impl + "\"");
+      }
+      LOG.debug("Enabling S3Guard, authoritative={}, implementation={}",
+          authoritative, implClass);
+      conf.setBoolean(METADATASTORE_AUTHORITATIVE, authoritative);
+      conf.set(S3_METADATA_STORE_IMPL, implClass);
+      conf.setBoolean(S3GUARD_DDB_TABLE_CREATE_KEY, true);
+    }
+  }
+
+  /**
+   * Is there a MetadataStore configured for s3a with authoritative enabled?
+   * @param conf Configuration to test.
+   * @return true iff there is a MetadataStore configured, and it is
+   * configured allow authoritative results.  This can result in reducing
+   * round trips to S3 service for cached results, which may affect FS/FC
+   * statistics.
+   */
+  public static boolean isMetadataStoreAuthoritative(Configuration conf) {
+    if (conf == null) {
+      return Constants.DEFAULT_METADATASTORE_AUTHORITATIVE;
+    }
+    return conf.getBoolean(
+        Constants.METADATASTORE_AUTHORITATIVE,
+        Constants.DEFAULT_METADATASTORE_AUTHORITATIVE);
+  }
+
+  /**
    * Reset all metrics in a list.
    * @param metrics metrics to reset
    */
@@ -504,6 +609,94 @@ public final class S3ATestUtils {
   }
 
   /**
+   * Verify the core size, block size and timestamp values of a file.
+   * @param status status entry to check
+   * @param size file size
+   * @param blockSize block size
+   * @param modTime modified time
+   */
+  public static void verifyFileStatus(FileStatus status, long size,
+      long blockSize, long modTime) {
+    verifyFileStatus(status, size, 0, modTime, 0, blockSize, null, null, null);
+  }
+
+  /**
+   * Verify the status entry of a file matches that expected.
+   * @param status status entry to check
+   * @param size file size
+   * @param replication replication factor (may be 0)
+   * @param modTime modified time
+   * @param accessTime access time (may be 0)
+   * @param blockSize block size
+   * @param owner owner (may be null)
+   * @param group user group (may be null)
+   * @param permission permission (may be null)
+   */
+  public static void verifyFileStatus(FileStatus status,
+      long size,
+      int replication,
+      long modTime,
+      long accessTime,
+      long blockSize,
+      String owner,
+      String group,
+      FsPermission permission) {
+    String details = status.toString();
+    assertFalse("Not a dir: " + details, status.isDirectory());
+    assertEquals("Mod time: " + details, modTime, status.getModificationTime());
+    assertEquals("File size: " + details, size, status.getLen());
+    assertEquals("Block size: " + details, blockSize, status.getBlockSize());
+    if (replication > 0) {
+      assertEquals("Replication value: " + details, replication,
+          status.getReplication());
+    }
+    if (accessTime != 0) {
+      assertEquals("Access time: " + details, accessTime,
+          status.getAccessTime());
+    }
+    if (owner != null) {
+      assertEquals("Owner: " + details, owner, status.getOwner());
+    }
+    if (group != null) {
+      assertEquals("Group: " + details, group, status.getGroup());
+    }
+    if (permission != null) {
+      assertEquals("Permission: " + details, permission,
+          status.getPermission());
+    }
+  }
+
+  /**
+   * Verify the status entry of a directory matches that expected.
+   * @param status status entry to check
+   * @param replication replication factor
+   * @param modTime modified time
+   * @param accessTime access time
+   * @param owner owner
+   * @param group user group
+   * @param permission permission.
+   */
+  public static void verifyDirStatus(FileStatus status,
+      int replication,
+      long modTime,
+      long accessTime,
+      String owner,
+      String group,
+      FsPermission permission) {
+    String details = status.toString();
+    assertTrue("Is a dir: " + details, status.isDirectory());
+    assertEquals("zero length: " + details, 0, status.getLen());
+
+    assertEquals("Mod time: " + details, modTime, status.getModificationTime());
+    assertEquals("Replication value: " + details, replication,
+        status.getReplication());
+    assertEquals("Access time: " + details, accessTime, status.getAccessTime());
+    assertEquals("Owner: " + details, owner, status.getOwner());
+    assertEquals("Group: " + details, group, status.getGroup());
+    assertEquals("Permission: " + details, permission, status.getPermission());
+  }
+
+  /**
    * Set a bucket specific property to a particular value.
    * If the generic key passed in has an {@code fs.s3a. prefix},
    * that's stripped off, so that when the the bucket properties are propagated

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestListing.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestListing.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestListing.java
new file mode 100644
index 0000000..e647327
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestListing.java
@@ -0,0 +1,118 @@
+/*
+ * 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;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import java.util.Set;
+
+import static org.apache.hadoop.fs.s3a.Listing.ACCEPT_ALL;
+import static org.apache.hadoop.fs.s3a.Listing.ProvidedFileStatusIterator;
+
+/**
+ * Place for the S3A listing classes; keeps all the small classes under control.
+ */
+public class TestListing extends AbstractS3AMockTest {
+
+  private static class MockRemoteIterator<FileStatus> implements
+      RemoteIterator<FileStatus> {
+    private Iterator<FileStatus> iterator;
+
+    MockRemoteIterator(Collection<FileStatus> source) {
+      iterator = source.iterator();
+    }
+
+    public boolean hasNext() {
+      return iterator.hasNext();
+    }
+
+    public FileStatus next() {
+      return iterator.next();
+    }
+  }
+
+  private FileStatus blankFileStatus(Path path) {
+    return new FileStatus(0, true, 0, 0, 0, path);
+  }
+
+  @Test
+  public void testTombstoneReconcilingIterator() throws Exception {
+    Path parent = new Path("/parent");
+    Path liveChild = new Path(parent, "/liveChild");
+    Path deletedChild = new Path(parent, "/deletedChild");
+    Path[] allFiles = {parent, liveChild, deletedChild};
+    Path[] liveFiles = {parent, liveChild};
+
+    Listing listing = new Listing(fs);
+    Collection<FileStatus> statuses = new ArrayList<>();
+    statuses.add(blankFileStatus(parent));
+    statuses.add(blankFileStatus(liveChild));
+    statuses.add(blankFileStatus(deletedChild));
+
+    Set<Path> tombstones = new HashSet<>();
+    tombstones.add(deletedChild);
+
+    RemoteIterator<FileStatus> sourceIterator = new MockRemoteIterator(
+        statuses);
+    RemoteIterator<LocatedFileStatus> locatedIterator =
+        listing.createLocatedFileStatusIterator(sourceIterator);
+    RemoteIterator<LocatedFileStatus> reconcilingIterator =
+        listing.createTombstoneReconcilingIterator(locatedIterator, tombstones);
+
+    Set<Path> expectedPaths = new HashSet<>();
+    expectedPaths.add(parent);
+    expectedPaths.add(liveChild);
+
+    Set<Path> actualPaths = new HashSet<>();
+    while (reconcilingIterator.hasNext()) {
+      actualPaths.add(reconcilingIterator.next().getPath());
+    }
+    Assert.assertTrue(actualPaths.equals(expectedPaths));
+  }
+
+  @Test
+  public void testProvidedFileStatusIteratorEnd() throws Exception {
+    FileStatus[] statuses = {
+        new FileStatus(100, false, 1, 8192, 0, new Path("s3a://blah/blah"))
+    };
+    ProvidedFileStatusIterator it = new ProvidedFileStatusIterator(statuses,
+        ACCEPT_ALL, new Listing.AcceptAllButS3nDirs());
+
+    Assert.assertTrue("hasNext() should return true first time", it.hasNext());
+    Assert.assertNotNull("first element should not be null", it.next());
+    Assert.assertFalse("hasNext() should now be false", it.hasNext());
+    try {
+      it.next();
+      Assert.fail("next() should have thrown exception");
+    } catch (NoSuchElementException e) {
+      // Correct behavior.  Any other exceptions are propagated as failure.
+      return;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java
index e1aef75..e493818 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java
@@ -39,7 +39,9 @@ public class ITestS3AFileContextStatistics extends FCStatisticsBaseTest {
 
   @After
   public void tearDown() throws Exception {
-    fc.delete(fileContextTestHelper.getTestRootPath(fc, "test"), true);
+    if (fc != null) {
+      fc.delete(fileContextTestHelper.getTestRootPath(fc, "test"), true);
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextURI.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextURI.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextURI.java
index fff1fcb..725646c 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextURI.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextURI.java
@@ -16,19 +16,29 @@ package org.apache.hadoop.fs.s3a.fileContext;
 import java.io.IOException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileContextURIBase;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
 import org.apache.hadoop.fs.s3a.S3ATestUtils;
 import org.junit.Before;
 import org.junit.Ignore;
 import org.junit.Test;
 
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.createTestFileSystem;
+
 /**
  * S3a implementation of FileContextURIBase.
  */
 public class ITestS3AFileContextURI extends FileContextURIBase {
 
+  private Configuration conf;
+  private boolean hasMetadataStore;
+
   @Before
   public void setUp() throws IOException, Exception {
-    Configuration conf = new Configuration();
+    conf = new Configuration();
+    try(S3AFileSystem s3aFS = createTestFileSystem(conf)) {
+      hasMetadataStore = s3aFS.hasMetadataStore();
+    }
     fc1 = S3ATestUtils.createTestFileContext(conf);
     fc2 = S3ATestUtils.createTestFileContext(conf); //different object, same FS
     super.setUp();
@@ -41,4 +51,11 @@ public class ITestS3AFileContextURI extends FileContextURIBase {
     // (the statistics tested with this method are not relevant for an S3FS)
   }
 
+  @Test
+  @Override
+  public void testModificationTime() throws IOException {
+    // skip modtime tests as there may be some inconsistency during creation
+    assume("modification time tests are skipped", !hasMetadataStore);
+    super.testModificationTime();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractMSContract.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractMSContract.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractMSContract.java
new file mode 100644
index 0000000..921d4a6
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractMSContract.java
@@ -0,0 +1,33 @@
+/*
+ * 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.FileSystem;
+
+import java.io.IOException;
+
+/**
+ * Test specification for MetadataStore contract tests. Supplies configuration
+ * and MetadataStore instance.
+ */
+public abstract class AbstractMSContract {
+
+  public abstract FileSystem getFileSystem() throws IOException;
+  public abstract MetadataStore getMetadataStore() throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..ceacdf3
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java
@@ -0,0 +1,161 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.concurrent.TimeUnit;
+
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.s3a.AbstractS3ATestBase;
+import org.apache.hadoop.fs.s3a.Constants;
+import org.apache.hadoop.fs.s3a.S3AFileStatus;
+import org.apache.hadoop.fs.s3a.S3ATestUtils;
+import org.apache.hadoop.io.IOUtils;
+
+import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.SUCCESS;
+
+/**
+ * Common functionality for S3GuardTool test cases.
+ */
+public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
+
+  protected static final String OWNER = "hdfs";
+
+  private MetadataStore ms;
+
+  protected static void expectResult(int expected,
+      String message,
+      S3GuardTool tool,
+      String... args) throws Exception {
+    assertEquals(message, expected, tool.run(args));
+  }
+
+  protected static void expectSuccess(
+      String message,
+      S3GuardTool tool,
+      String... args) throws Exception {
+    assertEquals(message, SUCCESS, tool.run(args));
+  }
+
+  protected MetadataStore getMetadataStore() {
+    return ms;
+  }
+
+  protected abstract MetadataStore newMetadataStore();
+
+  @Override
+  public void setup() throws Exception {
+    super.setup();
+    S3ATestUtils.assumeS3GuardState(true, getConfiguration());
+    ms = newMetadataStore();
+    ms.initialize(getFileSystem());
+  }
+
+  @Override
+  public void teardown() throws Exception {
+    super.teardown();
+    IOUtils.cleanupWithLogger(LOG, ms);
+  }
+
+  protected void mkdirs(Path path, boolean onS3, boolean onMetadataStore)
+      throws IOException {
+    if (onS3) {
+      getFileSystem().mkdirs(path);
+    }
+    if (onMetadataStore) {
+      S3AFileStatus status = new S3AFileStatus(true, path, OWNER);
+      ms.put(new PathMetadata(status));
+    }
+  }
+
+  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();
+    }
+  }
+
+  /**
+   * Create file either on S3 or in metadata store.
+   * @param path the file path.
+   * @param onS3 set to true to create the file on S3.
+   * @param onMetadataStore set to true to create the file on the
+   *                        metadata store.
+   * @throws IOException IO problem
+   */
+  protected void createFile(Path path, boolean onS3, boolean onMetadataStore)
+      throws IOException {
+    if (onS3) {
+      ContractTestUtils.touch(getFileSystem(), path);
+    }
+
+    if (onMetadataStore) {
+      S3AFileStatus status = new S3AFileStatus(100L, System.currentTimeMillis(),
+          getFileSystem().qualify(path), 512L, "hdfs");
+      putFile(ms, status);
+    }
+  }
+
+  private void testPruneCommand(Configuration cmdConf, String...args)
+      throws Exception {
+    Path parent = path("prune-cli");
+    try {
+      getFileSystem().mkdirs(parent);
+
+      S3GuardTool.Prune cmd = new S3GuardTool.Prune(cmdConf);
+      cmd.setMetadataStore(ms);
+
+      createFile(new Path(parent, "stale"), true, true);
+      Thread.sleep(TimeUnit.SECONDS.toMillis(2));
+      createFile(new Path(parent, "fresh"), true, true);
+
+      assertEquals(2, ms.listChildren(parent).getListing().size());
+      expectSuccess("Prune command did not exit successfully - see output", cmd,
+          args);
+      assertEquals(1, ms.listChildren(parent).getListing().size());
+    } finally {
+      getFileSystem().delete(parent, true);
+      ms.prune(Long.MAX_VALUE);
+    }
+  }
+
+  @Test
+  public void testPruneCommandCLI() throws Exception {
+    String testPath = path("testPruneCommandCLI").toString();
+    testPruneCommand(getFileSystem().getConf(),
+        "prune", "-seconds", "1", testPath);
+  }
+
+  @Test
+  public void testPruneCommandConf() throws Exception {
+    getConfiguration().setLong(Constants.S3GUARD_CLI_PRUNE_AGE,
+        TimeUnit.SECONDS.toMillis(1));
+    String testPath = path("testPruneCommandConf").toString();
+    testPruneCommand(getConfiguration(), "prune", testPath);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBLocalClientFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBLocalClientFactory.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBLocalClientFactory.java
new file mode 100644
index 0000000..0291acd
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBLocalClientFactory.java
@@ -0,0 +1,157 @@
+/*
+ * 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.File;
+import java.io.IOException;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.client.builder.AwsClientBuilder;
+import com.amazonaws.services.dynamodbv2.AmazonDynamoDB;
+import com.amazonaws.services.dynamodbv2.AmazonDynamoDBClientBuilder;
+import com.amazonaws.services.dynamodbv2.local.main.ServerRunner;
+import com.amazonaws.services.dynamodbv2.local.server.DynamoDBProxyServer;
+import org.apache.commons.lang3.StringUtils;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.s3a.DefaultS3ClientFactory;
+import org.apache.hadoop.net.ServerSocketUtil;
+
+import static org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProviderSet;
+import static org.apache.hadoop.fs.s3a.s3guard.DynamoDBClientFactory.DefaultDynamoDBClientFactory.getRegion;
+
+/**
+ * A DynamoDBClientFactory implementation that creates AmazonDynamoDB clients
+ * against an in-memory DynamoDBLocal server instance.
+ *
+ * You won't be charged bills for issuing any DynamoDB requests. However, the
+ * DynamoDBLocal is considered a simulator of the DynamoDB web service, so it
+ * may be stale or different. For example, the throttling is not yet supported
+ * in DynamoDBLocal. This is for testing purpose only.
+ *
+ * To use this for creating DynamoDB client in tests:
+ * <ol>
+ * <li>
+ *    As all DynamoDBClientFactory implementations, this should be configured.
+ * </li>
+ * <li>
+ *    The singleton DynamoDBLocal server instance is started automatically when
+ *    creating the AmazonDynamoDB client for the first time. It still merits to
+ *    launch the server before all the tests and fail fast if error happens.
+ * </li>
+ * <li>
+ *    The server can be stopped explicitly, which is not actually needed in
+ *    tests as JVM termination will do that.
+ * </li>
+ * </ol>
+ *
+ * @see DefaultDynamoDBClientFactory
+ */
+public class DynamoDBLocalClientFactory extends Configured
+    implements DynamoDBClientFactory {
+
+  /** The DynamoDBLocal dynamoDBLocalServer instance for testing. */
+  private static DynamoDBProxyServer dynamoDBLocalServer;
+  private static String ddbEndpoint;
+
+  private static final String SYSPROP_SQLITE_LIB = "sqlite4java.library.path";
+
+  @Override
+  public AmazonDynamoDB createDynamoDBClient(String defaultRegion)
+      throws IOException {
+    startSingletonServer();
+
+    final Configuration conf = getConf();
+    final AWSCredentialsProvider credentials =
+        createAWSCredentialProviderSet(null, conf);
+    final ClientConfiguration awsConf =
+        DefaultS3ClientFactory.createAwsConf(conf);
+    // fail fast in case of service errors
+    awsConf.setMaxErrorRetry(3);
+
+    final String region = getRegion(conf, defaultRegion);
+    LOG.info("Creating DynamoDBLocal client using endpoint {} in region {}",
+        ddbEndpoint, region);
+
+    return AmazonDynamoDBClientBuilder.standard()
+        .withCredentials(credentials)
+        .withClientConfiguration(awsConf)
+        .withEndpointConfiguration(
+            new AwsClientBuilder.EndpointConfiguration(ddbEndpoint, region))
+        .build();
+  }
+
+  /**
+   * Start a singleton in-memory DynamoDBLocal server if not started yet.
+   * @throws IOException if any error occurs
+   */
+  public synchronized static void startSingletonServer() throws IOException {
+    if (dynamoDBLocalServer != null) {
+      return;
+    }
+
+    // Set this property if it has not been set elsewhere
+    if (StringUtils.isEmpty(System.getProperty(SYSPROP_SQLITE_LIB))) {
+      String projectBuildDir = System.getProperty("project.build.directory");
+      if (StringUtils.isEmpty(projectBuildDir)) {
+        projectBuildDir = "target";
+      }
+      // sqlite4java lib should have been copied to $projectBuildDir/native-libs
+      System.setProperty(SYSPROP_SQLITE_LIB,
+          projectBuildDir + File.separator + "native-libs");
+      LOG.info("Setting {} -> {}",
+          SYSPROP_SQLITE_LIB, System.getProperty(SYSPROP_SQLITE_LIB));
+    }
+
+    try {
+      // Start an in-memory local DynamoDB instance
+      final String port = String.valueOf(ServerSocketUtil.getPort(0, 100));
+      ddbEndpoint = "http://localhost:" + port;
+      dynamoDBLocalServer = ServerRunner.createServerFromCommandLineArgs(
+          new String[]{"-inMemory", "-port", port});
+      dynamoDBLocalServer.start();
+      LOG.info("DynamoDBLocal singleton server was started at {}", ddbEndpoint);
+    } catch (Exception t) {
+      String msg = "Error starting DynamoDBLocal server at " + ddbEndpoint
+          + " " + t;
+      LOG.error(msg, t);
+      throw new IOException(msg, t);
+    }
+  }
+
+  /**
+   * Stop the in-memory DynamoDBLocal server if it is started.
+   * @throws IOException if any error occurs
+   */
+  public synchronized static void stopSingletonServer() throws IOException {
+    if (dynamoDBLocalServer != null) {
+      LOG.info("Shutting down the in-memory DynamoDBLocal server");
+      try {
+        dynamoDBLocalServer.stop();
+      } catch (Throwable t) {
+        String msg = "Error stopping DynamoDBLocal server at " + ddbEndpoint;
+        LOG.error(msg, t);
+        throw new IOException(msg, t);
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardConcurrentOps.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardConcurrentOps.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardConcurrentOps.java
new file mode 100644
index 0000000..c6838a0
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardConcurrentOps.java
@@ -0,0 +1,160 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.s3guard;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.amazonaws.services.dynamodbv2.document.DynamoDB;
+import com.amazonaws.services.dynamodbv2.document.Table;
+import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException;
+import org.junit.Assume;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.s3a.AbstractS3ATestBase;
+import org.apache.hadoop.fs.s3a.Constants;
+
+import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_DDB_REGION_KEY;
+
+/**
+ * Tests concurrent operations on S3Guard.
+ */
+public class ITestS3GuardConcurrentOps extends AbstractS3ATestBase {
+
+  @Rule
+  public final Timeout timeout = new Timeout(5 * 60 * 1000);
+
+  private void failIfTableExists(DynamoDB db, String tableName) {
+    boolean tableExists = true;
+    try {
+      Table table = db.getTable(tableName);
+      table.describe();
+    } catch (ResourceNotFoundException e) {
+      tableExists = false;
+    }
+    if (tableExists) {
+      fail("Table already exists: " + tableName);
+    }
+  }
+
+  private void deleteTable(DynamoDB db, String tableName) throws
+      InterruptedException {
+    try {
+      Table table = db.getTable(tableName);
+      table.waitForActive();
+      table.delete();
+      table.waitForDelete();
+    } catch (ResourceNotFoundException e) {
+      LOG.warn("Failed to delete {}, as it was not found", tableName, e);
+    }
+  }
+
+  @Test
+  public void testConcurrentTableCreations() throws Exception {
+    final Configuration conf = getConfiguration();
+    Assume.assumeTrue("Test only applies when DynamoDB is used for S3Guard",
+        conf.get(Constants.S3_METADATA_STORE_IMPL).equals(
+            Constants.S3GUARD_METASTORE_DYNAMO));
+
+    DynamoDBMetadataStore ms = new DynamoDBMetadataStore();
+    ms.initialize(getFileSystem());
+    DynamoDB db = ms.getDynamoDB();
+
+    String tableName = "testConcurrentTableCreations" + new Random().nextInt();
+    conf.setBoolean(Constants.S3GUARD_DDB_TABLE_CREATE_KEY, true);
+    conf.set(Constants.S3GUARD_DDB_TABLE_NAME_KEY, tableName);
+
+    String region = conf.getTrimmed(S3GUARD_DDB_REGION_KEY);
+    if (StringUtils.isEmpty(region)) {
+      // no region set, so pick it up from the test bucket
+      conf.set(S3GUARD_DDB_REGION_KEY, getFileSystem().getBucketLocation());
+    }
+    int concurrentOps = 16;
+    int iterations = 4;
+
+    failIfTableExists(db, tableName);
+
+    for (int i = 0; i < iterations; i++) {
+      ExecutorService executor = Executors.newFixedThreadPool(
+          concurrentOps, new ThreadFactory() {
+            private AtomicInteger count = new AtomicInteger(0);
+
+            public Thread newThread(Runnable r) {
+              return new Thread(r,
+                  "testConcurrentTableCreations" + count.getAndIncrement());
+            }
+          });
+      ((ThreadPoolExecutor) executor).prestartAllCoreThreads();
+      Future<Exception>[] futures = new Future[concurrentOps];
+      for (int f = 0; f < concurrentOps; f++) {
+        final int index = f;
+        futures[f] = executor.submit(new Callable<Exception>() {
+          @Override
+          public Exception call() throws Exception {
+
+            ContractTestUtils.NanoTimer timer =
+                new ContractTestUtils.NanoTimer();
+
+            Exception result = null;
+            try (DynamoDBMetadataStore store = new DynamoDBMetadataStore()) {
+              store.initialize(conf);
+            } catch (Exception e) {
+              LOG.error(e.getClass() + ": " + e.getMessage());
+              result = e;
+            }
+
+            timer.end("Parallel DynamoDB client creation %d", index);
+            LOG.info("Parallel DynamoDB client creation {} ran from {} to {}",
+                index, timer.getStartTime(), timer.getEndTime());
+            return result;
+          }
+        });
+      }
+      List<Exception> exceptions = new ArrayList<>(concurrentOps);
+      for (int f = 0; f < concurrentOps; f++) {
+        Exception outcome = futures[f].get();
+        if (outcome != null) {
+          exceptions.add(outcome);
+        }
+      }
+      deleteTable(db, tableName);
+      int exceptionsThrown = exceptions.size();
+      if (exceptionsThrown > 0) {
+        // at least one exception was thrown. Fail the test & nest the first
+        // exception caught
+        throw new AssertionError(exceptionsThrown + "/" + concurrentOps +
+            " threads threw exceptions while initializing on iteration " + i,
+            exceptions.get(0));
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java
new file mode 100644
index 0000000..c13dfc4
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java
@@ -0,0 +1,134 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.Random;
+import java.util.concurrent.Callable;
+
+import com.amazonaws.services.dynamodbv2.document.DynamoDB;
+import com.amazonaws.services.dynamodbv2.document.Table;
+import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.Destroy;
+import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.Init;
+import org.apache.hadoop.test.LambdaTestUtils;
+
+/**
+ * Test S3Guard related CLI commands against DynamoDB.
+ */
+public class ITestS3GuardToolDynamoDB extends AbstractS3GuardToolTestBase {
+
+  @Override
+  protected MetadataStore newMetadataStore() {
+    return new DynamoDBMetadataStore();
+  }
+
+  // Check the existence of a given DynamoDB table.
+  private static boolean exist(DynamoDB dynamoDB, String tableName) {
+    assertNotNull(dynamoDB);
+    assertNotNull(tableName);
+    assertFalse("empty table name", tableName.isEmpty());
+    try {
+      Table table = dynamoDB.getTable(tableName);
+      table.describe();
+    } catch (ResourceNotFoundException e) {
+      return false;
+    }
+    return true;
+  }
+
+  @Test
+  public void testInvalidRegion() throws Exception {
+    final String testTableName = "testInvalidRegion" + new Random().nextInt();
+    final String testRegion = "invalidRegion";
+    // Initialize MetadataStore
+    final Init initCmd = new Init(getFileSystem().getConf());
+    LambdaTestUtils.intercept(IOException.class,
+        new Callable<String>() {
+          @Override
+          public String call() throws Exception {
+            int res = initCmd.run(new String[]{
+                "init",
+                "-region", testRegion,
+                "-meta", "dynamodb://" + testTableName
+            });
+            return "Use of invalid region did not fail, returning " + res
+                + "- table may have been " +
+                "created and not cleaned up: " + testTableName;
+          }
+        });
+  }
+
+  @Test
+  public void testDynamoDBInitDestroyCycle() throws Exception {
+    String testTableName = "testDynamoDBInitDestroy" + new Random().nextInt();
+    String testS3Url = path(testTableName).toString();
+    S3AFileSystem fs = getFileSystem();
+    DynamoDB db = null;
+    try {
+      // Initialize MetadataStore
+      Init initCmd = new Init(fs.getConf());
+      expectSuccess("Init command did not exit successfully - see output",
+          initCmd,
+          "init", "-meta", "dynamodb://" + testTableName, testS3Url);
+      // Verify it exists
+      MetadataStore ms = getMetadataStore();
+      assertTrue("metadata store should be DynamoDBMetadataStore",
+          ms instanceof DynamoDBMetadataStore);
+      DynamoDBMetadataStore dynamoMs = (DynamoDBMetadataStore) ms;
+      db = dynamoMs.getDynamoDB();
+      assertTrue(String.format("%s does not exist", testTableName),
+          exist(db, testTableName));
+
+      // Destroy MetadataStore
+      Destroy destroyCmd = new Destroy(fs.getConf());
+
+      expectSuccess("Destroy command did not exit successfully - see output",
+          destroyCmd,
+          "destroy", "-meta", "dynamodb://" + testTableName, testS3Url);
+      // Verify it does not exist
+      assertFalse(String.format("%s still exists", testTableName),
+          exist(db, testTableName));
+
+      // delete again and expect success again
+      expectSuccess("Destroy command did not exit successfully - see output",
+          destroyCmd,
+          "destroy", "-meta", "dynamodb://" + testTableName, testS3Url);
+    } catch (ResourceNotFoundException e) {
+      throw new AssertionError(
+          String.format("DynamoDB table %s does not exist", testTableName),
+          e);
+    } finally {
+      LOG.warn("Table may have not been cleaned up: " +
+          testTableName);
+      if (db != null) {
+        Table table = db.getTable(testTableName);
+        if (table != null) {
+          try {
+            table.delete();
+            table.waitForDelete();
+          } catch (ResourceNotFoundException e) { /* Ignore */ }
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1afc6aa/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java
new file mode 100644
index 0000000..181cdfb
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java
@@ -0,0 +1,149 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.BufferedReader;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.PrintStream;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.Diff;
+
+import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.SUCCESS;
+
+/**
+ * Test S3Guard related CLI commands against a LocalMetadataStore.
+ */
+public class ITestS3GuardToolLocal extends AbstractS3GuardToolTestBase {
+
+  @Override
+  protected MetadataStore newMetadataStore() {
+    return new LocalMetadataStore();
+  }
+
+  @Test
+  public void testImportCommand() throws Exception {
+    S3AFileSystem fs = getFileSystem();
+    MetadataStore ms = getMetadataStore();
+    Path parent = path("test-import");
+    fs.mkdirs(parent);
+    Path dir = new Path(parent, "a");
+    fs.mkdirs(dir);
+    Path emptyDir = new Path(parent, "emptyDir");
+    fs.mkdirs(emptyDir);
+    for (int i = 0; i < 10; i++) {
+      String child = String.format("file-%d", i);
+      try (FSDataOutputStream out = fs.create(new Path(dir, child))) {
+        out.write(1);
+      }
+    }
+
+    S3GuardTool.Import cmd = new S3GuardTool.Import(fs.getConf());
+    cmd.setStore(ms);
+
+    expectSuccess("Import command did not exit successfully - see output",
+        cmd,
+        "import", parent.toString());
+
+    DirListingMetadata children =
+        ms.listChildren(dir);
+    assertEquals("Unexpected number of paths imported", 10, children
+        .getListing().size());
+    assertEquals("Expected 2 items: empty directory and a parent directory", 2,
+        ms.listChildren(parent).getListing().size());
+    // assertTrue(children.isAuthoritative());
+  }
+
+  @Test
+  public void testDiffCommand() throws IOException {
+    S3AFileSystem fs = getFileSystem();
+    MetadataStore ms = getMetadataStore();
+    Set<Path> filesOnS3 = new HashSet<>(); // files on S3.
+    Set<Path> filesOnMS = new HashSet<>(); // files on metadata store.
+
+    Path testPath = path("test-diff");
+    mkdirs(testPath, true, true);
+
+    Path msOnlyPath = new Path(testPath, "ms_only");
+    mkdirs(msOnlyPath, false, true);
+    filesOnMS.add(msOnlyPath);
+    for (int i = 0; i < 5; i++) {
+      Path file = new Path(msOnlyPath, String.format("file-%d", i));
+      createFile(file, false, true);
+      filesOnMS.add(file);
+    }
+
+    Path s3OnlyPath = new Path(testPath, "s3_only");
+    mkdirs(s3OnlyPath, true, false);
+    filesOnS3.add(s3OnlyPath);
+    for (int i = 0; i < 5; i++) {
+      Path file = new Path(s3OnlyPath, String.format("file-%d", i));
+      createFile(file, true, false);
+      filesOnS3.add(file);
+    }
+
+    ByteArrayOutputStream buf = new ByteArrayOutputStream();
+    PrintStream out = new PrintStream(buf);
+    Diff cmd = new Diff(fs.getConf());
+    cmd.setStore(ms);
+    assertEquals("Diff command did not exit successfully - see output", SUCCESS,
+        cmd.run(new String[]{"diff", "-meta", "local://metadata",
+            testPath.toString()}, out));
+    out.close();
+
+    Set<Path> actualOnS3 = new HashSet<>();
+    Set<Path> actualOnMS = new HashSet<>();
+    boolean duplicates = false;
+    try (BufferedReader reader =
+             new BufferedReader(new InputStreamReader(
+                 new ByteArrayInputStream(buf.toByteArray())))) {
+      String line;
+      while ((line = reader.readLine()) != null) {
+        String[] fields = line.split("\\s");
+        assertEquals("[" + line + "] does not have enough fields",
+            4, fields.length);
+        String where = fields[0];
+        Path path = new Path(fields[3]);
+        if (Diff.S3_PREFIX.equals(where)) {
+          duplicates = duplicates || actualOnS3.contains(path);
+          actualOnS3.add(path);
+        } else if (Diff.MS_PREFIX.equals(where)) {
+          duplicates = duplicates || actualOnMS.contains(path);
+          actualOnMS.add(path);
+        } else {
+          fail("Unknown prefix: " + where);
+        }
+      }
+    }
+    String actualOut = out.toString();
+    assertEquals("Mismatched metadata store outputs: " + actualOut,
+        filesOnMS, actualOnMS);
+    assertEquals("Mismatched s3 outputs: " + actualOut, filesOnS3, actualOnS3);
+    assertFalse("Diff contained duplicates", duplicates);
+  }
+}


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