Sfoglia il codice sorgente

HADOOP-16424. S3Guard fsck: Check internal consistency of the MetadataStore (#1691). Contributed by Gabor Bota.

Gabor Bota 5 anni fa
parent
commit
875a3e97dd

+ 255 - 4
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java

@@ -24,12 +24,21 @@ import java.security.InvalidParameterException;
 import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Queue;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
+import com.amazonaws.services.dynamodbv2.document.Item;
+import com.amazonaws.services.dynamodbv2.document.PrimaryKey;
+import com.amazonaws.services.dynamodbv2.document.ScanOutcome;
+import com.amazonaws.services.dynamodbv2.document.Table;
+import com.amazonaws.services.dynamodbv2.document.internal.IteratorSupport;
+import com.amazonaws.services.dynamodbv2.document.spec.GetItemSpec;
+import com.amazonaws.services.dynamodbv2.xspec.ExpressionSpecBuilder;
 import com.google.common.base.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -38,10 +47,14 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.s3a.S3AFileStatus;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.Tristate;
 import org.apache.hadoop.util.StopWatch;
 
 import static java.util.stream.Collectors.toList;
 import static java.util.stream.Collectors.toSet;
+import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.itemToPathMetadata;
+import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.pathToKey;
+import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.pathToParentKey;
 
 /**
  * Main class for the FSCK factored out from S3GuardTool
@@ -51,6 +64,7 @@ import static java.util.stream.Collectors.toSet;
  * Functions:
  * <ul>
  *   <li>Checking metadata consistency between S3 and metadatastore</li>
+ *   <li>Checking the internal metadata consistency</li>
  * </ul>
  */
 public class S3GuardFsck {
@@ -306,7 +320,7 @@ public class S3GuardFsck {
     }
 
     if(msPathMetadata.getFileStatus().getVersionId() == null
-        || s3FileStatus.getVersionId() == null ) {
+        || s3FileStatus.getVersionId() == null) {
       LOG.debug("Missing versionIDs skipped. A HEAD request is "
           + "required for each object to get the versionID.");
     } else if(!s3FileStatus.getVersionId().equals(msFileStatus.getVersionId())) {
@@ -349,7 +363,11 @@ public class S3GuardFsck {
       this.msPathMetadata = pm;
       this.s3DirListing = null;
       this.msDirListing = null;
-      this.path = status.getPath();
+      if (status != null) {
+        this.path = status.getPath();
+      } else {
+        this.path = pm.getFileStatus().getPath();
+      }
     }
 
     ComparePair(List<FileStatus> s3DirListing, DirListingMetadata msDirListing) {
@@ -396,6 +414,230 @@ public class S3GuardFsck {
     }
   }
 
+  /**
+   * Check the DynamoDB metadatastore internally for consistency.
+   * <pre>
+   * Tasks to do here:
+   *  - find orphan entries (entries without a parent).
+   *  - find if a file's parent is not a directory (so the parent is a file).
+   *  - find entries where the parent is a tombstone.
+   *  - warn: no lastUpdated field.
+   * </pre>
+   */
+  public List<ComparePair> checkDdbInternalConsistency(Path basePath)
+      throws IOException {
+    Preconditions.checkArgument(basePath.isAbsolute(), "path must be absolute");
+
+    List<ComparePair> comparePairs = new ArrayList<>();
+    String rootStr = basePath.toString();
+    LOG.info("Root for internal consistency check: {}", rootStr);
+    StopWatch stopwatch = new StopWatch();
+    stopwatch.start();
+
+    final Table table = metadataStore.getTable();
+    final String username = metadataStore.getUsername();
+    DDBTree ddbTree = new DDBTree();
+
+    /*
+     * I. Root node construction
+     * - If the root node is the real bucket root, a node is constructed instead of
+     *   doing a query to the ddb because the bucket root is not stored.
+     * - If the root node is not a real bucket root then the entry is queried from
+     *   the ddb and constructed from the result.
+     */
+
+    DDBPathMetadata baseMeta;
+
+    if (!basePath.isRoot()) {
+      PrimaryKey rootKey = pathToKey(basePath);
+      final GetItemSpec spec = new GetItemSpec()
+          .withPrimaryKey(rootKey)
+          .withConsistentRead(true);
+      final Item baseItem = table.getItem(spec);
+      baseMeta = itemToPathMetadata(baseItem, username);
+
+      if (baseMeta == null) {
+        throw new FileNotFoundException(
+            "Base element metadata is null. " +
+                "This means the base path element is missing, or wrong path was " +
+                "passed as base path to the internal ddb consistency checker.");
+      }
+    } else {
+      baseMeta = new DDBPathMetadata(
+          new S3AFileStatus(Tristate.UNKNOWN, basePath, username)
+      );
+    }
+
+    DDBTreeNode root = new DDBTreeNode(baseMeta);
+    ddbTree.addNode(root);
+    ddbTree.setRoot(root);
+
+    /*
+     * II. Build and check the descendant tree:
+     * 1. query all nodes where the prefix is the given root, and put it in the tree
+     * 2. Check connectivity: check if each parent is in the hashmap
+     *    - This is done in O(n): we only need to find the parent based on the
+     *      path with a hashmap lookup.
+     *    - Do a test if the graph is connected - if the parent is not in the
+     *      hashmap, we found an orphan entry.
+     *
+     * 3. Do test the elements for errors:
+     *    - File is a parent of a file.
+     *    - Entries where the parent is tombstoned but the entries are not.
+     *    - Warn on no lastUpdated field.
+     *
+     */
+    ExpressionSpecBuilder builder = new ExpressionSpecBuilder();
+    builder.withCondition(
+        ExpressionSpecBuilder.S("parent")
+            .beginsWith(pathToParentKey(basePath))
+    );
+    final IteratorSupport<Item, ScanOutcome> resultIterator = table.scan(
+        builder.buildForScan()).iterator();
+    resultIterator.forEachRemaining(item -> {
+      final DDBPathMetadata pmd = itemToPathMetadata(item, username);
+      DDBTreeNode ddbTreeNode = new DDBTreeNode(pmd);
+      ddbTree.addNode(ddbTreeNode);
+    });
+
+    LOG.debug("Root: {}", ddbTree.getRoot());
+
+    for (Map.Entry<Path, DDBTreeNode> entry : ddbTree.getContentMap().entrySet()) {
+      final DDBTreeNode node = entry.getValue();
+      final ComparePair pair = new ComparePair(null, node.val);
+      // let's skip the root node when checking.
+      if (node.getVal().getFileStatus().getPath().isRoot()) {
+        continue;
+      }
+
+      if(node.getVal().getLastUpdated() == 0) {
+        pair.violations.add(Violation.NO_LASTUPDATED_FIELD);
+      }
+
+      // skip further checking the basenode which is not the actual bucket root.
+      if (node.equals(ddbTree.getRoot())) {
+        continue;
+      }
+
+      final Path parent = node.getFileStatus().getPath().getParent();
+      final DDBTreeNode parentNode = ddbTree.getContentMap().get(parent);
+      if (parentNode == null) {
+        pair.violations.add(Violation.ORPHAN_DDB_ENTRY);
+      } else {
+        if (!node.isTombstoned() && !parentNode.isDirectory()) {
+          pair.violations.add(Violation.PARENT_IS_A_FILE);
+        }
+        if(!node.isTombstoned() && parentNode.isTombstoned()) {
+          pair.violations.add(Violation.PARENT_TOMBSTONED);
+        }
+      }
+
+      if (!pair.violations.isEmpty()) {
+        comparePairs.add(pair);
+      }
+
+      node.setParent(parentNode);
+    }
+
+    // Create a handler and handle each violated pairs
+    S3GuardFsckViolationHandler handler =
+        new S3GuardFsckViolationHandler(rawFS, metadataStore);
+    comparePairs.forEach(handler::handle);
+
+    stopwatch.stop();
+    LOG.info("Total scan time: {}s", stopwatch.now(TimeUnit.SECONDS));
+    LOG.info("Scanned entries: {}", ddbTree.contentMap.size());
+
+    return comparePairs;
+  }
+
+  /**
+   * DDBTree is the tree that represents the structure of items in the DynamoDB.
+   */
+  public static class DDBTree {
+    private final Map<Path, DDBTreeNode> contentMap = new HashMap<>();
+    private DDBTreeNode root;
+
+    public DDBTree() {
+    }
+
+    public Map<Path, DDBTreeNode> getContentMap() {
+      return contentMap;
+    }
+
+    public DDBTreeNode getRoot() {
+      return root;
+    }
+
+    public void setRoot(DDBTreeNode root) {
+      this.root = root;
+    }
+
+    public void addNode(DDBTreeNode pm) {
+      contentMap.put(pm.getVal().getFileStatus().getPath(), pm);
+    }
+
+    @Override
+    public String toString() {
+      return "DDBTree{" +
+          "contentMap=" + contentMap +
+          ", root=" + root +
+          '}';
+    }
+  }
+
+  /**
+   * Tree node for DDBTree.
+   */
+  private static final class DDBTreeNode {
+    private final DDBPathMetadata val;
+    private DDBTreeNode parent;
+    private final List<DDBPathMetadata> children;
+
+    private DDBTreeNode(DDBPathMetadata pm) {
+      this.val = pm;
+      this.parent = null;
+      this.children = new ArrayList<>();
+    }
+
+    public DDBPathMetadata getVal() {
+      return val;
+    }
+
+    public DDBTreeNode getParent() {
+      return parent;
+    }
+
+    public void setParent(DDBTreeNode parent) {
+      this.parent = parent;
+    }
+
+    public List<DDBPathMetadata> getChildren() {
+      return children;
+    }
+
+    public boolean isDirectory() {
+      return val.getFileStatus().isDirectory();
+    }
+
+    public S3AFileStatus getFileStatus() {
+      return val.getFileStatus();
+    }
+
+    public boolean isTombstoned() {
+      return val.isDeleted();
+    }
+
+    @Override
+    public String toString() {
+      return "DDBTreeNode{" +
+          "val=" + val +
+          ", parent=" + parent +
+          ", children=" + children +
+          '}';
+    }
+  }
+
   /**
    * Violation with severity and the handler.
    * Defines the severity of the violation between 0-2
@@ -437,7 +679,7 @@ public class S3GuardFsck {
     AUTHORITATIVE_DIRECTORY_CONTENT_MISMATCH(1,
         S3GuardFsckViolationHandler.AuthDirContentMismatch.class),
     /**
-     * An entry in the MS is tombstoned, but the object is not deleted on S3
+     * An entry in the MS is tombstoned, but the object is not deleted on S3.
      */
     TOMBSTONED_IN_MS_NOT_DELETED_IN_S3(0,
         S3GuardFsckViolationHandler.TombstonedInMsNotDeletedInS3.class),
@@ -462,7 +704,16 @@ public class S3GuardFsck {
      * Don't worry too much if we don't have an etag.
      */
     NO_ETAG(2,
-        S3GuardFsckViolationHandler.NoEtag.class);
+        S3GuardFsckViolationHandler.NoEtag.class),
+    /**
+     * The entry does not have a parent in ddb.
+     */
+    ORPHAN_DDB_ENTRY(0, S3GuardFsckViolationHandler.OrphanDDBEntry.class),
+    /**
+     * The entry's lastUpdated field is empty.
+     */
+    NO_LASTUPDATED_FIELD(2,
+        S3GuardFsckViolationHandler.NoLastUpdatedField.class);
 
     private final int severity;
     private final Class<? extends S3GuardFsckViolationHandler.ViolationHandler> handler;

+ 30 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java

@@ -343,4 +343,34 @@ public class S3GuardFsckViolationHandler {
       return "The entry for the path is tombstoned in the MS.";
     }
   }
+
+  /**
+   * The violation handler there's no parent in the MetadataStore.
+   */
+  public static class OrphanDDBEntry extends ViolationHandler {
+
+    public OrphanDDBEntry(S3GuardFsck.ComparePair comparePair) {
+      super(comparePair);
+    }
+
+    @Override
+    public String getError() {
+      return "The DDB entry is orphan - there is no parent in the MS.";
+    }
+  }
+
+  /**
+   * The violation handler when there's no last updated field for the entry.
+   */
+  public static class NoLastUpdatedField extends ViolationHandler {
+
+    public NoLastUpdatedField(S3GuardFsck.ComparePair comparePair) {
+      super(comparePair);
+    }
+
+    @Override
+    public String getError() {
+      return "No lastUpdated field provided for the entry.";
+    }
+  }
 }

+ 33 - 9
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java

@@ -1544,6 +1544,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
    */
   static class Fsck extends S3GuardTool {
     public static final String CHECK_FLAG = "check";
+    public static final String DDB_MS_CONSISTENCY_FLAG = "internal";
 
     public static final String NAME = "fsck";
     public static final String PURPOSE = "Compares S3 with MetadataStore, and "
@@ -1553,10 +1554,12 @@ public abstract class S3GuardTool extends Configured implements Tool {
         "\t" + PURPOSE + "\n\n" +
         "Common options:\n" +
         "  -" + CHECK_FLAG + " Check the metadata store for errors, but do "
-        + "not fix any issues.\n";
+        + "not fix any issues.\n"+
+        "  -" + DDB_MS_CONSISTENCY_FLAG + " Check the dynamodb metadata store "
+        + "for internal consistency.\n";
 
     Fsck(Configuration conf) {
-      super(conf, CHECK_FLAG);
+      super(conf, CHECK_FLAG, DDB_MS_CONSISTENCY_FLAG);
     }
 
     @Override
@@ -1578,6 +1581,21 @@ public abstract class S3GuardTool extends Configured implements Tool {
       }
       int exitValue = EXIT_SUCCESS;
 
+      final CommandFormat commandFormat = getCommandFormat();
+
+      // check if there's more than one arguments
+      int flags = 0;
+      if (commandFormat.getOpt(CHECK_FLAG)) {
+        flags++;
+      }
+      if (commandFormat.getOpt(DDB_MS_CONSISTENCY_FLAG)) {
+        flags++;
+      }
+      if (flags > 1) {
+        out.println(USAGE);
+        throw invalidArgs("There should be only one parameter used for checking.");
+      }
+
       String s3Path = paths.get(0);
       try {
         initS3AFileSystem(s3Path);
@@ -1600,25 +1618,25 @@ public abstract class S3GuardTool extends Configured implements Tool {
 
       if (ms == null ||
           !(ms instanceof DynamoDBMetadataStore)) {
-        errorln(s3Path + " path uses MS: " + ms);
+        errorln(s3Path + " path uses metadata store: " + ms);
         errorln(NAME + " can be only used with a DynamoDB backed s3a bucket.");
         errorln(USAGE);
         return ERROR;
       }
 
-      final CommandFormat commandFormat = getCommandFormat();
+      List<S3GuardFsck.ComparePair> violations;
+
       if (commandFormat.getOpt(CHECK_FLAG)) {
         // do the check
         S3GuardFsck s3GuardFsck = new S3GuardFsck(fs, ms);
         try {
-          final List<S3GuardFsck.ComparePair> comparePairs
-              = s3GuardFsck.compareS3ToMs(fs.qualify(root));
-          if (comparePairs.size() > 0) {
-            exitValue = EXIT_FAIL;
-          }
+          violations = s3GuardFsck.compareS3ToMs(fs.qualify(root));
         } catch (IOException e) {
           throw e;
         }
+      } else if (commandFormat.getOpt(DDB_MS_CONSISTENCY_FLAG)) {
+        S3GuardFsck s3GuardFsck = new S3GuardFsck(fs, ms);
+        violations = s3GuardFsck.checkDdbInternalConsistency(fs.qualify(root));
       } else {
         errorln("No supported operation is selected.");
         errorln(USAGE);
@@ -1626,6 +1644,12 @@ public abstract class S3GuardTool extends Configured implements Tool {
       }
 
       out.flush();
+
+      // We fail if there were compare pairs, as the returned compare pairs
+      // contain issues.
+      if (violations == null || violations.size() > 0) {
+        exitValue = EXIT_FAIL;
+      }
       return exitValue;
     }
   }

+ 38 - 1
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md

@@ -923,10 +923,47 @@ Metadata Store Diagnostics:
   table={ ... }
   write-capacity=20
 ```
-
 *Note*: There is a limit to how many times in a 24 hour period the capacity
 of a bucket can be changed, either through this command or the AWS console.
 
+### Check the consistency of the metadata store, `s3guard fsck`
+
+Compares S3 with MetadataStore, and returns a failure status if any
+rules or invariants are violated. Only works with DynamoDB metadata stores.
+
+```bash
+hadoop s3guard fsck [-check | -internal] (s3a://BUCKET | s3a://PATH_PREFIX)
+```
+
+`-check` operation checks the metadata store from the S3 perspective, but
+does not fix any issues.
+The consistency issues will be logged in ERROR loglevel.
+
+`-internal` operation checks the internal consistency of the metadata store,
+but does not fix any issues.
+
+The errors found will be logged at the ERROR log level.
+
+*Note*: `-check` and `-internal` operations can be used only as separate
+commands. Running `fsck` with both will result in an error.
+
+Example
+
+```bash
+hadoop s3guard fsck -check s3a://ireland-1/path_prefix/
+```
+
+Checks the metadata store while iterating through the S3 bucket.
+The path_prefix will be used as the root element of the check.
+
+```bash
+hadoop s3guard fsck -internal s3a://ireland-1/path_prefix/
+```
+
+Checks the metadata store internal consistency.
+The path_prefix will be used as the root element of the check.
+
+
 ## Debugging and Error Handling
 
 If you run into network connectivity issues, or have a machine failure in the

+ 118 - 2
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java

@@ -435,10 +435,126 @@ public class ITestS3GuardFsck extends AbstractS3ATestBase {
 
       assertComparePairsSize(comparePairs, 1);
 
-      // check fil1 that there's the violation
+      // check if the violation is there
       checkForViolationInPairs(file, comparePairs,
           S3GuardFsck.Violation.TOMBSTONED_IN_MS_NOT_DELETED_IN_S3);
-      // check the child that there's no NO_ETAG violation
+    } finally {
+      cleanup(file, cwd);
+    }
+  }
+
+  @Test
+  public void checkDdbInternalConsistency() throws Exception {
+    final S3GuardFsck s3GuardFsck = new S3GuardFsck(rawFs, metadataStore);
+    final DynamoDBMetadataStore ms =
+        (DynamoDBMetadataStore) guardedFs.getMetadataStore();
+    s3GuardFsck.checkDdbInternalConsistency(
+        new Path("s3a://" + guardedFs.getBucket() + "/"));
+  }
+
+  @Test
+  public void testDdbInternalNoLastUpdatedField() throws Exception {
+    final Path cwd = path("/" + getMethodName() + "-" + UUID.randomUUID());
+    final Path file = new Path(cwd, "file");
+    try {
+      final S3AFileStatus s3AFileStatus = new S3AFileStatus(100, 100, file, 100,
+          "test", "etag", "version");
+      final PathMetadata pathMetadata = new PathMetadata(s3AFileStatus);
+      pathMetadata.setLastUpdated(0);
+      metadataStore.put(pathMetadata);
+
+      final S3GuardFsck s3GuardFsck = new S3GuardFsck(rawFs, metadataStore);
+      final List<S3GuardFsck.ComparePair> comparePairs =
+          s3GuardFsck.checkDdbInternalConsistency(cwd);
+
+      assertComparePairsSize(comparePairs, 1);
+
+      // check if the violation is there
+      checkForViolationInPairs(file, comparePairs,
+          S3GuardFsck.Violation.NO_LASTUPDATED_FIELD);
+    } finally {
+      cleanup(file, cwd);
+    }
+  }
+
+  @Test
+  public void testDdbInternalOrphanEntry() throws Exception {
+    final Path cwd = path("/" + getMethodName() + "-" + UUID.randomUUID());
+    final Path parentDir = new Path(cwd, "directory");
+    final Path file = new Path(parentDir, "file");
+    try {
+      final S3AFileStatus s3AFileStatus = new S3AFileStatus(100, 100, file, 100,
+          "test", "etag", "version");
+      final PathMetadata pathMetadata = new PathMetadata(s3AFileStatus);
+      pathMetadata.setLastUpdated(1000);
+      metadataStore.put(pathMetadata);
+      metadataStore.forgetMetadata(parentDir);
+
+      final S3GuardFsck s3GuardFsck = new S3GuardFsck(rawFs, metadataStore);
+      final List<S3GuardFsck.ComparePair> comparePairs =
+          s3GuardFsck.checkDdbInternalConsistency(cwd);
+
+      // check if the violation is there
+      assertComparePairsSize(comparePairs, 1);
+      checkForViolationInPairs(file, comparePairs,
+          S3GuardFsck.Violation.ORPHAN_DDB_ENTRY);
+    } finally {
+      cleanup(file, cwd);
+    }
+  }
+
+  @Test
+  public void testDdbInternalParentIsAFile() throws Exception {
+    final Path cwd = path("/" + getMethodName() + "-" + UUID.randomUUID());
+    final Path parentDir = new Path(cwd, "directory");
+    final Path file = new Path(parentDir, "file");
+    try {
+      final S3AFileStatus s3AFileStatus = new S3AFileStatus(100, 100, file, 100,
+          "test", "etag", "version");
+      final PathMetadata pathMetadata = new PathMetadata(s3AFileStatus);
+      pathMetadata.setLastUpdated(1000);
+      metadataStore.put(pathMetadata);
+
+      final S3AFileStatus dirAsFile = MetadataStoreTestBase
+          .basicFileStatus(parentDir, 1, false, 1);
+      final PathMetadata dirAsFilePm = new PathMetadata(dirAsFile);
+      dirAsFilePm.setLastUpdated(100);
+      metadataStore.put(dirAsFilePm);
+
+      final S3GuardFsck s3GuardFsck = new S3GuardFsck(rawFs, metadataStore);
+      final List<S3GuardFsck.ComparePair> comparePairs =
+          s3GuardFsck.checkDdbInternalConsistency(cwd);
+
+      // check if the violation is there
+      assertComparePairsSize(comparePairs, 1);
+      checkForViolationInPairs(file, comparePairs,
+          S3GuardFsck.Violation.PARENT_IS_A_FILE);
+    } finally {
+      cleanup(file, cwd);
+    }
+  }
+
+  @Test
+  public void testDdbInternalParentTombstoned() throws Exception {
+    final Path cwd = path("/" + getMethodName() + "-" + UUID.randomUUID());
+    final Path parentDir = new Path(cwd, "directory");
+    final Path file = new Path(parentDir, "file");
+    try {
+      final S3AFileStatus s3AFileStatus = new S3AFileStatus(100, 100, file, 100,
+          "test", "etag", "version");
+      final PathMetadata pathMetadata = new PathMetadata(s3AFileStatus);
+      pathMetadata.setLastUpdated(1000);
+      metadataStore.put(pathMetadata);
+      metadataStore.delete(parentDir, null);
+
+      final S3GuardFsck s3GuardFsck = new S3GuardFsck(rawFs, metadataStore);
+      final List<S3GuardFsck.ComparePair> comparePairs =
+          s3GuardFsck.checkDdbInternalConsistency(cwd);
+
+      // check if the violation is there
+      assertComparePairsSize(comparePairs, 1);
+      checkForViolationInPairs(file, comparePairs,
+          S3GuardFsck.Violation.PARENT_TOMBSTONED);
     } finally {
       cleanup(file, cwd);
     }

+ 32 - 3
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java

@@ -304,11 +304,11 @@ public class ITestS3GuardToolDynamoDB extends AbstractS3GuardToolTestBase {
 
   @Test
   public void testCLIFsckWithParam() throws Exception {
+    LOG.info("This test serves the purpose to run fsck with the correct " +
+        "parameters, so there will be no exception thrown.");
     final int result = run(S3GuardTool.Fsck.NAME, "-check",
         "s3a://" + getFileSystem().getBucket());
-    LOG.info("This test serves the purpose to run fsck with the correct " +
-        "parameters, so there will be no exception thrown. " +
-        "The return value of the run: {}", result);
+    LOG.info("The return value of the run: {}", result);
   }
 
   @Test
@@ -324,4 +324,33 @@ public class ITestS3GuardToolDynamoDB extends AbstractS3GuardToolTestBase {
         () -> run(S3GuardTool.Fsck.NAME, "-check",
             "s3a://this-bucket-does-not-exist-" + UUID.randomUUID()));
   }
+
+  @Test
+  public void testCLIFsckDDbInternalWrongS3APath() throws Exception {
+    intercept(FileNotFoundException.class, "wrong path",
+        () -> run(S3GuardTool.Fsck.NAME, "-"+Fsck.DDB_MS_CONSISTENCY_FLAG,
+            "s3a://" + getFileSystem().getBucket() + "/" + UUID.randomUUID()));
+  }
+
+  @Test
+  public void testCLIFsckDDbInternalParam() throws Exception {
+    describe("This test serves the purpose to run fsck with the correct " +
+        "parameters, so there will be no exception thrown.");
+    final int result = run(S3GuardTool.Fsck.NAME,
+        "-" + Fsck.DDB_MS_CONSISTENCY_FLAG,
+        "s3a://" + getFileSystem().getBucket());
+    LOG.info("The return value of the run: {}", result);
+  }
+
+  @Test
+  public void testCLIFsckCheckExclusive() throws Exception {
+    describe("There should be only one check param when running fsck." +
+        "If more then one param is passed, the command should fail." +
+        "This provide exclusive run for checks so the user is able to define " +
+        "the order of checking.");
+    intercept(ExitUtil.ExitException.class, "only one parameter",
+        () -> run(S3GuardTool.Fsck.NAME,
+        "-" + Fsck.DDB_MS_CONSISTENCY_FLAG, "-" + Fsck.CHECK_FLAG,
+        "s3a://" + getFileSystem().getBucket()));
+  }
 }