|
@@ -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;
|